risingwave_meta/controller/
streaming_job.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
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet};
use std::num::NonZeroUsize;

use anyhow::anyhow;
use itertools::Itertools;
use risingwave_common::hash::VnodeCountCompat;
use risingwave_common::util::column_index_mapping::ColIndexMapping;
use risingwave_common::util::stream_graph_visitor::visit_stream_node;
use risingwave_common::{bail, current_cluster_version};
use risingwave_connector::WithPropertiesExt;
use risingwave_meta_model::actor::ActorStatus;
use risingwave_meta_model::actor_dispatcher::DispatcherType;
use risingwave_meta_model::object::ObjectType;
use risingwave_meta_model::prelude::{
    Actor, ActorDispatcher, Fragment, Index, Object, ObjectDependency, Sink, Source,
    StreamingJob as StreamingJobModel, Table,
};
use risingwave_meta_model::table::TableType;
use risingwave_meta_model::{
    actor, actor_dispatcher, fragment, index, object, object_dependency, sink, source,
    streaming_job, table, ActorId, ActorUpstreamActors, ColumnCatalogArray, CreateType, DatabaseId,
    ExprNodeArray, FragmentId, I32Array, IndexId, JobStatus, ObjectId, SchemaId, SinkId, SourceId,
    StreamNode, StreamingParallelism, TableId, TableVersion, UserId,
};
use risingwave_pb::catalog::source::PbOptionalAssociatedTableId;
use risingwave_pb::catalog::table::{PbOptionalAssociatedSourceId, PbTableVersion};
use risingwave_pb::catalog::{PbCreateType, PbTable};
use risingwave_pb::meta::list_rate_limits_response::RateLimitInfo;
use risingwave_pb::meta::relation::{PbRelationInfo, RelationInfo};
use risingwave_pb::meta::subscribe_response::{
    Info as NotificationInfo, Info, Operation as NotificationOperation, Operation,
};
use risingwave_pb::meta::{
    PbFragmentWorkerSlotMapping, PbRelation, PbRelationGroup, Relation, RelationGroup,
};
use risingwave_pb::source::{PbConnectorSplit, PbConnectorSplits};
use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::update_mutation::PbMergeUpdate;
use risingwave_pb::stream_plan::{
    PbDispatcher, PbDispatcherType, PbFragmentTypeFlag, PbStreamActor,
};
use sea_orm::sea_query::{BinOper, Expr, Query, SimpleExpr};
use sea_orm::ActiveValue::Set;
use sea_orm::{
    ActiveEnum, ActiveModelTrait, ColumnTrait, DatabaseTransaction, EntityTrait, IntoActiveModel,
    IntoSimpleExpr, JoinType, ModelTrait, NotSet, PaginatorTrait, QueryFilter, QuerySelect,
    RelationTrait, TransactionTrait,
};

use crate::barrier::{ReplaceTablePlan, Reschedule};
use crate::controller::catalog::CatalogController;
use crate::controller::rename::ReplaceTableExprRewriter;
use crate::controller::utils::{
    build_relation_group_for_delete, check_relation_name_duplicate, check_sink_into_table_cycle,
    ensure_object_id, ensure_user_id, get_fragment_actor_ids, get_fragment_mappings,
    get_internal_tables_by_id, rebuild_fragment_mapping_from_actors, PartialObject,
};
use crate::controller::ObjectModel;
use crate::manager::{NotificationVersion, StreamingJob};
use crate::model::{StreamContext, StreamJobFragments, TableParallelism};
use crate::stream::SplitAssignment;
use crate::{MetaError, MetaResult};

impl CatalogController {
    pub async fn create_streaming_job_obj(
        txn: &DatabaseTransaction,
        obj_type: ObjectType,
        owner_id: UserId,
        database_id: Option<DatabaseId>,
        schema_id: Option<SchemaId>,
        create_type: PbCreateType,
        ctx: &StreamContext,
        streaming_parallelism: StreamingParallelism,
        max_parallelism: usize,
    ) -> MetaResult<ObjectId> {
        let obj = Self::create_object(txn, obj_type, owner_id, database_id, schema_id).await?;
        let job = streaming_job::ActiveModel {
            job_id: Set(obj.oid),
            job_status: Set(JobStatus::Initial),
            create_type: Set(create_type.into()),
            timezone: Set(ctx.timezone.clone()),
            parallelism: Set(streaming_parallelism),
            max_parallelism: Set(max_parallelism as _),
        };
        job.insert(txn).await?;

        Ok(obj.oid)
    }

    /// Create catalogs for the streaming job, then notify frontend about them if the job is a
    /// materialized view.
    ///
    /// Some of the fields in the given streaming job are placeholders, which will
    /// be updated later in `prepare_streaming_job` and notify again in `finish_streaming_job`.
    pub async fn create_job_catalog(
        &self,
        streaming_job: &mut StreamingJob,
        ctx: &StreamContext,
        parallelism: &Option<Parallelism>,
        max_parallelism: usize,
        mut dependencies: HashSet<ObjectId>,
    ) -> MetaResult<()> {
        let inner = self.inner.write().await;
        let txn = inner.db.begin().await?;
        let create_type = streaming_job.create_type();

        let streaming_parallelism = match parallelism {
            None => StreamingParallelism::Adaptive,
            Some(n) => StreamingParallelism::Fixed(n.parallelism as _),
        };

        ensure_user_id(streaming_job.owner() as _, &txn).await?;
        ensure_object_id(ObjectType::Database, streaming_job.database_id() as _, &txn).await?;
        ensure_object_id(ObjectType::Schema, streaming_job.schema_id() as _, &txn).await?;
        check_relation_name_duplicate(
            &streaming_job.name(),
            streaming_job.database_id() as _,
            streaming_job.schema_id() as _,
            &txn,
        )
        .await?;

        // TODO(rc): pass all dependencies uniformly, deprecate `dependent_relations` and `dependent_secret_ids`.
        dependencies.extend(
            streaming_job
                .dependent_relations()
                .into_iter()
                .map(|id| id as ObjectId),
        );

        // check if any dependency is in altering status.
        if !dependencies.is_empty() {
            let altering_cnt = ObjectDependency::find()
                .join(
                    JoinType::InnerJoin,
                    object_dependency::Relation::Object1.def(),
                )
                .join(JoinType::InnerJoin, object::Relation::StreamingJob.def())
                .filter(
                    object_dependency::Column::Oid
                        .is_in(dependencies.clone())
                        .and(object::Column::ObjType.eq(ObjectType::Table))
                        .and(streaming_job::Column::JobStatus.ne(JobStatus::Created))
                        .and(
                            // It means the referring table is just dummy for altering.
                            object::Column::Oid.not_in_subquery(
                                Query::select()
                                    .column(table::Column::TableId)
                                    .from(Table)
                                    .to_owned(),
                            ),
                        ),
                )
                .count(&txn)
                .await?;
            if altering_cnt != 0 {
                return Err(MetaError::permission_denied(
                    "some dependent relations are being altered",
                ));
            }
        }

        let mut relations = vec![];

        match streaming_job {
            StreamingJob::MaterializedView(table) => {
                let job_id = Self::create_streaming_job_obj(
                    &txn,
                    ObjectType::Table,
                    table.owner as _,
                    Some(table.database_id as _),
                    Some(table.schema_id as _),
                    create_type,
                    ctx,
                    streaming_parallelism,
                    max_parallelism,
                )
                .await?;
                table.id = job_id as _;
                let table_model: table::ActiveModel = table.clone().into();
                Table::insert(table_model).exec(&txn).await?;

                relations.push(Relation {
                    relation_info: Some(RelationInfo::Table(table.to_owned())),
                });
            }
            StreamingJob::Sink(sink, _) => {
                if let Some(target_table_id) = sink.target_table {
                    if check_sink_into_table_cycle(
                        target_table_id as ObjectId,
                        dependencies.iter().cloned().collect(),
                        &txn,
                    )
                    .await?
                    {
                        bail!("Creating such a sink will result in circular dependency.");
                    }
                }

                let job_id = Self::create_streaming_job_obj(
                    &txn,
                    ObjectType::Sink,
                    sink.owner as _,
                    Some(sink.database_id as _),
                    Some(sink.schema_id as _),
                    create_type,
                    ctx,
                    streaming_parallelism,
                    max_parallelism,
                )
                .await?;
                sink.id = job_id as _;
                let sink_model: sink::ActiveModel = sink.clone().into();
                Sink::insert(sink_model).exec(&txn).await?;
            }
            StreamingJob::Table(src, table, _) => {
                let job_id = Self::create_streaming_job_obj(
                    &txn,
                    ObjectType::Table,
                    table.owner as _,
                    Some(table.database_id as _),
                    Some(table.schema_id as _),
                    create_type,
                    ctx,
                    streaming_parallelism,
                    max_parallelism,
                )
                .await?;
                table.id = job_id as _;
                if let Some(src) = src {
                    let src_obj = Self::create_object(
                        &txn,
                        ObjectType::Source,
                        src.owner as _,
                        Some(src.database_id as _),
                        Some(src.schema_id as _),
                    )
                    .await?;
                    src.id = src_obj.oid as _;
                    src.optional_associated_table_id =
                        Some(PbOptionalAssociatedTableId::AssociatedTableId(job_id as _));
                    table.optional_associated_source_id = Some(
                        PbOptionalAssociatedSourceId::AssociatedSourceId(src_obj.oid as _),
                    );
                    let source: source::ActiveModel = src.clone().into();
                    Source::insert(source).exec(&txn).await?;
                }
                let table_model: table::ActiveModel = table.clone().into();
                Table::insert(table_model).exec(&txn).await?;
            }
            StreamingJob::Index(index, table) => {
                ensure_object_id(ObjectType::Table, index.primary_table_id as _, &txn).await?;
                let job_id = Self::create_streaming_job_obj(
                    &txn,
                    ObjectType::Index,
                    index.owner as _,
                    Some(index.database_id as _),
                    Some(index.schema_id as _),
                    create_type,
                    ctx,
                    streaming_parallelism,
                    max_parallelism,
                )
                .await?;
                // to be compatible with old implementation.
                index.id = job_id as _;
                index.index_table_id = job_id as _;
                table.id = job_id as _;

                ObjectDependency::insert(object_dependency::ActiveModel {
                    oid: Set(index.primary_table_id as _),
                    used_by: Set(table.id as _),
                    ..Default::default()
                })
                .exec(&txn)
                .await?;

                let table_model: table::ActiveModel = table.clone().into();
                Table::insert(table_model).exec(&txn).await?;
                let index_model: index::ActiveModel = index.clone().into();
                Index::insert(index_model).exec(&txn).await?;
            }
            StreamingJob::Source(src) => {
                let job_id = Self::create_streaming_job_obj(
                    &txn,
                    ObjectType::Source,
                    src.owner as _,
                    Some(src.database_id as _),
                    Some(src.schema_id as _),
                    create_type,
                    ctx,
                    streaming_parallelism,
                    max_parallelism,
                )
                .await?;
                src.id = job_id as _;
                let source_model: source::ActiveModel = src.clone().into();
                Source::insert(source_model).exec(&txn).await?;
            }
        }

        // collect dependent secrets.
        dependencies.extend(
            streaming_job
                .dependent_secret_ids()?
                .into_iter()
                .map(|secret_id| secret_id as ObjectId),
        );

        // record object dependency.
        if !dependencies.is_empty() {
            ObjectDependency::insert_many(dependencies.into_iter().map(|oid| {
                object_dependency::ActiveModel {
                    oid: Set(oid),
                    used_by: Set(streaming_job.id() as _),
                    ..Default::default()
                }
            }))
            .exec(&txn)
            .await?;
        }

        txn.commit().await?;

        if !relations.is_empty() {
            self.notify_frontend(
                Operation::Add,
                Info::RelationGroup(RelationGroup { relations }),
            )
            .await;
        }

        Ok(())
    }

    /// Create catalogs for internal tables, then notify frontend about them if the job is a
    /// materialized view.
    ///
    /// Some of the fields in the given "incomplete" internal tables are placeholders, which will
    /// be updated later in `prepare_streaming_job` and notify again in `finish_streaming_job`.
    ///
    /// Returns a mapping from the temporary table id to the actual global table id.
    pub async fn create_internal_table_catalog(
        &self,
        job: &StreamingJob,
        mut incomplete_internal_tables: Vec<PbTable>,
    ) -> MetaResult<HashMap<u32, u32>> {
        let job_id = job.id() as ObjectId;
        let inner = self.inner.write().await;
        let txn = inner.db.begin().await?;
        let mut table_id_map = HashMap::new();
        for table in &mut incomplete_internal_tables {
            let table_id = Self::create_object(
                &txn,
                ObjectType::Table,
                table.owner as _,
                Some(table.database_id as _),
                Some(table.schema_id as _),
            )
            .await?
            .oid;
            table_id_map.insert(table.id, table_id as u32);
            table.id = table_id as _;

            let table_model = table::ActiveModel {
                table_id: Set(table_id as _),
                belongs_to_job_id: Set(Some(job_id)),
                fragment_id: NotSet,
                ..table.clone().into()
            };
            Table::insert(table_model).exec(&txn).await?;
        }
        txn.commit().await?;

        if job.is_materialized_view() {
            self.notify_frontend(
                Operation::Add,
                Info::RelationGroup(RelationGroup {
                    relations: incomplete_internal_tables
                        .iter()
                        .map(|table| Relation {
                            relation_info: Some(RelationInfo::Table(table.clone())),
                        })
                        .collect(),
                }),
            )
            .await;
        }

        Ok(table_id_map)
    }

    // TODO: In this function, we also update the `Table` model in the meta store.
    // Given that we've ensured the tables inside `TableFragments` are complete, shall we consider
    // making them the source of truth and performing a full replacement for those in the meta store?
    pub async fn prepare_streaming_job(
        &self,
        stream_job_fragments: &StreamJobFragments,
        streaming_job: &StreamingJob,
        for_replace: bool,
    ) -> MetaResult<()> {
        let fragment_actors =
            Self::extract_fragment_and_actors_from_fragments(stream_job_fragments.to_protobuf())?;
        let all_tables = stream_job_fragments.all_tables();
        let inner = self.inner.write().await;
        let txn = inner.db.begin().await?;

        // Add fragments.
        let (fragments, actor_with_dispatchers): (Vec<_>, Vec<_>) = fragment_actors
            .into_iter()
            .map(|(fragment, actors, actor_dispatchers)| (fragment, (actors, actor_dispatchers)))
            .unzip();
        for fragment in fragments {
            let fragment_id = fragment.fragment_id;
            let state_table_ids = fragment.state_table_ids.inner_ref().clone();

            let fragment = fragment.into_active_model();
            Fragment::insert(fragment).exec(&txn).await?;

            // Fields including `fragment_id` and `vnode_count` were placeholder values before.
            // After table fragments are created, update them for all internal tables.
            if !for_replace {
                for state_table_id in state_table_ids {
                    // Table's vnode count is not always the fragment's vnode count, so we have to
                    // look up the table from `TableFragments`.
                    // See `ActorGraphBuilder::new`.
                    let table = all_tables
                        .get(&(state_table_id as u32))
                        .unwrap_or_else(|| panic!("table {} not found", state_table_id));
                    assert_eq!(table.fragment_id, fragment_id as u32);
                    let vnode_count = table.vnode_count();

                    table::ActiveModel {
                        table_id: Set(state_table_id as _),
                        fragment_id: Set(Some(fragment_id)),
                        vnode_count: Set(vnode_count as _),
                        ..Default::default()
                    }
                    .update(&txn)
                    .await?;
                }
            }
        }

        // Add actors and actor dispatchers.
        for (actors, actor_dispatchers) in actor_with_dispatchers {
            for actor in actors {
                let actor = actor.into_active_model();
                Actor::insert(actor).exec(&txn).await?;
            }
            for (_, actor_dispatchers) in actor_dispatchers {
                for actor_dispatcher in actor_dispatchers {
                    let mut actor_dispatcher = actor_dispatcher.into_active_model();
                    actor_dispatcher.id = NotSet;
                    ActorDispatcher::insert(actor_dispatcher).exec(&txn).await?;
                }
            }
        }

        if !for_replace {
            // // Update dml fragment id.
            if let StreamingJob::Table(_, table, ..) = streaming_job {
                Table::update(table::ActiveModel {
                    table_id: Set(table.id as _),
                    dml_fragment_id: Set(table.dml_fragment_id.map(|id| id as _)),
                    ..Default::default()
                })
                .exec(&txn)
                .await?;
            }
        }

        txn.commit().await?;

        Ok(())
    }

    /// `try_abort_creating_streaming_job` is used to abort the job that is under initial status or in `FOREGROUND` mode.
    /// It returns (true, _) if the job is not found or aborted.
    /// It returns (_, Some(`database_id`)) is the `database_id` of the `job_id` exists
    pub async fn try_abort_creating_streaming_job(
        &self,
        job_id: ObjectId,
        is_cancelled: bool,
    ) -> MetaResult<(bool, Option<DatabaseId>)> {
        let mut inner = self.inner.write().await;
        let txn = inner.db.begin().await?;

        let obj = Object::find_by_id(job_id).one(&txn).await?;
        let Some(obj) = obj else {
            tracing::warn!(
                id = job_id,
                "streaming job not found when aborting creating, might be cleaned by recovery"
            );
            return Ok((true, None));
        };
        let database_id = obj
            .database_id
            .ok_or_else(|| anyhow!("obj has no database id: {:?}", obj))?;

        if !is_cancelled {
            let streaming_job = streaming_job::Entity::find_by_id(job_id).one(&txn).await?;
            if let Some(streaming_job) = streaming_job {
                assert_ne!(streaming_job.job_status, JobStatus::Created);
                if streaming_job.create_type == CreateType::Background
                    && streaming_job.job_status == JobStatus::Creating
                {
                    // If the job is created in background and still in creating status, we should not abort it and let recovery to handle it.
                    tracing::warn!(
                        id = job_id,
                        "streaming job is created in background and still in creating status"
                    );
                    return Ok((false, Some(database_id)));
                }
            }
        }

        let internal_table_ids = get_internal_tables_by_id(job_id, &txn).await?;

        // Get the notification info if the job is a materialized view.
        let table_obj = Table::find_by_id(job_id).one(&txn).await?;
        let mut objs = vec![];
        if let Some(table) = &table_obj
            && table.table_type == TableType::MaterializedView
        {
            let obj: Option<PartialObject> = Object::find_by_id(job_id)
                .select_only()
                .columns([
                    object::Column::Oid,
                    object::Column::ObjType,
                    object::Column::SchemaId,
                    object::Column::DatabaseId,
                ])
                .into_partial_model()
                .one(&txn)
                .await?;
            let obj =
                obj.ok_or_else(|| MetaError::catalog_id_not_found("streaming job", job_id))?;
            objs.push(obj);
            let internal_table_objs: Vec<PartialObject> = Object::find()
                .select_only()
                .columns([
                    object::Column::Oid,
                    object::Column::ObjType,
                    object::Column::SchemaId,
                    object::Column::DatabaseId,
                ])
                .join(JoinType::InnerJoin, object::Relation::Table.def())
                .filter(table::Column::BelongsToJobId.eq(job_id))
                .into_partial_model()
                .all(&txn)
                .await?;
            objs.extend(internal_table_objs);
        }

        Object::delete_by_id(job_id).exec(&txn).await?;
        if !internal_table_ids.is_empty() {
            Object::delete_many()
                .filter(object::Column::Oid.is_in(internal_table_ids))
                .exec(&txn)
                .await?;
        }
        if let Some(t) = &table_obj
            && let Some(source_id) = t.optional_associated_source_id
        {
            Object::delete_by_id(source_id).exec(&txn).await?;
        }

        for tx in inner
            .creating_table_finish_notifier
            .remove(&job_id)
            .into_iter()
            .flatten()
        {
            let err = if is_cancelled {
                MetaError::cancelled(format!("streaming job {job_id} is cancelled"))
            } else {
                MetaError::catalog_id_not_found(
                    "stream job",
                    format!("streaming job {job_id} failed"),
                )
            };
            let _ = tx.send(Err(err));
        }
        txn.commit().await?;

        if !objs.is_empty() {
            // We also have notified the frontend about these objects,
            // so we need to notify the frontend to delete them here.
            self.notify_frontend(Operation::Delete, build_relation_group_for_delete(objs))
                .await;
        }
        Ok((true, Some(database_id)))
    }

    pub async fn post_collect_job_fragments(
        &self,
        job_id: ObjectId,
        actor_ids: Vec<crate::model::ActorId>,
        new_actor_dispatchers: HashMap<crate::model::ActorId, Vec<PbDispatcher>>,
        split_assignment: &SplitAssignment,
    ) -> MetaResult<()> {
        let inner = self.inner.write().await;
        let txn = inner.db.begin().await?;

        Actor::update_many()
            .col_expr(
                actor::Column::Status,
                SimpleExpr::from(ActorStatus::Running.into_value()),
            )
            .filter(
                actor::Column::ActorId
                    .is_in(actor_ids.into_iter().map(|id| id as ActorId).collect_vec()),
            )
            .exec(&txn)
            .await?;

        for splits in split_assignment.values() {
            for (actor_id, splits) in splits {
                let splits = splits.iter().map(PbConnectorSplit::from).collect_vec();
                let connector_splits = &PbConnectorSplits { splits };
                actor::ActiveModel {
                    actor_id: Set(*actor_id as _),
                    splits: Set(Some(connector_splits.into())),
                    ..Default::default()
                }
                .update(&txn)
                .await?;
            }
        }

        let mut actor_dispatchers = vec![];
        for (actor_id, dispatchers) in new_actor_dispatchers {
            for dispatcher in dispatchers {
                let mut actor_dispatcher =
                    actor_dispatcher::Model::from((actor_id, dispatcher)).into_active_model();
                actor_dispatcher.id = NotSet;
                actor_dispatchers.push(actor_dispatcher);
            }
        }

        if !actor_dispatchers.is_empty() {
            ActorDispatcher::insert_many(actor_dispatchers)
                .exec(&txn)
                .await?;
        }

        // Mark job as CREATING.
        streaming_job::ActiveModel {
            job_id: Set(job_id),
            job_status: Set(JobStatus::Creating),
            ..Default::default()
        }
        .update(&txn)
        .await?;

        txn.commit().await?;

        Ok(())
    }

    pub async fn create_job_catalog_for_replace(
        &self,
        streaming_job: &StreamingJob,
        ctx: &StreamContext,
        version: &PbTableVersion,
        specified_parallelism: &Option<NonZeroUsize>,
        max_parallelism: usize,
    ) -> MetaResult<ObjectId> {
        let id = streaming_job.id();
        let inner = self.inner.write().await;
        let txn = inner.db.begin().await?;

        // 1. check version.
        let original_version: Option<TableVersion> = Table::find_by_id(id as TableId)
            .select_only()
            .column(table::Column::Version)
            .into_tuple()
            .one(&txn)
            .await?
            .ok_or_else(|| MetaError::catalog_id_not_found(ObjectType::Table.as_str(), id))?;
        let original_version = original_version.expect("version for table should exist");
        if version.version != original_version.to_protobuf().version + 1 {
            return Err(MetaError::permission_denied("table version is stale"));
        }

        // 2. check concurrent replace.
        let referring_cnt = ObjectDependency::find()
            .join(
                JoinType::InnerJoin,
                object_dependency::Relation::Object1.def(),
            )
            .join(JoinType::InnerJoin, object::Relation::StreamingJob.def())
            .filter(
                object_dependency::Column::Oid
                    .eq(id as ObjectId)
                    .and(object::Column::ObjType.eq(ObjectType::Table))
                    .and(streaming_job::Column::JobStatus.ne(JobStatus::Created)),
            )
            .count(&txn)
            .await?;
        if referring_cnt != 0 {
            return Err(MetaError::permission_denied(
                "table is being altered or referenced by some creating jobs",
            ));
        }

        // 3. check parallelism.
        let original_max_parallelism: i32 = StreamingJobModel::find_by_id(id as ObjectId)
            .select_only()
            .column(streaming_job::Column::MaxParallelism)
            .into_tuple()
            .one(&txn)
            .await?
            .ok_or_else(|| MetaError::catalog_id_not_found(ObjectType::Table.as_str(), id))?;

        if original_max_parallelism != max_parallelism as i32 {
            // We already override the max parallelism in `StreamFragmentGraph` before entering this function.
            // This should not happen in normal cases.
            bail!(
                "cannot use a different max parallelism \
                 when altering or creating/dropping a sink into an existing table, \
                 original: {}, new: {}",
                original_max_parallelism,
                max_parallelism
            );
        }

        let parallelism = match specified_parallelism {
            None => StreamingParallelism::Adaptive,
            Some(n) => StreamingParallelism::Fixed(n.get() as _),
        };

        // 4. create streaming object for new replace table.
        let obj_id = Self::create_streaming_job_obj(
            &txn,
            ObjectType::Table,
            streaming_job.owner() as _,
            Some(streaming_job.database_id() as _),
            Some(streaming_job.schema_id() as _),
            PbCreateType::Foreground,
            ctx,
            parallelism,
            max_parallelism,
        )
        .await?;

        // 5. record dependency for new replace table.
        ObjectDependency::insert(object_dependency::ActiveModel {
            oid: Set(id as _),
            used_by: Set(obj_id as _),
            ..Default::default()
        })
        .exec(&txn)
        .await?;

        txn.commit().await?;

        Ok(obj_id)
    }

    /// `finish_streaming_job` marks job related objects as `Created` and notify frontend.
    pub async fn finish_streaming_job(
        &self,
        job_id: ObjectId,
        replace_table_job_info: Option<ReplaceTablePlan>,
    ) -> MetaResult<()> {
        let mut inner = self.inner.write().await;
        let txn = inner.db.begin().await?;

        let job_type = Object::find_by_id(job_id)
            .select_only()
            .column(object::Column::ObjType)
            .into_tuple()
            .one(&txn)
            .await?
            .ok_or_else(|| MetaError::catalog_id_not_found("streaming job", job_id))?;

        // update `created_at` as now() and `created_at_cluster_version` as current cluster version.
        let res = Object::update_many()
            .col_expr(object::Column::CreatedAt, Expr::current_timestamp().into())
            .col_expr(
                object::Column::CreatedAtClusterVersion,
                current_cluster_version().into(),
            )
            .filter(object::Column::Oid.eq(job_id))
            .exec(&txn)
            .await?;
        if res.rows_affected == 0 {
            return Err(MetaError::catalog_id_not_found("streaming job", job_id));
        }

        // mark the target stream job as `Created`.
        let job = streaming_job::ActiveModel {
            job_id: Set(job_id),
            job_status: Set(JobStatus::Created),
            ..Default::default()
        };
        job.update(&txn).await?;

        // notify frontend: job, internal tables.
        let internal_table_objs = Table::find()
            .find_also_related(Object)
            .filter(table::Column::BelongsToJobId.eq(job_id))
            .all(&txn)
            .await?;
        let mut relations = internal_table_objs
            .iter()
            .map(|(table, obj)| PbRelation {
                relation_info: Some(PbRelationInfo::Table(
                    ObjectModel(table.clone(), obj.clone().unwrap()).into(),
                )),
            })
            .collect_vec();
        let mut notification_op = NotificationOperation::Add;

        match job_type {
            ObjectType::Table => {
                let (table, obj) = Table::find_by_id(job_id)
                    .find_also_related(Object)
                    .one(&txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found("table", job_id))?;
                if table.table_type == TableType::MaterializedView {
                    notification_op = NotificationOperation::Update;
                }

                if let Some(source_id) = table.optional_associated_source_id {
                    let (src, obj) = Source::find_by_id(source_id)
                        .find_also_related(Object)
                        .one(&txn)
                        .await?
                        .ok_or_else(|| MetaError::catalog_id_not_found("source", source_id))?;
                    relations.push(PbRelation {
                        relation_info: Some(PbRelationInfo::Source(
                            ObjectModel(src, obj.unwrap()).into(),
                        )),
                    });
                }
                relations.push(PbRelation {
                    relation_info: Some(PbRelationInfo::Table(
                        ObjectModel(table, obj.unwrap()).into(),
                    )),
                });
            }
            ObjectType::Sink => {
                let (sink, obj) = Sink::find_by_id(job_id)
                    .find_also_related(Object)
                    .one(&txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found("sink", job_id))?;
                relations.push(PbRelation {
                    relation_info: Some(PbRelationInfo::Sink(
                        ObjectModel(sink, obj.unwrap()).into(),
                    )),
                });
            }
            ObjectType::Index => {
                let (index, obj) = Index::find_by_id(job_id)
                    .find_also_related(Object)
                    .one(&txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found("index", job_id))?;
                {
                    let (table, obj) = Table::find_by_id(index.index_table_id)
                        .find_also_related(Object)
                        .one(&txn)
                        .await?
                        .ok_or_else(|| {
                            MetaError::catalog_id_not_found("table", index.index_table_id)
                        })?;
                    relations.push(PbRelation {
                        relation_info: Some(PbRelationInfo::Table(
                            ObjectModel(table, obj.unwrap()).into(),
                        )),
                    });
                }
                relations.push(PbRelation {
                    relation_info: Some(PbRelationInfo::Index(
                        ObjectModel(index, obj.unwrap()).into(),
                    )),
                });
            }
            ObjectType::Source => {
                let (source, obj) = Source::find_by_id(job_id)
                    .find_also_related(Object)
                    .one(&txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found("source", job_id))?;
                relations.push(PbRelation {
                    relation_info: Some(PbRelationInfo::Source(
                        ObjectModel(source, obj.unwrap()).into(),
                    )),
                });
            }
            _ => unreachable!("invalid job type: {:?}", job_type),
        }

        let fragment_mapping = get_fragment_mappings(&txn, job_id).await?;

        let replace_table_mapping_update = match replace_table_job_info {
            Some(ReplaceTablePlan {
                streaming_job,
                merge_updates,
                tmp_id,
                ..
            }) => {
                let incoming_sink_id = job_id;

                let (relations, fragment_mapping) = Self::finish_replace_streaming_job_inner(
                    tmp_id as ObjectId,
                    merge_updates,
                    None,
                    Some(incoming_sink_id as _),
                    None,
                    vec![],
                    &txn,
                    streaming_job,
                )
                .await?;

                Some((relations, fragment_mapping))
            }
            None => None,
        };

        txn.commit().await?;

        self.notify_fragment_mapping(NotificationOperation::Add, fragment_mapping)
            .await;

        let mut version = self
            .notify_frontend(
                notification_op,
                NotificationInfo::RelationGroup(PbRelationGroup { relations }),
            )
            .await;

        if let Some((relations, fragment_mapping)) = replace_table_mapping_update {
            self.notify_fragment_mapping(NotificationOperation::Add, fragment_mapping)
                .await;
            version = self
                .notify_frontend(
                    NotificationOperation::Update,
                    NotificationInfo::RelationGroup(PbRelationGroup { relations }),
                )
                .await;
        }
        if let Some(txs) = inner.creating_table_finish_notifier.remove(&job_id) {
            for tx in txs {
                let _ = tx.send(Ok(version));
            }
        }

        Ok(())
    }

    pub async fn finish_replace_streaming_job(
        &self,
        tmp_id: ObjectId,
        streaming_job: StreamingJob,
        merge_updates: Vec<PbMergeUpdate>,
        table_col_index_mapping: Option<ColIndexMapping>,
        creating_sink_id: Option<SinkId>,
        dropping_sink_id: Option<SinkId>,
        updated_sink_catalogs: Vec<SinkId>,
    ) -> MetaResult<NotificationVersion> {
        let inner = self.inner.write().await;
        let txn = inner.db.begin().await?;

        let (relations, fragment_mapping) = Self::finish_replace_streaming_job_inner(
            tmp_id,
            merge_updates,
            table_col_index_mapping,
            creating_sink_id,
            dropping_sink_id,
            updated_sink_catalogs,
            &txn,
            streaming_job,
        )
        .await?;

        txn.commit().await?;

        // FIXME: Do not notify frontend currently, because frontend nodes might refer to old table
        // catalog and need to access the old fragment. Let frontend nodes delete the old fragment
        // when they receive table catalog change.
        // self.notify_fragment_mapping(NotificationOperation::Delete, old_fragment_mappings)
        //     .await;
        self.notify_fragment_mapping(NotificationOperation::Add, fragment_mapping)
            .await;
        let version = self
            .notify_frontend(
                NotificationOperation::Update,
                NotificationInfo::RelationGroup(PbRelationGroup { relations }),
            )
            .await;

        Ok(version)
    }

    pub async fn finish_replace_streaming_job_inner(
        tmp_id: ObjectId,
        merge_updates: Vec<PbMergeUpdate>,
        table_col_index_mapping: Option<ColIndexMapping>,
        creating_sink_id: Option<SinkId>,
        dropping_sink_id: Option<SinkId>,
        updated_sink_catalogs: Vec<SinkId>,
        txn: &DatabaseTransaction,
        streaming_job: StreamingJob,
    ) -> MetaResult<(Vec<Relation>, Vec<PbFragmentWorkerSlotMapping>)> {
        // Question: The source catalog should be remain unchanged?
        let StreamingJob::Table(_, table, ..) = streaming_job else {
            unreachable!("unexpected job: {streaming_job:?}")
        };

        let job_id = table.id as ObjectId;

        let original_table_catalogs = Table::find_by_id(job_id)
            .select_only()
            .columns([table::Column::Columns])
            .into_tuple::<ColumnCatalogArray>()
            .one(txn)
            .await?
            .ok_or_else(|| MetaError::catalog_id_not_found("table", job_id))?;

        // For sinks created in earlier versions, we need to set the original_target_columns.
        for sink_id in updated_sink_catalogs {
            sink::ActiveModel {
                sink_id: Set(sink_id as _),
                original_target_columns: Set(Some(original_table_catalogs.clone())),
                ..Default::default()
            }
            .update(txn)
            .await?;
        }

        let mut table = table::ActiveModel::from(table);
        let mut incoming_sinks = table.incoming_sinks.as_ref().inner_ref().clone();
        if let Some(sink_id) = creating_sink_id {
            debug_assert!(!incoming_sinks.contains(&{ sink_id }));
            incoming_sinks.push(sink_id as _);
        }

        if let Some(sink_id) = dropping_sink_id {
            let drained = incoming_sinks.extract_if(|id| *id == sink_id).collect_vec();
            debug_assert_eq!(drained, vec![sink_id]);
        }

        table.incoming_sinks = Set(incoming_sinks.into());
        let table = table.update(txn).await?;

        // Fields including `fragment_id` were placeholder values before.
        // After table fragments are created, update them for all internal tables.
        let fragment_info: Vec<(FragmentId, I32Array)> = Fragment::find()
            .select_only()
            .columns([
                fragment::Column::FragmentId,
                fragment::Column::StateTableIds,
            ])
            .filter(fragment::Column::JobId.eq(tmp_id))
            .into_tuple()
            .all(txn)
            .await?;
        for (fragment_id, state_table_ids) in fragment_info {
            for state_table_id in state_table_ids.into_inner() {
                table::ActiveModel {
                    table_id: Set(state_table_id as _),
                    fragment_id: Set(Some(fragment_id)),
                    // No need to update `vnode_count` because it must remain the same.
                    ..Default::default()
                }
                .update(txn)
                .await?;
            }
        }

        // let old_fragment_mappings = get_fragment_mappings(&txn, job_id).await?;
        // 1. replace old fragments/actors with new ones.
        Fragment::delete_many()
            .filter(fragment::Column::JobId.eq(job_id))
            .exec(txn)
            .await?;
        Fragment::update_many()
            .col_expr(fragment::Column::JobId, SimpleExpr::from(job_id))
            .filter(fragment::Column::JobId.eq(tmp_id))
            .exec(txn)
            .await?;

        // 2. update merges.
        let fragment_replace_map: HashMap<_, _> = merge_updates
            .iter()
            .map(|update| {
                (
                    update.upstream_fragment_id,
                    (
                        update.new_upstream_fragment_id.unwrap(),
                        update.added_upstream_actor_id.clone(),
                    ),
                )
            })
            .collect();

        // TODO: remove cache upstream fragment/actor ids and derive them from `actor_dispatcher` table.
        let mut to_update_fragment_ids = HashSet::new();
        for merge_update in merge_updates {
            assert!(merge_update.removed_upstream_actor_id.is_empty());
            assert!(merge_update.new_upstream_fragment_id.is_some());
            let (actor_id, fragment_id, mut upstream_actors) =
                Actor::find_by_id(merge_update.actor_id as ActorId)
                    .select_only()
                    .columns([
                        actor::Column::ActorId,
                        actor::Column::FragmentId,
                        actor::Column::UpstreamActorIds,
                    ])
                    .into_tuple::<(ActorId, FragmentId, ActorUpstreamActors)>()
                    .one(txn)
                    .await?
                    .ok_or_else(|| {
                        MetaError::catalog_id_not_found("actor", merge_update.actor_id)
                    })?;

            assert!(upstream_actors
                .0
                .remove(&(merge_update.upstream_fragment_id as FragmentId))
                .is_some());
            upstream_actors.0.insert(
                merge_update.new_upstream_fragment_id.unwrap() as _,
                merge_update
                    .added_upstream_actor_id
                    .iter()
                    .map(|id| *id as _)
                    .collect(),
            );
            actor::ActiveModel {
                actor_id: Set(actor_id),
                upstream_actor_ids: Set(upstream_actors),
                ..Default::default()
            }
            .update(txn)
            .await?;

            to_update_fragment_ids.insert(fragment_id);
        }
        for fragment_id in to_update_fragment_ids {
            let (fragment_id, mut stream_node, mut upstream_fragment_id) =
                Fragment::find_by_id(fragment_id)
                    .select_only()
                    .columns([
                        fragment::Column::FragmentId,
                        fragment::Column::StreamNode,
                        fragment::Column::UpstreamFragmentId,
                    ])
                    .into_tuple::<(FragmentId, StreamNode, I32Array)>()
                    .one(txn)
                    .await?
                    .map(|(id, node, upstream)| (id, node.to_protobuf(), upstream))
                    .ok_or_else(|| MetaError::catalog_id_not_found("fragment", fragment_id))?;
            visit_stream_node(&mut stream_node, |body| {
                if let PbNodeBody::Merge(m) = body
                    && let Some((new_fragment_id, new_actor_ids)) =
                        fragment_replace_map.get(&m.upstream_fragment_id)
                {
                    m.upstream_fragment_id = *new_fragment_id;
                    m.upstream_actor_id.clone_from(new_actor_ids);
                }
            });
            for fragment_id in &mut upstream_fragment_id.0 {
                if let Some((new_fragment_id, _)) = fragment_replace_map.get(&(*fragment_id as _)) {
                    *fragment_id = *new_fragment_id as _;
                }
            }
            fragment::ActiveModel {
                fragment_id: Set(fragment_id),
                stream_node: Set(StreamNode::from(&stream_node)),
                upstream_fragment_id: Set(upstream_fragment_id),
                ..Default::default()
            }
            .update(txn)
            .await?;
        }

        // 3. remove dummy object.
        Object::delete_by_id(tmp_id).exec(txn).await?;

        // 4. update catalogs and notify.
        let mut relations = vec![];
        let table_obj = table
            .find_related(Object)
            .one(txn)
            .await?
            .ok_or_else(|| MetaError::catalog_id_not_found("object", table.table_id))?;
        relations.push(PbRelation {
            relation_info: Some(PbRelationInfo::Table(ObjectModel(table, table_obj).into())),
        });
        if let Some(table_col_index_mapping) = table_col_index_mapping {
            let expr_rewriter = ReplaceTableExprRewriter {
                table_col_index_mapping,
            };

            let index_items: Vec<(IndexId, ExprNodeArray)> = Index::find()
                .select_only()
                .columns([index::Column::IndexId, index::Column::IndexItems])
                .filter(index::Column::PrimaryTableId.eq(job_id))
                .into_tuple()
                .all(txn)
                .await?;
            for (index_id, nodes) in index_items {
                let mut pb_nodes = nodes.to_protobuf();
                pb_nodes
                    .iter_mut()
                    .for_each(|x| expr_rewriter.rewrite_expr(x));
                let index = index::ActiveModel {
                    index_id: Set(index_id),
                    index_items: Set(pb_nodes.into()),
                    ..Default::default()
                }
                .update(txn)
                .await?;
                let index_obj = index
                    .find_related(Object)
                    .one(txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found("object", index.index_id))?;
                relations.push(PbRelation {
                    relation_info: Some(PbRelationInfo::Index(
                        ObjectModel(index, index_obj).into(),
                    )),
                });
            }
        }

        let fragment_mapping: Vec<_> = get_fragment_mappings(txn, job_id as _).await?;

        Ok((relations, fragment_mapping))
    }

    /// `try_abort_replacing_streaming_job` is used to abort the replacing streaming job, the input `job_id` is the dummy job id.
    pub async fn try_abort_replacing_streaming_job(&self, job_id: ObjectId) -> MetaResult<()> {
        let inner = self.inner.write().await;
        Object::delete_by_id(job_id).exec(&inner.db).await?;
        Ok(())
    }

    // edit the `rate_limit` of the `Source` node in given `source_id`'s fragments
    // return the actor_ids to be applied
    pub async fn update_source_rate_limit_by_source_id(
        &self,
        source_id: SourceId,
        rate_limit: Option<u32>,
    ) -> MetaResult<HashMap<FragmentId, Vec<ActorId>>> {
        let inner = self.inner.read().await;
        let txn = inner.db.begin().await?;

        {
            let active_source = source::ActiveModel {
                source_id: Set(source_id),
                rate_limit: Set(rate_limit.map(|v| v as i32)),
                ..Default::default()
            };
            active_source.update(&txn).await?;
        }

        let (source, obj) = Source::find_by_id(source_id)
            .find_also_related(Object)
            .one(&txn)
            .await?
            .ok_or_else(|| {
                MetaError::catalog_id_not_found(ObjectType::Source.as_str(), source_id)
            })?;

        let is_fs_source = source.with_properties.inner_ref().is_new_fs_connector();
        let streaming_job_ids: Vec<ObjectId> =
            if let Some(table_id) = source.optional_associated_table_id {
                vec![table_id]
            } else if let Some(source_info) = &source.source_info
                && source_info.to_protobuf().is_shared()
            {
                vec![source_id]
            } else {
                ObjectDependency::find()
                    .select_only()
                    .column(object_dependency::Column::UsedBy)
                    .filter(object_dependency::Column::Oid.eq(source_id))
                    .into_tuple()
                    .all(&txn)
                    .await?
            };

        if streaming_job_ids.is_empty() {
            return Err(MetaError::invalid_parameter(format!(
                "source id {source_id} not used by any streaming job"
            )));
        }

        let fragments: Vec<(FragmentId, i32, StreamNode)> = Fragment::find()
            .select_only()
            .columns([
                fragment::Column::FragmentId,
                fragment::Column::FragmentTypeMask,
                fragment::Column::StreamNode,
            ])
            .filter(fragment::Column::JobId.is_in(streaming_job_ids))
            .into_tuple()
            .all(&txn)
            .await?;
        let mut fragments = fragments
            .into_iter()
            .map(|(id, mask, stream_node)| (id, mask, stream_node.to_protobuf()))
            .collect_vec();

        fragments.retain_mut(|(_, fragment_type_mask, stream_node)| {
            let mut found = false;
            if *fragment_type_mask & PbFragmentTypeFlag::Source as i32 != 0 {
                visit_stream_node(stream_node, |node| {
                    if let PbNodeBody::Source(node) = node {
                        if let Some(node_inner) = &mut node.source_inner
                            && node_inner.source_id == source_id as u32
                        {
                            node_inner.rate_limit = rate_limit;
                            found = true;
                        }
                    }
                });
            }
            if is_fs_source {
                // in older versions, there's no fragment type flag for `FsFetch` node,
                // so we just scan all fragments for StreamFsFetch node if using fs connector
                visit_stream_node(stream_node, |node| {
                    if let PbNodeBody::StreamFsFetch(node) = node {
                        *fragment_type_mask |= PbFragmentTypeFlag::FsFetch as i32;
                        if let Some(node_inner) = &mut node.node_inner
                            && node_inner.source_id == source_id as u32
                        {
                            node_inner.rate_limit = rate_limit;
                            found = true;
                        }
                    }
                });
            }
            found
        });

        assert!(
            !fragments.is_empty(),
            "source id should be used by at least one fragment"
        );
        let fragment_ids = fragments.iter().map(|(id, _, _)| *id).collect_vec();
        for (id, fragment_type_mask, stream_node) in fragments {
            fragment::ActiveModel {
                fragment_id: Set(id),
                fragment_type_mask: Set(fragment_type_mask),
                stream_node: Set(StreamNode::from(&stream_node)),
                ..Default::default()
            }
            .update(&txn)
            .await?;
        }
        let fragment_actors = get_fragment_actor_ids(&txn, fragment_ids).await?;

        txn.commit().await?;

        let relation_info = PbRelationInfo::Source(ObjectModel(source, obj.unwrap()).into());
        let relation = PbRelation {
            relation_info: Some(relation_info),
        };
        let _version = self
            .notify_frontend(
                NotificationOperation::Update,
                NotificationInfo::RelationGroup(PbRelationGroup {
                    relations: vec![relation],
                }),
            )
            .await;

        Ok(fragment_actors)
    }

    // edit the `rate_limit` of the `Chain` node in given `table_id`'s fragments
    // return the actor_ids to be applied
    pub async fn update_backfill_rate_limit_by_job_id(
        &self,
        job_id: ObjectId,
        rate_limit: Option<u32>,
    ) -> MetaResult<HashMap<FragmentId, Vec<ActorId>>> {
        let inner = self.inner.read().await;
        let txn = inner.db.begin().await?;

        let fragments: Vec<(FragmentId, i32, StreamNode)> = Fragment::find()
            .select_only()
            .columns([
                fragment::Column::FragmentId,
                fragment::Column::FragmentTypeMask,
                fragment::Column::StreamNode,
            ])
            .filter(fragment::Column::JobId.eq(job_id))
            .into_tuple()
            .all(&txn)
            .await?;
        let mut fragments = fragments
            .into_iter()
            .map(|(id, mask, stream_node)| (id, mask, stream_node.to_protobuf()))
            .collect_vec();

        fragments.retain_mut(|(_, fragment_type_mask, stream_node)| {
            let mut found = false;
            if *fragment_type_mask & PbFragmentTypeFlag::backfill_rate_limit_fragments() != 0 {
                visit_stream_node(stream_node, |node| match node {
                    PbNodeBody::StreamCdcScan(node) => {
                        node.rate_limit = rate_limit;
                        found = true;
                    }
                    PbNodeBody::StreamScan(node) => {
                        node.rate_limit = rate_limit;
                        found = true;
                    }
                    PbNodeBody::SourceBackfill(node) => {
                        node.rate_limit = rate_limit;
                        found = true;
                    }
                    _ => {}
                });
            }
            found
        });

        if fragments.is_empty() {
            return Err(MetaError::invalid_parameter(format!(
                "stream scan node or source node not found in job id {job_id}"
            )));
        }
        let fragment_ids = fragments.iter().map(|(id, _, _)| *id).collect_vec();
        for (id, _, stream_node) in fragments {
            fragment::ActiveModel {
                fragment_id: Set(id),
                stream_node: Set(StreamNode::from(&stream_node)),
                ..Default::default()
            }
            .update(&txn)
            .await?;
        }
        let fragment_actors = get_fragment_actor_ids(&txn, fragment_ids).await?;

        txn.commit().await?;

        Ok(fragment_actors)
    }

    pub async fn post_apply_reschedules(
        &self,
        reschedules: HashMap<FragmentId, Reschedule>,
        table_parallelism_assignment: HashMap<
            risingwave_common::catalog::TableId,
            TableParallelism,
        >,
    ) -> MetaResult<()> {
        fn update_actors(
            actors: &mut Vec<ActorId>,
            to_remove: &HashSet<ActorId>,
            to_create: &Vec<ActorId>,
        ) {
            let actor_id_set: HashSet<_> = actors.iter().copied().collect();
            for actor_id in to_create {
                debug_assert!(!actor_id_set.contains(actor_id));
            }
            for actor_id in to_remove {
                debug_assert!(actor_id_set.contains(actor_id));
            }

            actors.retain(|actor_id| !to_remove.contains(actor_id));
            actors.extend_from_slice(to_create);
        }

        let new_created_actors: HashSet<_> = reschedules
            .values()
            .flat_map(|reschedule| {
                reschedule
                    .added_actors
                    .values()
                    .flatten()
                    .map(|actor_id| *actor_id as ActorId)
            })
            .collect();

        let inner = self.inner.write().await;

        let txn = inner.db.begin().await?;

        let mut fragment_mapping_to_notify = vec![];

        // for assert only
        let mut assert_dispatcher_update_checker = HashSet::new();

        for (
            fragment_id,
            Reschedule {
                added_actors,
                removed_actors,
                vnode_bitmap_updates,
                actor_splits,
                newly_created_actors,
                upstream_fragment_dispatcher_ids,
                upstream_dispatcher_mapping,
                downstream_fragment_ids,
            },
        ) in reschedules
        {
            // drop removed actors
            Actor::delete_many()
                .filter(
                    actor::Column::ActorId
                        .is_in(removed_actors.iter().map(|id| *id as ActorId).collect_vec()),
                )
                .exec(&txn)
                .await?;

            // add new actors
            for (
                PbStreamActor {
                    actor_id,
                    fragment_id,
                    mut nodes,
                    dispatcher,
                    upstream_actor_id,
                    vnode_bitmap,
                    expr_context,
                    ..
                },
                actor_status,
            ) in newly_created_actors
            {
                let mut actor_upstreams = BTreeMap::<FragmentId, BTreeSet<ActorId>>::new();
                let mut new_actor_dispatchers = vec![];

                if let Some(nodes) = &mut nodes {
                    visit_stream_node(nodes, |node| {
                        if let PbNodeBody::Merge(node) = node {
                            actor_upstreams
                                .entry(node.upstream_fragment_id as FragmentId)
                                .or_default()
                                .extend(node.upstream_actor_id.iter().map(|id| *id as ActorId));
                        }
                    });
                }

                let actor_upstreams: BTreeMap<FragmentId, Vec<ActorId>> = actor_upstreams
                    .into_iter()
                    .map(|(k, v)| (k, v.into_iter().collect()))
                    .collect();

                debug_assert_eq!(
                    actor_upstreams
                        .values()
                        .flatten()
                        .cloned()
                        .sorted()
                        .collect_vec(),
                    upstream_actor_id
                        .iter()
                        .map(|actor_id| *actor_id as i32)
                        .sorted()
                        .collect_vec()
                );

                let actor_upstreams = ActorUpstreamActors(actor_upstreams);

                let splits = actor_splits
                    .get(&actor_id)
                    .map(|splits| splits.iter().map(PbConnectorSplit::from).collect_vec());

                Actor::insert(actor::ActiveModel {
                    actor_id: Set(actor_id as _),
                    fragment_id: Set(fragment_id as _),
                    status: Set(ActorStatus::Running),
                    splits: Set(splits.map(|splits| (&PbConnectorSplits { splits }).into())),
                    worker_id: Set(actor_status.worker_id() as _),
                    upstream_actor_ids: Set(actor_upstreams),
                    vnode_bitmap: Set(vnode_bitmap.as_ref().map(|bitmap| bitmap.into())),
                    expr_context: Set(expr_context.as_ref().unwrap().into()),
                })
                .exec(&txn)
                .await?;

                for PbDispatcher {
                    r#type: dispatcher_type,
                    dist_key_indices,
                    output_indices,
                    hash_mapping,
                    dispatcher_id,
                    downstream_actor_id,
                } in dispatcher
                {
                    new_actor_dispatchers.push(actor_dispatcher::ActiveModel {
                        id: Default::default(),
                        actor_id: Set(actor_id as _),
                        dispatcher_type: Set(PbDispatcherType::try_from(dispatcher_type)
                            .unwrap()
                            .into()),
                        dist_key_indices: Set(dist_key_indices.into()),
                        output_indices: Set(output_indices.into()),
                        hash_mapping: Set(hash_mapping.as_ref().map(|mapping| mapping.into())),
                        dispatcher_id: Set(dispatcher_id as _),
                        downstream_actor_ids: Set(downstream_actor_id.into()),
                    })
                }
                if !new_actor_dispatchers.is_empty() {
                    ActorDispatcher::insert_many(new_actor_dispatchers)
                        .exec(&txn)
                        .await?;
                }
            }

            // actor update
            for (actor_id, bitmap) in vnode_bitmap_updates {
                let actor = Actor::find_by_id(actor_id as ActorId)
                    .one(&txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found("actor", actor_id))?;

                let mut actor = actor.into_active_model();
                actor.vnode_bitmap = Set(Some((&bitmap.to_protobuf()).into()));
                actor.update(&txn).await?;
            }

            // Update actor_splits for existing actors
            for (actor_id, splits) in actor_splits {
                if new_created_actors.contains(&(actor_id as ActorId)) {
                    continue;
                }

                let actor = Actor::find_by_id(actor_id as ActorId)
                    .one(&txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found("actor", actor_id))?;

                let mut actor = actor.into_active_model();
                let splits = splits.iter().map(PbConnectorSplit::from).collect_vec();
                actor.splits = Set(Some((&PbConnectorSplits { splits }).into()));
                actor.update(&txn).await?;
            }

            // fragment update
            let fragment = Fragment::find_by_id(fragment_id)
                .one(&txn)
                .await?
                .ok_or_else(|| MetaError::catalog_id_not_found("fragment", fragment_id))?;

            let job_actors = fragment
                .find_related(Actor)
                .all(&txn)
                .await?
                .into_iter()
                .map(|actor| {
                    (
                        fragment_id,
                        fragment.distribution_type,
                        actor.actor_id,
                        actor.vnode_bitmap,
                        actor.worker_id,
                        actor.status,
                    )
                })
                .collect_vec();

            fragment_mapping_to_notify.extend(rebuild_fragment_mapping_from_actors(job_actors));

            // for downstream and upstream
            let removed_actor_ids: HashSet<_> = removed_actors
                .iter()
                .map(|actor_id| *actor_id as ActorId)
                .collect();

            let added_actor_ids = added_actors
                .values()
                .flatten()
                .map(|actor_id| *actor_id as ActorId)
                .collect_vec();

            // first step, upstream fragment
            for (upstream_fragment_id, dispatcher_id) in upstream_fragment_dispatcher_ids {
                let upstream_fragment = Fragment::find_by_id(upstream_fragment_id as FragmentId)
                    .one(&txn)
                    .await?
                    .ok_or_else(|| MetaError::catalog_id_not_found("fragment", fragment_id))?;

                let all_dispatchers = actor_dispatcher::Entity::find()
                    .join(JoinType::InnerJoin, actor_dispatcher::Relation::Actor.def())
                    .filter(actor::Column::FragmentId.eq(upstream_fragment.fragment_id))
                    .filter(actor_dispatcher::Column::DispatcherId.eq(dispatcher_id as i32))
                    .all(&txn)
                    .await?;

                for dispatcher in all_dispatchers {
                    debug_assert!(assert_dispatcher_update_checker.insert(dispatcher.id));
                    if new_created_actors.contains(&dispatcher.actor_id) {
                        continue;
                    }

                    let mut dispatcher = dispatcher.into_active_model();

                    // Only hash dispatcher needs mapping
                    if dispatcher.dispatcher_type.as_ref() == &DispatcherType::Hash {
                        dispatcher.hash_mapping = Set(upstream_dispatcher_mapping
                            .as_ref()
                            .map(|m| risingwave_meta_model::ActorMapping::from(&m.to_protobuf())));
                    }

                    let mut new_downstream_actor_ids =
                        dispatcher.downstream_actor_ids.as_ref().inner_ref().clone();

                    update_actors(
                        new_downstream_actor_ids.as_mut(),
                        &removed_actor_ids,
                        &added_actor_ids,
                    );

                    dispatcher.downstream_actor_ids = Set(new_downstream_actor_ids.into());
                    dispatcher.update(&txn).await?;
                }
            }

            // second step, downstream fragment
            for downstream_fragment_id in downstream_fragment_ids {
                let actors = Actor::find()
                    .filter(actor::Column::FragmentId.eq(downstream_fragment_id as FragmentId))
                    .all(&txn)
                    .await?;

                for actor in actors {
                    if new_created_actors.contains(&actor.actor_id) {
                        continue;
                    }

                    let mut actor = actor.into_active_model();

                    let mut new_upstream_actor_ids =
                        actor.upstream_actor_ids.as_ref().inner_ref().clone();

                    update_actors(
                        new_upstream_actor_ids.get_mut(&fragment_id).unwrap(),
                        &removed_actor_ids,
                        &added_actor_ids,
                    );

                    actor.upstream_actor_ids = Set(new_upstream_actor_ids.into());

                    actor.update(&txn).await?;
                }
            }
        }

        for (table_id, parallelism) in table_parallelism_assignment {
            let mut streaming_job = StreamingJobModel::find_by_id(table_id.table_id() as ObjectId)
                .one(&txn)
                .await?
                .ok_or_else(|| MetaError::catalog_id_not_found("table", table_id))?
                .into_active_model();

            streaming_job.parallelism = Set(match parallelism {
                TableParallelism::Adaptive => StreamingParallelism::Adaptive,
                TableParallelism::Fixed(n) => StreamingParallelism::Fixed(n as _),
                TableParallelism::Custom => StreamingParallelism::Custom,
            });

            streaming_job.update(&txn).await?;
        }

        txn.commit().await?;
        self.notify_fragment_mapping(Operation::Update, fragment_mapping_to_notify)
            .await;

        Ok(())
    }

    /// Note: `FsFetch` created in old versions are not included.
    /// Since this is only used for debugging, it should be fine.
    pub async fn list_rate_limits(&self) -> MetaResult<Vec<RateLimitInfo>> {
        let inner = self.inner.read().await;
        let txn = inner.db.begin().await?;

        let fragments: Vec<(FragmentId, ObjectId, i32, StreamNode)> = Fragment::find()
            .select_only()
            .columns([
                fragment::Column::FragmentId,
                fragment::Column::JobId,
                fragment::Column::FragmentTypeMask,
                fragment::Column::StreamNode,
            ])
            .filter(fragment_type_mask_intersects(
                PbFragmentTypeFlag::rate_limit_fragments(),
            ))
            .into_tuple()
            .all(&txn)
            .await?;

        let mut rate_limits = Vec::new();
        for (fragment_id, job_id, fragment_type_mask, stream_node) in fragments {
            let mut stream_node = stream_node.to_protobuf();
            let mut rate_limit = None;
            let mut node_name = None;

            visit_stream_node(&mut stream_node, |node| {
                match node {
                    // source rate limit
                    PbNodeBody::Source(node) => {
                        if let Some(node_inner) = &mut node.source_inner {
                            debug_assert!(
                                rate_limit.is_none(),
                                "one fragment should only have 1 rate limit node"
                            );
                            rate_limit = node_inner.rate_limit;
                            node_name = Some("SOURCE");
                        }
                    }
                    PbNodeBody::StreamFsFetch(node) => {
                        if let Some(node_inner) = &mut node.node_inner {
                            debug_assert!(
                                rate_limit.is_none(),
                                "one fragment should only have 1 rate limit node"
                            );
                            rate_limit = node_inner.rate_limit;
                            node_name = Some("FS_FETCH");
                        }
                    }
                    // backfill rate limit
                    PbNodeBody::SourceBackfill(node) => {
                        debug_assert!(
                            rate_limit.is_none(),
                            "one fragment should only have 1 rate limit node"
                        );
                        rate_limit = node.rate_limit;
                        node_name = Some("SOURCE_BACKFILL");
                    }
                    PbNodeBody::StreamScan(node) => {
                        debug_assert!(
                            rate_limit.is_none(),
                            "one fragment should only have 1 rate limit node"
                        );
                        rate_limit = node.rate_limit;
                        node_name = Some("STREAM_SCAN");
                    }
                    PbNodeBody::StreamCdcScan(node) => {
                        debug_assert!(
                            rate_limit.is_none(),
                            "one fragment should only have 1 rate limit node"
                        );
                        rate_limit = node.rate_limit;
                        node_name = Some("STREAM_CDC_SCAN");
                    }
                    _ => {}
                }
            });

            if let Some(rate_limit) = rate_limit {
                rate_limits.push(RateLimitInfo {
                    fragment_id: fragment_id as u32,
                    job_id: job_id as u32,
                    fragment_type_mask: fragment_type_mask as u32,
                    rate_limit,
                    node_name: node_name.unwrap().to_string(),
                });
            }
        }

        Ok(rate_limits)
    }
}

fn bitflag_intersects(column: SimpleExpr, value: i32) -> SimpleExpr {
    column
        .binary(BinOper::Custom("&"), value)
        .binary(BinOper::NotEqual, 0)
}

fn fragment_type_mask_intersects(value: i32) -> SimpleExpr {
    bitflag_intersects(fragment::Column::FragmentTypeMask.into_simple_expr(), value)
}