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
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

//! Persistent metadata storage for the coordinator.

use std::collections::{BTreeMap, HashMap, HashSet, VecDeque};
use std::path::Path;
use std::sync::{Arc, Mutex, MutexGuard};
use std::time::{Duration, Instant};

use anyhow::bail;
use chrono::{DateTime, TimeZone, Utc};
use dataflow_types::{
    sinks::SinkEnvelope, sources::persistence::EnvelopePersistDesc,
    sources::persistence::SourcePersistDesc, sources::ExternalSourceConnector, sources::MzOffset,
};
use expr::PartitionId;
use itertools::Itertools;
use lazy_static::lazy_static;
use ore::collections::CollectionExt;
use ore::metrics::MetricsRegistry;
use ore::now::{to_datetime, EpochMillis, NowFn};
use regex::Regex;
use repr::Timestamp;
use serde::{Deserialize, Serialize};
use tracing::{info, trace};

use build_info::DUMMY_BUILD_INFO;
use dataflow_types::{
    sinks::{SinkConnector, SinkConnectorBuilder},
    sources::{SourceConnector, Timeline},
};
use expr::{ExprHumanizer, GlobalId, MirScalarExpr, OptimizedMirRelationExpr};
use repr::{RelationDesc, ScalarType};
use sql::ast::display::AstDisplay;
use sql::ast::{Expr, Raw};
use sql::catalog::{
    CatalogError as SqlCatalogError, CatalogItem as SqlCatalogItem,
    CatalogItemType as SqlCatalogItemType, SessionCatalog,
};
use sql::names::{DatabaseSpecifier, FullName, PartialName, SchemaName};
use sql::plan::{
    CreateIndexPlan, CreateSinkPlan, CreateSourcePlan, CreateTablePlan, CreateTypePlan,
    CreateViewPlan, Params, Plan, PlanContext, StatementDesc,
};
use transform::Optimizer;
use uuid::Uuid;

use crate::catalog::builtin::{
    Builtin, BUILTINS, BUILTIN_ROLES, INFORMATION_SCHEMA, MZ_CATALOG_SCHEMA, MZ_INTERNAL_SCHEMA,
    MZ_TEMP_SCHEMA, PG_CATALOG_SCHEMA,
};
use crate::persistcfg::PersistConfig;
use crate::session::{PreparedStatement, Session};
use crate::CoordError;

mod builtin_table_updates;
mod config;
mod error;
mod migrate;

pub mod builtin;
pub mod storage;

pub use crate::catalog::builtin_table_updates::BuiltinTableUpdate;
pub use crate::catalog::config::Config;
pub use crate::catalog::error::Error;
pub use crate::catalog::error::ErrorKind;

const SYSTEM_CONN_ID: u32 = 0;
const SYSTEM_USER: &str = "mz_system";

// TODO@jldlaughlin: Better assignment strategy for system type OIDs.
// https://github.com/MaterializeInc/materialize/pull/4316#discussion_r496238962
pub const FIRST_USER_OID: u32 = 20_000;

/// A `Catalog` keeps track of the SQL objects known to the planner.
///
/// For each object, it keeps track of both forward and reverse dependencies:
/// i.e., which objects are depended upon by the object, and which objects
/// depend upon the object. It enforces the SQL rules around dropping: an object
/// cannot be dropped until all of the objects that depend upon it are dropped.
/// It also enforces uniqueness of names.
///
/// SQL mandates a hierarchy of exactly three layers. A catalog contains
/// databases, databases contain schemas, and schemas contain catalog items,
/// like sources, sinks, view, and indexes.
///
/// To the outside world, databases, schemas, and items are all identified by
/// name. Items can be referred to by their [`FullName`], which fully and
/// unambiguously specifies the item, or a [`PartialName`], which can omit the
/// database name and/or the schema name. Partial names can be converted into
/// full names via a complicated resolution process documented by the
/// [`Catalog::resolve`] method.
///
/// The catalog also maintains special "ambient schemas": virtual schemas,
/// implicitly present in all databases, that house various system views.
/// The big examples of ambient schemas are `pg_catalog` and `mz_catalog`.
#[derive(Debug, Clone)]
pub struct Catalog {
    state: CatalogState,
    storage: Arc<Mutex<storage::Connection>>,
    oid_counter: u32,
    transient_revision: u64,
    config: sql::catalog::CatalogConfig,
}

#[derive(Debug, Clone)]
pub struct CatalogState {
    by_name: BTreeMap<String, Database>,
    by_id: BTreeMap<GlobalId, CatalogEntry>,
    by_oid: HashMap<u32, GlobalId>,
    /// Contains only enabled indexes from objects in the catalog; does not
    /// contain indexes disabled by e.g. the disable_user_indexes flag.
    enabled_indexes: HashMap<GlobalId, Vec<(GlobalId, Vec<MirScalarExpr>)>>,
    ambient_schemas: BTreeMap<String, Schema>,
    temporary_schemas: HashMap<u32, Schema>,
    roles: HashMap<String, Role>,
}

impl CatalogState {
    /// Encapsulates the logic for creating a source description for a source or table in the catalog.
    pub fn source_description_for(
        &self,
        id: GlobalId,
    ) -> Option<dataflow_types::sources::SourceDesc> {
        let entry = self.get_by_id(&id);

        match entry.item() {
            CatalogItem::Table(table) => {
                let connector = SourceConnector::Local {
                    timeline: table.timeline(),
                    persisted_name: table.persist_name.clone(),
                };
                Some(dataflow_types::sources::SourceDesc {
                    name: entry.name().to_string(),
                    connector,
                    desc: table.desc.clone(),
                })
            }
            CatalogItem::Source(source) => {
                let connector = source.connector.clone();
                Some(dataflow_types::sources::SourceDesc {
                    name: entry.name().to_string(),
                    connector,
                    desc: source.desc.clone(),
                })
            }
            _ => None,
        }
    }

    pub fn enabled_indexes(&self) -> &HashMap<GlobalId, Vec<(GlobalId, Vec<MirScalarExpr>)>> {
        &self.enabled_indexes
    }

    /// Finds the nearest indexes that can satisfy the views or sources whose
    /// identifiers are listed in `ids`.
    ///
    /// Returns the identifiers of all discovered indexes, and the identifiers of
    /// the discovered unmaterialized sources required to satisfy ids. The returned list
    /// of indexes is incomplete iff `ids` depends on at least one unmaterialized source.
    pub fn nearest_indexes(&self, ids: &[GlobalId]) -> (Vec<GlobalId>, Vec<GlobalId>) {
        fn has_indexes(catalog: &CatalogState, id: GlobalId) -> bool {
            matches!(
                catalog.get_by_id(&id).item(),
                CatalogItem::Table(_) | CatalogItem::Source(_) | CatalogItem::View(_)
            )
        }

        fn inner(
            catalog: &CatalogState,
            id: GlobalId,
            indexes: &mut Vec<GlobalId>,
            unmaterialized: &mut Vec<GlobalId>,
        ) {
            if !has_indexes(catalog, id) {
                return;
            }

            // Include all indexes on an id so the dataflow builder can safely use any
            // of them.
            if !catalog.enabled_indexes[&id].is_empty() {
                indexes.extend(catalog.enabled_indexes[&id].iter().map(|(id, _)| id));
                return;
            }

            match catalog.get_by_id(&id).item() {
                view @ CatalogItem::View(_) => {
                    // Unmaterialized view. Recursively search its dependencies.
                    for id in view.uses() {
                        inner(catalog, *id, indexes, unmaterialized)
                    }
                }
                CatalogItem::Source(_) => {
                    // Unmaterialized source. Record that we are missing at
                    // least one index.
                    unmaterialized.push(id);
                }
                CatalogItem::Table(_) => (),
                _ => unreachable!(),
            }
        }

        let mut indexes = vec![];
        let mut unmaterialized = vec![];
        for id in ids {
            inner(self, *id, &mut indexes, &mut unmaterialized)
        }
        indexes.sort();
        indexes.dedup();

        unmaterialized.sort();
        unmaterialized.dedup();
        (indexes, unmaterialized)
    }

    /// Computes the IDs of any indexes that transitively depend on this catalog
    /// entry.
    pub fn dependent_indexes(&self, id: GlobalId) -> Vec<GlobalId> {
        let mut out = Vec::new();
        self.dependent_indexes_inner(id, &mut out);
        out
    }

    fn dependent_indexes_inner(&self, id: GlobalId, out: &mut Vec<GlobalId>) {
        let entry = self.get_by_id(&id);
        match entry.item() {
            CatalogItem::Index(_) => out.push(id),
            _ => {
                for id in entry.used_by() {
                    self.dependent_indexes_inner(*id, out)
                }
            }
        }
    }

    pub fn uses_tables(&self, id: GlobalId) -> bool {
        match self.get_by_id(&id).item() {
            CatalogItem::Table(_) => true,
            item @ CatalogItem::View(_) => item.uses().iter().any(|id| self.uses_tables(*id)),
            CatalogItem::Source(_)
            | CatalogItem::Func(_)
            | CatalogItem::Index(_)
            | CatalogItem::Sink(_)
            | CatalogItem::Type(_) => false,
        }
    }

    pub fn get_by_id(&self, id: &GlobalId) -> &CatalogEntry {
        &self.by_id[id]
    }

    pub fn insert_item(&mut self, id: GlobalId, oid: u32, name: FullName, item: CatalogItem) {
        if !id.is_system() && !item.is_placeholder() {
            info!("create {} {} ({})", item.typ(), name, id);
        }

        let entry = CatalogEntry {
            item,
            name,
            id,
            oid,
            used_by: Vec::new(),
        };
        for u in entry.uses() {
            match self.by_id.get_mut(&u) {
                Some(metadata) => metadata.used_by.push(entry.id),
                None => panic!(
                    "Catalog: missing dependent catalog item {} while installing {}",
                    &u, entry.name
                ),
            }
        }

        self.populate_enabled_indexes(id, entry.item());

        let conn_id = entry.item().conn_id().unwrap_or(SYSTEM_CONN_ID);
        let schema = self
            .get_schema_mut(&entry.name.database, &entry.name.schema, conn_id)
            .expect("catalog out of sync");
        if let CatalogItem::Func(_) = entry.item() {
            schema.functions.insert(entry.name.item.clone(), entry.id);
        } else {
            schema.items.insert(entry.name.item.clone(), entry.id);
        }

        self.by_oid.insert(oid, entry.id);
        self.by_id.insert(entry.id, entry.clone());
    }

    pub fn populate_enabled_indexes(&mut self, id: GlobalId, item: &CatalogItem) {
        match item {
            CatalogItem::Table(_) | CatalogItem::Source(_) | CatalogItem::View(_) => {
                self.enabled_indexes.entry(id).or_insert_with(Vec::new);
            }
            CatalogItem::Index(index) => {
                if index.enabled {
                    let idxs = self
                        .enabled_indexes
                        .get_mut(&index.on)
                        .expect("object known to exist");

                    // If index not already enabled, add it.
                    if !idxs.iter().any(|(index_id, _)| index_id == &id) {
                        idxs.push((id, index.keys.clone()));
                    }
                }
            }
            CatalogItem::Func(_) | CatalogItem::Sink(_) | CatalogItem::Type(_) => (),
        }
    }

    /// Gets the schema map for the database matching `database_spec`.
    fn get_schema(
        &self,
        database_spec: &DatabaseSpecifier,
        schema_name: &str,
        conn_id: u32,
    ) -> Option<&Schema> {
        // Keep in sync with `get_schemas_mut`.
        match database_spec {
            DatabaseSpecifier::Ambient if schema_name == MZ_TEMP_SCHEMA => {
                self.temporary_schemas.get(&conn_id)
            }
            DatabaseSpecifier::Ambient => self.ambient_schemas.get(schema_name),
            DatabaseSpecifier::Name(name) => self
                .by_name
                .get(name)
                .and_then(|db| db.schemas.get(schema_name)),
        }
    }

    /// Like `get_schemas`, but returns a `mut` reference.
    fn get_schema_mut(
        &mut self,
        database_spec: &DatabaseSpecifier,
        schema_name: &str,
        conn_id: u32,
    ) -> Option<&mut Schema> {
        // Keep in sync with `get_schemas`.
        match database_spec {
            DatabaseSpecifier::Ambient if schema_name == MZ_TEMP_SCHEMA => {
                self.temporary_schemas.get_mut(&conn_id)
            }
            DatabaseSpecifier::Ambient => self.ambient_schemas.get_mut(schema_name),
            DatabaseSpecifier::Name(name) => self
                .by_name
                .get_mut(name)
                .and_then(|db| db.schemas.get_mut(schema_name)),
        }
    }

    /// Reports whether the item identified by `id` is considered volatile.
    ///
    /// `None` indicates that the volatility of `id` is unknown.
    pub fn is_volatile(&self, id: GlobalId) -> Volatility {
        use Volatility::*;

        let item = self.get_by_id(&id).item();
        match item {
            CatalogItem::Source(source) => match &source.connector {
                SourceConnector::External { connector, .. } => match &connector {
                    ExternalSourceConnector::PubNub(_) => Volatile,
                    ExternalSourceConnector::Kinesis(_) => Volatile,
                    _ => Unknown,
                },
                SourceConnector::Local { .. } => Volatile,
            },
            CatalogItem::Index(_) | CatalogItem::View(_) | CatalogItem::Sink(_) => {
                // Volatility follows trinary logic like SQL. If even one
                // volatile dependency exists, then this item is volatile.
                // Otherwise, if a single dependency with unknown volatility
                // exists, then this item is also of unknown volatility. Only if
                // all dependencies are nonvolatile (including the trivial case
                // of no dependencies) is this item nonvolatile.
                item.uses().iter().fold(Nonvolatile, |memo, id| {
                    match (memo, self.is_volatile(*id)) {
                        (Volatile, _) | (_, Volatile) => Volatile,
                        (Unknown, _) | (_, Unknown) => Unknown,
                        (Nonvolatile, Nonvolatile) => Nonvolatile,
                    }
                })
            }
            // TODO: Persisted tables should be Nonvolatile.
            CatalogItem::Table(_) => Volatile,
            CatalogItem::Type(_) => Unknown,
            CatalogItem::Func(_) => Unknown,
        }
    }

    pub fn get_by_oid(&self, oid: &u32) -> &CatalogEntry {
        let id = &self.by_oid[oid];
        &self.by_id[id]
    }
}

#[derive(Debug)]
pub struct ConnCatalog<'a> {
    catalog: &'a Catalog,
    conn_id: u32,
    database: String,
    search_path: &'a [&'a str],
    user: String,
    prepared_statements: Option<&'a HashMap<String, PreparedStatement>>,
}

impl ConnCatalog<'_> {
    pub fn conn_id(&self) -> u32 {
        self.conn_id
    }
}

#[derive(Debug, Deserialize, Serialize, Clone)]
pub struct Database {
    pub name: String,
    pub id: i64,
    #[serde(skip)]
    pub oid: u32,
    pub schemas: BTreeMap<String, Schema>,
}

#[derive(Debug, Deserialize, Serialize, Clone)]
pub struct Schema {
    pub name: SchemaName,
    pub id: i64,
    #[serde(skip)]
    pub oid: u32,
    pub items: BTreeMap<String, GlobalId>,
    pub functions: BTreeMap<String, GlobalId>,
}

#[derive(Debug, Serialize, Clone)]
pub struct Role {
    pub name: String,
    pub id: i64,
    #[serde(skip)]
    pub oid: u32,
}

#[derive(Clone, Debug)]
pub struct CatalogEntry {
    item: CatalogItem,
    used_by: Vec<GlobalId>,
    id: GlobalId,
    oid: u32,
    name: FullName,
}

#[derive(Debug, Clone, Serialize)]
pub enum CatalogItem {
    Table(Table),
    Source(Source),
    View(View),
    Sink(Sink),
    Index(Index),
    Type(Type),
    Func(Func),
}

#[derive(Debug, Clone, Serialize)]
pub struct Table {
    pub create_sql: String,
    pub desc: RelationDesc,
    #[serde(skip)]
    pub defaults: Vec<Expr<Raw>>,
    pub conn_id: Option<u32>,
    pub depends_on: Vec<GlobalId>,
    pub persist_name: Option<String>,
}

impl Table {
    // The Coordinator controls insertions for tables (including system tables),
    // so they are realtime.
    pub fn timeline(&self) -> Timeline {
        Timeline::EpochMilliseconds
    }
}

#[derive(Debug, Clone, Serialize)]
pub struct Source {
    pub create_sql: String,
    pub connector: SourceConnector,
    pub persist_details: Option<SerializedSourcePersistDetails>,
    pub desc: RelationDesc,
}

#[derive(Debug, Clone, Serialize)]
pub struct Sink {
    pub create_sql: String,
    pub from: GlobalId,
    pub connector: SinkConnectorState,
    pub envelope: SinkEnvelope,
    pub with_snapshot: bool,
    pub depends_on: Vec<GlobalId>,
}

#[derive(Debug, Clone, Serialize)]
pub enum SinkConnectorState {
    Pending(SinkConnectorBuilder),
    Ready(SinkConnector),
}

#[derive(Debug, Clone, Serialize)]
pub struct View {
    pub create_sql: String,
    pub optimized_expr: OptimizedMirRelationExpr,
    pub desc: RelationDesc,
    pub conn_id: Option<u32>,
    pub depends_on: Vec<GlobalId>,
}

#[derive(Debug, Clone, Serialize)]
pub struct Index {
    pub create_sql: String,
    pub on: GlobalId,
    pub keys: Vec<MirScalarExpr>,
    pub conn_id: Option<u32>,
    pub depends_on: Vec<GlobalId>,
    pub enabled: bool,
}

#[derive(Debug, Clone, Serialize)]
pub struct Type {
    pub create_sql: String,
    pub inner: TypeInner,
    pub depends_on: Vec<GlobalId>,
}

#[derive(Debug, Clone, Serialize)]
pub enum TypeInner {
    Array {
        element_id: GlobalId,
    },
    Base,
    List {
        element_id: GlobalId,
    },
    Map {
        key_id: GlobalId,
        value_id: GlobalId,
    },
    Pseudo,
}

impl From<sql::plan::TypeInner> for TypeInner {
    fn from(t: sql::plan::TypeInner) -> TypeInner {
        match t {
            sql::plan::TypeInner::List { element_id } => TypeInner::List { element_id },
            sql::plan::TypeInner::Map { key_id, value_id } => TypeInner::Map { key_id, value_id },
        }
    }
}

#[derive(Debug, Clone, Serialize)]
pub struct Func {
    #[serde(skip)]
    pub inner: &'static sql::func::Func,
}

#[derive(Debug, Clone, Serialize)]
pub enum Volatility {
    Volatile,
    Nonvolatile,
    Unknown,
}

impl Volatility {
    fn as_str(&self) -> &'static str {
        match self {
            Volatility::Volatile => "volatile",
            Volatility::Nonvolatile => "nonvolatile",
            Volatility::Unknown => "unknown",
        }
    }
}

impl CatalogItem {
    /// Returns a string indicating the type of this catalog entry.
    fn typ(&self) -> sql::catalog::CatalogItemType {
        match self {
            CatalogItem::Table(_) => sql::catalog::CatalogItemType::Table,
            CatalogItem::Source(_) => sql::catalog::CatalogItemType::Source,
            CatalogItem::Sink(_) => sql::catalog::CatalogItemType::Sink,
            CatalogItem::View(_) => sql::catalog::CatalogItemType::View,
            CatalogItem::Index(_) => sql::catalog::CatalogItemType::Index,
            CatalogItem::Type(_) => sql::catalog::CatalogItemType::Type,
            CatalogItem::Func(_) => sql::catalog::CatalogItemType::Func,
        }
    }

    pub fn desc(&self, name: &FullName) -> Result<&RelationDesc, SqlCatalogError> {
        match &self {
            CatalogItem::Source(src) => Ok(&src.desc),
            CatalogItem::Table(tbl) => Ok(&tbl.desc),
            CatalogItem::View(view) => Ok(&view.desc),
            CatalogItem::Func(_)
            | CatalogItem::Index(_)
            | CatalogItem::Sink(_)
            | CatalogItem::Type(_) => Err(SqlCatalogError::InvalidDependency {
                name: name.to_string(),
                typ: self.typ(),
            }),
        }
    }

    pub fn func(&self, name: &FullName) -> Result<&'static sql::func::Func, SqlCatalogError> {
        match &self {
            CatalogItem::Func(func) => Ok(func.inner),
            _ => Err(SqlCatalogError::UnknownFunction(name.to_string())),
        }
    }

    pub fn source_connector(&self, name: &FullName) -> Result<&SourceConnector, SqlCatalogError> {
        match &self {
            CatalogItem::Source(source) => Ok(&source.connector),
            _ => Err(SqlCatalogError::UnknownSource(name.to_string())),
        }
    }

    /// Collects the identifiers of the dataflows that this item depends
    /// upon.
    pub fn uses(&self) -> &[GlobalId] {
        match self {
            CatalogItem::Func(_) => &[],
            CatalogItem::Index(idx) => &idx.depends_on,
            CatalogItem::Sink(sink) => &sink.depends_on,
            CatalogItem::Source(_) => &[],
            CatalogItem::Table(table) => &table.depends_on,
            CatalogItem::Type(typ) => &typ.depends_on,
            CatalogItem::View(view) => &view.depends_on,
        }
    }

    /// Indicates whether this item is a placeholder for a future item
    /// or if it's actually a real item.
    pub fn is_placeholder(&self) -> bool {
        match self {
            CatalogItem::Func(_)
            | CatalogItem::Index(_)
            | CatalogItem::Source(_)
            | CatalogItem::Table(_)
            | CatalogItem::Type(_)
            | CatalogItem::View(_) => false,
            CatalogItem::Sink(s) => match s.connector {
                SinkConnectorState::Pending(_) => true,
                SinkConnectorState::Ready(_) => false,
            },
        }
    }

    /// Returns the connection ID that this item belongs to, if this item is
    /// temporary.
    pub fn conn_id(&self) -> Option<u32> {
        match self {
            CatalogItem::View(view) => view.conn_id,
            CatalogItem::Index(index) => index.conn_id,
            CatalogItem::Table(table) => table.conn_id,
            _ => None,
        }
    }

    /// Indicates whether this item is temporary or not.
    pub fn is_temporary(&self) -> bool {
        self.conn_id().is_some()
    }

    /// Returns a clone of `self` with all instances of `from` renamed to `to`
    /// (with the option of including the item's own name) or errors if request
    /// is ambiguous.
    fn rename_item_refs(
        &self,
        from: FullName,
        to_item_name: String,
        rename_self: bool,
    ) -> Result<CatalogItem, String> {
        let do_rewrite = |create_sql: String| -> Result<String, String> {
            let mut create_stmt = sql::parse::parse(&create_sql).unwrap().into_element();
            if rename_self {
                sql::ast::transform::create_stmt_rename(&mut create_stmt, to_item_name.clone());
            }
            // Determination of what constitutes an ambiguous request is done here.
            sql::ast::transform::create_stmt_rename_refs(&mut create_stmt, from, to_item_name)?;
            Ok(create_stmt.to_ast_string_stable())
        };

        match self {
            CatalogItem::Table(i) => {
                let mut i = i.clone();
                i.create_sql = do_rewrite(i.create_sql)?;
                Ok(CatalogItem::Table(i))
            }
            CatalogItem::Source(i) => {
                let mut i = i.clone();
                i.create_sql = do_rewrite(i.create_sql)?;
                Ok(CatalogItem::Source(i))
            }
            CatalogItem::Sink(i) => {
                let mut i = i.clone();
                i.create_sql = do_rewrite(i.create_sql)?;
                Ok(CatalogItem::Sink(i))
            }
            CatalogItem::View(i) => {
                let mut i = i.clone();
                i.create_sql = do_rewrite(i.create_sql)?;
                Ok(CatalogItem::View(i))
            }
            CatalogItem::Index(i) => {
                let mut i = i.clone();
                i.create_sql = do_rewrite(i.create_sql)?;
                Ok(CatalogItem::Index(i))
            }
            CatalogItem::Func(_) | CatalogItem::Type(_) => {
                unreachable!("{}s cannot be renamed", self.typ())
            }
        }
    }

    pub fn requires_single_materialization(&self) -> bool {
        if let CatalogItem::Source(Source {
            connector: SourceConnector::External { ref connector, .. },
            ..
        }) = self
        {
            connector.requires_single_materialization()
        } else {
            false
        }
    }
}

impl CatalogEntry {
    /// Reports the description of the datums produced by this catalog item.
    pub fn desc(&self) -> Result<&RelationDesc, SqlCatalogError> {
        self.item.desc(&self.name)
    }

    /// Returns the [`sql::func::Func`] associated with this `CatalogEntry`.
    pub fn func(&self) -> Result<&'static sql::func::Func, SqlCatalogError> {
        self.item.func(&self.name)
    }

    /// Returns the inner [`Index`] if this entry is an index, else `None`.
    pub fn index(&self) -> Option<&Index> {
        match self.item() {
            CatalogItem::Index(idx) => Some(idx),
            _ => None,
        }
    }

    /// Returns the inner [`Source`] if this entry is a source, else `None`.
    pub fn source(&self) -> Option<&Source> {
        match self.item() {
            CatalogItem::Source(src) => Some(src),
            _ => None,
        }
    }

    /// Returns the [`dataflow_types::sources::SourceConnector`] associated with
    /// this `CatalogEntry`.
    pub fn source_connector(&self) -> Result<&SourceConnector, SqlCatalogError> {
        self.item.source_connector(&self.name)
    }

    /// Reports whether this catalog entry is a table.
    pub fn is_table(&self) -> bool {
        matches!(self.item(), CatalogItem::Table(_))
    }

    /// Collects the identifiers of the dataflows that this dataflow depends
    /// upon.
    pub fn uses(&self) -> &[GlobalId] {
        self.item.uses()
    }

    /// Returns the `CatalogItem` associated with this catalog entry.
    pub fn item(&self) -> &CatalogItem {
        &self.item
    }

    /// Returns the global ID of this catalog entry.
    pub fn id(&self) -> GlobalId {
        self.id
    }

    /// Returns the OID of this catalog entry.
    pub fn oid(&self) -> u32 {
        self.oid
    }

    /// Returns the name of this catalog entry.
    pub fn name(&self) -> &FullName {
        &self.name
    }

    /// Returns the identifiers of the dataflows that depend upon this dataflow.
    pub fn used_by(&self) -> &[GlobalId] {
        &self.used_by
    }
}

impl Catalog {
    /// Opens or creates a catalog that stores data at `path`.
    ///
    /// Returns the catalog and a list of updates to builtin tables that
    /// describe the initial state of the catalog.
    pub async fn open(config: Config<'_>) -> Result<(Catalog, Vec<BuiltinTableUpdate>), Error> {
        let mut catalog = Catalog {
            state: CatalogState {
                by_name: BTreeMap::new(),
                by_id: BTreeMap::new(),
                by_oid: HashMap::new(),
                enabled_indexes: HashMap::new(),
                ambient_schemas: BTreeMap::new(),
                temporary_schemas: HashMap::new(),
                roles: HashMap::new(),
            },
            oid_counter: FIRST_USER_OID,
            transient_revision: 0,
            config: sql::catalog::CatalogConfig {
                start_time: to_datetime((config.now)()),
                start_instant: Instant::now(),
                nonce: rand::random(),
                experimental_mode: config.storage.experimental_mode(),
                safe_mode: config.safe_mode,
                cluster_id: config.storage.cluster_id(),
                session_id: Uuid::new_v4(),
                build_info: config.build_info,
                aws_external_id: config.aws_external_id.clone(),
                timestamp_frequency: config.timestamp_frequency,
                now: config.now.clone(),
                disable_user_indexes: config.disable_user_indexes,
            },
            storage: Arc::new(Mutex::new(config.storage)),
        };

        catalog.create_temporary_schema(SYSTEM_CONN_ID)?;

        let databases = catalog.storage().load_databases()?;
        for (id, name) in databases {
            let oid = catalog.allocate_oid()?;
            catalog.state.by_name.insert(
                name.clone(),
                Database {
                    name: name.clone(),
                    id,
                    oid,
                    schemas: BTreeMap::new(),
                },
            );
        }

        let schemas = catalog.storage().load_schemas()?;
        for (id, database_name, schema_name) in schemas {
            let oid = catalog.allocate_oid()?;
            let schemas = match &database_name {
                Some(database_name) => catalog
                    .state
                    .by_name
                    .get_mut(database_name)
                    .map(|db| &mut db.schemas)
                    .expect("catalog out of sync"),
                None => &mut catalog.state.ambient_schemas,
            };
            schemas.insert(
                schema_name.clone(),
                Schema {
                    name: SchemaName {
                        database: database_name.into(),
                        schema: schema_name.clone(),
                    },
                    id,
                    oid,
                    items: BTreeMap::new(),
                    functions: BTreeMap::new(),
                },
            );
        }

        let roles = catalog.storage().load_roles()?;
        let builtin_roles = BUILTIN_ROLES.iter().map(|b| (b.id, b.name.to_owned()));
        for (id, name) in roles.into_iter().chain(builtin_roles) {
            let oid = catalog.allocate_oid()?;
            catalog.state.roles.insert(
                name.clone(),
                Role {
                    name: name.clone(),
                    id,
                    oid,
                },
            );
        }

        for builtin in BUILTINS.values() {
            let name = FullName {
                database: DatabaseSpecifier::Ambient,
                schema: builtin.schema().into(),
                item: builtin.name().into(),
            };
            match builtin {
                Builtin::Log(log) if config.enable_logging => {
                    let index_name = format!("{}_primary_idx", log.name);
                    let oid = catalog.allocate_oid()?;
                    catalog.state.insert_item(
                        log.id,
                        oid,
                        name.clone(),
                        CatalogItem::Source(Source {
                            create_sql: "TODO".to_string(),
                            connector: dataflow_types::sources::SourceConnector::Local {
                                timeline: Timeline::EpochMilliseconds,
                                persisted_name: None,
                            },
                            persist_details: None,
                            desc: log.variant.desc(),
                        }),
                    );
                    let oid = catalog.allocate_oid()?;
                    catalog.state.insert_item(
                        log.index_id,
                        oid,
                        FullName {
                            database: DatabaseSpecifier::Ambient,
                            schema: MZ_CATALOG_SCHEMA.into(),
                            item: index_name.clone(),
                        },
                        CatalogItem::Index(Index {
                            on: log.id,
                            keys: log
                                .variant
                                .index_by()
                                .into_iter()
                                .map(MirScalarExpr::Column)
                                .collect(),
                            create_sql: super::coord::index_sql(
                                index_name,
                                name,
                                &log.variant.desc(),
                                &log.variant.index_by(),
                            ),
                            conn_id: None,
                            depends_on: vec![log.id],
                            enabled: catalog.index_enabled_by_default(&log.index_id),
                        }),
                    );
                }

                Builtin::Table(table) => {
                    let index_name = format!("{}_primary_idx", table.name);
                    let index_columns = table.desc.typ().default_key();
                    let index_sql = super::coord::index_sql(
                        index_name.clone(),
                        name.clone(),
                        &table.desc,
                        &index_columns,
                    );
                    let oid = catalog.allocate_oid()?;
                    let persist_name = if table.persistent {
                        config
                            .persister
                            .new_table_persist_name(table.id, &name.to_string())
                    } else {
                        None
                    };
                    catalog.state.insert_item(
                        table.id,
                        oid,
                        name.clone(),
                        CatalogItem::Table(Table {
                            create_sql: "TODO".to_string(),
                            desc: table.desc.clone(),
                            defaults: vec![Expr::null(); table.desc.arity()],
                            conn_id: None,
                            depends_on: vec![],
                            persist_name,
                        }),
                    );
                    let oid = catalog.allocate_oid()?;
                    catalog.state.insert_item(
                        table.index_id,
                        oid,
                        FullName {
                            database: DatabaseSpecifier::Ambient,
                            schema: MZ_CATALOG_SCHEMA.into(),
                            item: index_name,
                        },
                        CatalogItem::Index(Index {
                            on: table.id,
                            keys: index_columns
                                .iter()
                                .map(|i| MirScalarExpr::Column(*i))
                                .collect(),
                            create_sql: index_sql,
                            conn_id: None,
                            depends_on: vec![table.id],
                            enabled: catalog.index_enabled_by_default(&table.index_id),
                        }),
                    );
                }

                Builtin::View(view) if config.enable_logging || !view.needs_logs => {
                    let table_persist_name = None;
                    let source_persist_details = None;
                    let item = catalog
                        .parse_item(
                            view.id,
                            view.sql.into(),
                            None,
                            table_persist_name,
                            source_persist_details,
                        )
                        .unwrap_or_else(|e| {
                            panic!(
                                "internal error: failed to load bootstrap view:\n\
                                    {}\n\
                                    error:\n\
                                    {:?}",
                                view.name, e
                            )
                        });
                    let oid = catalog.allocate_oid()?;
                    catalog.state.insert_item(view.id, oid, name, item);
                }

                Builtin::Type(typ) => {
                    catalog.state.insert_item(
                        typ.id,
                        typ.oid(),
                        FullName {
                            database: DatabaseSpecifier::Ambient,
                            schema: PG_CATALOG_SCHEMA.into(),
                            item: typ.name().to_owned(),
                        },
                        CatalogItem::Type(Type {
                            create_sql: format!("CREATE TYPE {}", typ.name()),
                            inner: match typ.kind() {
                                postgres_types::Kind::Array(element_type) => {
                                    let element_id = catalog.state.ambient_schemas
                                        [PG_CATALOG_SCHEMA]
                                        .items[element_type.name()];
                                    TypeInner::Array { element_id }
                                }
                                postgres_types::Kind::Pseudo => TypeInner::Pseudo,
                                postgres_types::Kind::Simple => TypeInner::Base,
                                _ => unreachable!(),
                            },
                            depends_on: vec![],
                        }),
                    );
                }

                Builtin::Func(func) => {
                    let oid = catalog.allocate_oid()?;
                    catalog.state.insert_item(
                        func.id,
                        oid,
                        name.clone(),
                        CatalogItem::Func(Func { inner: func.inner }),
                    );
                }

                _ => (),
            }
        }
        if !config.skip_migrations {
            let last_seen_version = catalog.storage().get_catalog_content_version()?;
            crate::catalog::migrate::migrate(&mut catalog).map_err(|e| {
                Error::new(ErrorKind::FailedMigration {
                    last_seen_version,
                    this_version: catalog.config.build_info.version,
                    cause: e.to_string(),
                })
            })?;
            catalog
                .storage()
                .set_catalog_content_version(catalog.config.build_info.version)?;
        }

        let mut storage = catalog.storage();
        let mut tx = storage.transaction()?;
        let catalog = Self::load_catalog_items(&mut tx, &catalog)?;
        tx.commit()?;

        let mut builtin_table_updates = vec![];
        for (schema_name, schema) in &catalog.state.ambient_schemas {
            let db_spec = DatabaseSpecifier::Ambient;
            builtin_table_updates.push(catalog.state.pack_schema_update(&db_spec, schema_name, 1));
            for (_item_name, item_id) in &schema.items {
                builtin_table_updates.extend(catalog.state.pack_item_update(*item_id, 1));
            }
            for (_item_name, function_id) in &schema.functions {
                builtin_table_updates.extend(catalog.state.pack_item_update(*function_id, 1));
            }
        }
        for (db_name, db) in &catalog.state.by_name {
            builtin_table_updates.push(catalog.state.pack_database_update(db_name, 1));
            let db_spec = DatabaseSpecifier::Name(db_name.clone());
            for (schema_name, schema) in &db.schemas {
                builtin_table_updates.push(catalog.state.pack_schema_update(
                    &db_spec,
                    schema_name,
                    1,
                ));
                for (_item_name, item_id) in &schema.items {
                    builtin_table_updates.extend(catalog.state.pack_item_update(*item_id, 1));
                }
                for (_item_name, function_id) in &schema.functions {
                    builtin_table_updates.extend(catalog.state.pack_item_update(*function_id, 1));
                }
            }
        }
        for (role_name, _role) in &catalog.state.roles {
            builtin_table_updates.push(catalog.state.pack_role_update(role_name, 1));
        }

        Ok((catalog, builtin_table_updates))
    }

    /// Retuns the catalog's transient revision, which starts at 1 and is
    /// incremented on every change. This is not persisted to disk, and will
    /// restart on every load.
    pub fn transient_revision(&self) -> u64 {
        self.transient_revision
    }

    /// Takes a catalog which only has items in its on-disk storage ("unloaded")
    /// and cannot yet resolve names, and returns a catalog loaded with those
    /// items.
    ///
    /// This function requires transactions to support loading a catalog with
    /// the transaction's currently in-flight updates to existing catalog
    /// objects, which is necessary for at least one catalog migration.
    ///
    /// TODO(justin): it might be nice if these were two different types.
    pub fn load_catalog_items(
        tx: &mut storage::Transaction,
        c: &Catalog,
    ) -> Result<Catalog, Error> {
        let mut c = c.clone();
        let items = tx.load_items()?;
        for (id, name, def) in items {
            // TODO(benesch): a better way of detecting when a view has depended
            // upon a non-existent logging view. This is fine for now because
            // the only goal is to produce a nicer error message; we'll bail out
            // safely even if the error message we're sniffing out changes.
            lazy_static! {
                static ref LOGGING_ERROR: Regex =
                    Regex::new("unknown catalog item 'mz_catalog.[^']*'").unwrap();
            }
            let item = match c.deserialize_item(id, def) {
                Ok(item) => item,
                Err(e) if LOGGING_ERROR.is_match(&e.to_string()) => {
                    return Err(Error::new(ErrorKind::UnsatisfiableLoggingDependency {
                        depender_name: name.to_string(),
                    }));
                }
                Err(e) => {
                    return Err(Error::new(ErrorKind::Corruption {
                        detail: format!("failed to deserialize item {} ({}): {}", id, name, e),
                    }))
                }
            };
            let oid = c.allocate_oid()?;
            c.state.insert_item(id, oid, name, item);
        }
        c.transient_revision = 1;
        Ok(c)
    }

    /// Opens the catalog at `path` with parameters set appropriately for debug
    /// contexts, like in tests.
    ///
    /// WARNING! This function can arbitrarily fail because it does not make any
    /// effort to adjust the catalog's contents' structure or semantics to the
    /// currently running version, i.e. it does not apply any migrations.
    ///
    /// This function should not be called in production contexts. Use
    /// [`Catalog::open`] with appropriately set configuration parameters
    /// instead.
    pub async fn open_debug(path: &Path, now: NowFn) -> Result<Catalog, anyhow::Error> {
        let experimental_mode = None;
        let metrics_registry = &MetricsRegistry::new();
        let storage = storage::Connection::open(path, experimental_mode)?;
        let (catalog, _) = Self::open(Config {
            storage,
            enable_logging: true,
            experimental_mode,
            safe_mode: false,
            build_info: &DUMMY_BUILD_INFO,
            aws_external_id: None,
            timestamp_frequency: Duration::from_secs(1),
            now,
            skip_migrations: true,
            metrics_registry,
            disable_user_indexes: false,
            persister: &PersistConfig::disabled()
                .init(Uuid::new_v4(), DUMMY_BUILD_INFO, metrics_registry)
                .await?,
        })
        .await?;
        Ok(catalog)
    }

    pub fn for_session<'a>(&'a self, session: &'a Session) -> ConnCatalog<'a> {
        ConnCatalog {
            catalog: self,
            conn_id: session.conn_id(),
            database: session.vars().database().into(),
            search_path: session.vars().search_path(),
            user: session.user().into(),
            prepared_statements: Some(session.prepared_statements()),
        }
    }

    pub fn for_sessionless_user(&self, user: String) -> ConnCatalog {
        ConnCatalog {
            catalog: self,
            conn_id: SYSTEM_CONN_ID,
            database: "materialize".into(),
            search_path: &[],
            user,
            prepared_statements: None,
        }
    }

    // Leaving the system's search path empty allows us to catch issues
    // where catalog object names have not been normalized correctly.
    pub fn for_system_session(&self) -> ConnCatalog {
        self.for_sessionless_user(SYSTEM_USER.into())
    }

    fn storage(&self) -> MutexGuard<storage::Connection> {
        self.storage.lock().expect("lock poisoned")
    }

    pub fn allocate_id(&mut self) -> Result<GlobalId, Error> {
        self.storage().allocate_id()
    }

    pub fn allocate_oid(&mut self) -> Result<u32, Error> {
        let oid = self.oid_counter;
        if oid == u32::max_value() {
            return Err(Error::new(ErrorKind::OidExhaustion));
        }
        self.oid_counter += 1;
        Ok(oid)
    }

    pub fn resolve_schema(
        &self,
        current_database: &str,
        database: Option<String>,
        schema_name: &str,
        conn_id: u32,
    ) -> Result<&Schema, SqlCatalogError> {
        let database_spec = match database {
            // If a database is explicitly specified, validate it. Note that we
            // intentionally do not validate `current_database` to permit
            // querying `mz_catalog` with an invalid session database, e.g., so
            // that you can run `SHOW DATABASES` to *find* a valid database.
            Some(database) if !self.state.by_name.contains_key(&database) => {
                return Err(SqlCatalogError::UnknownDatabase(database));
            }
            Some(database) => DatabaseSpecifier::Name(database),
            None => DatabaseSpecifier::Name(current_database.into()),
        };

        // First try to find the schema in the named database.
        if let Some(schema) = self.get_schema(&database_spec, schema_name, conn_id) {
            return Ok(schema);
        }

        // Then fall back to the ambient database.
        if let Some(schema) = self.get_schema(&DatabaseSpecifier::Ambient, schema_name, conn_id) {
            return Ok(schema);
        }

        Err(SqlCatalogError::UnknownSchema(schema_name.into()))
    }

    /// Resolves `name` to a non-function [`CatalogEntry`].
    pub fn resolve_item(
        &self,
        current_database: &str,
        search_path: &[&str],
        name: &PartialName,
        conn_id: u32,
    ) -> Result<&CatalogEntry, SqlCatalogError> {
        self.resolve(
            |schema| &schema.items,
            current_database,
            search_path,
            name,
            conn_id,
        )
    }

    /// Resolves `name` to a function [`CatalogEntry`].
    pub fn resolve_function(
        &self,
        current_database: &str,
        search_path: &[&str],
        name: &PartialName,
        conn_id: u32,
    ) -> Result<&CatalogEntry, SqlCatalogError> {
        self.resolve(
            |schema| &schema.functions,
            current_database,
            search_path,
            name,
            conn_id,
        )
    }

    /// Resolves [`PartialName`] into a [`FullName`].
    ///
    /// If `name` does not specify a database, the `current_database` is used.
    /// If `name` does not specify a schema, then the schemas in `search_path`
    /// are searched in order.
    #[allow(clippy::useless_let_if_seq)]
    pub fn resolve(
        &self,
        get_schema_entries: fn(&Schema) -> &BTreeMap<String, GlobalId>,
        current_database: &str,
        search_path: &[&str],
        name: &PartialName,
        conn_id: u32,
    ) -> Result<&CatalogEntry, SqlCatalogError> {
        // If a schema name was specified, just try to find the item in that
        // schema. If no schema was specified, try to find the item in the connection's
        // temporary schema. If the item is not found, try to find the item in every
        // schema in the search path.
        //
        // This is written strangely to work around limitations in Rust's
        // temporary lifetime inference [0]. Ideally the following would work,
        // but it does not:
        //
        //     let schemas = match name.schema {
        //         Some(name) => &[name],
        //         None => search_path,
        //     }
        //
        // [0]: https://github.com/rust-lang/rust/issues/15023
        let mut schemas = &[name.schema.as_deref().unwrap_or("")][..];
        if name.schema.is_none() {
            let temp_schema = self
                .get_schema(&DatabaseSpecifier::Ambient, MZ_TEMP_SCHEMA, conn_id)
                .expect("missing temporary schema for connection");
            if let Some(id) = temp_schema.items.get(&name.item) {
                return Ok(&self.state.by_id[id]);
            } else {
                schemas = search_path;
            }
        }

        for &schema_name in schemas {
            let database_name = name.database.clone();
            let schema =
                match self.resolve_schema(&current_database, database_name, schema_name, conn_id) {
                    Ok(schema) => schema,
                    Err(SqlCatalogError::UnknownSchema(_)) => continue,
                    Err(e) => return Err(e),
                };

            if let Some(id) = get_schema_entries(schema).get(&name.item) {
                return Ok(&self.state.by_id[id]);
            }
        }
        Err(SqlCatalogError::UnknownItem(name.to_string()))
    }

    pub fn state(&self) -> &CatalogState {
        &self.state
    }

    /// Returns the named catalog item, if it exists.
    pub fn try_get(&self, name: &FullName, conn_id: u32) -> Option<&CatalogEntry> {
        self.get_schema(&name.database, &name.schema, conn_id)
            .and_then(|schema| schema.items.get(&name.item))
            .map(|id| &self.state.by_id[id])
    }

    pub fn try_get_by_id(&self, id: GlobalId) -> Option<&CatalogEntry> {
        self.state.by_id.get(&id)
    }

    pub fn get_by_id(&self, id: &GlobalId) -> &CatalogEntry {
        self.state.get_by_id(id)
    }

    pub fn insert_item(&mut self, id: GlobalId, oid: u32, name: FullName, item: CatalogItem) {
        self.state.insert_item(id, oid, name, item)
    }

    pub fn get_by_oid(&self, oid: &u32) -> &CatalogEntry {
        self.state.get_by_oid(oid)
    }

    /// Creates a new schema in the `Catalog` for temporary items
    /// indicated by the TEMPORARY or TEMP keywords.
    pub fn create_temporary_schema(&mut self, conn_id: u32) -> Result<(), Error> {
        let oid = self.allocate_oid()?;
        self.state.temporary_schemas.insert(
            conn_id,
            Schema {
                name: SchemaName {
                    database: DatabaseSpecifier::Ambient,
                    schema: MZ_TEMP_SCHEMA.into(),
                },
                id: -1,
                oid,
                items: BTreeMap::new(),
                functions: BTreeMap::new(),
            },
        );
        Ok(())
    }

    fn item_exists_in_temp_schemas(&mut self, conn_id: u32, item_name: &str) -> bool {
        self.state.temporary_schemas[&conn_id]
            .items
            .contains_key(item_name)
    }

    pub fn drop_temp_item_ops(&mut self, conn_id: u32) -> Vec<Op> {
        let ids: Vec<GlobalId> = self.state.temporary_schemas[&conn_id]
            .items
            .values()
            .cloned()
            .collect();
        self.drop_items_ops(&ids)
    }

    pub fn drop_temporary_schema(&mut self, conn_id: u32) -> Result<(), Error> {
        if !self.state.temporary_schemas[&conn_id].items.is_empty() {
            return Err(Error::new(ErrorKind::SchemaNotEmpty(MZ_TEMP_SCHEMA.into())));
        }
        self.state.temporary_schemas.remove(&conn_id);
        Ok(())
    }

    fn get_schema(
        &self,
        database_spec: &DatabaseSpecifier,
        schema_name: &str,
        conn_id: u32,
    ) -> Option<&Schema> {
        self.state.get_schema(database_spec, schema_name, conn_id)
    }

    pub fn drop_database_ops(&mut self, name: String) -> Vec<Op> {
        let mut ops = vec![];
        let mut seen = HashSet::new();
        if let Some(database) = self.state.by_name.get(&name) {
            for (schema_name, schema) in &database.schemas {
                Self::drop_schema_items(schema, &self.state.by_id, &mut ops, &mut seen);
                ops.push(Op::DropSchema {
                    database_name: DatabaseSpecifier::Name(name.clone()),
                    schema_name: schema_name.clone(),
                });
            }
            ops.push(Op::DropDatabase { name });
        }
        ops
    }

    pub fn drop_schema_ops(&mut self, name: SchemaName) -> Vec<Op> {
        let mut ops = vec![];
        let mut seen = HashSet::new();
        if let DatabaseSpecifier::Name(database_name) = name.database {
            if let Some(database) = self.state.by_name.get(&database_name) {
                if let Some(schema) = database.schemas.get(&name.schema) {
                    Self::drop_schema_items(schema, &self.state.by_id, &mut ops, &mut seen);
                    ops.push(Op::DropSchema {
                        database_name: DatabaseSpecifier::Name(database_name),
                        schema_name: name.schema,
                    })
                }
            }
        }
        ops
    }

    pub fn drop_items_ops(&mut self, ids: &[GlobalId]) -> Vec<Op> {
        let mut ops = vec![];
        let mut seen = HashSet::new();
        for &id in ids {
            Self::drop_item_cascade(id, &self.state.by_id, &mut ops, &mut seen);
        }
        ops
    }

    fn drop_schema_items(
        schema: &Schema,
        by_id: &BTreeMap<GlobalId, CatalogEntry>,
        ops: &mut Vec<Op>,
        seen: &mut HashSet<GlobalId>,
    ) {
        for &id in schema.items.values() {
            Self::drop_item_cascade(id, by_id, ops, seen)
        }
    }

    fn drop_item_cascade(
        id: GlobalId,
        by_id: &BTreeMap<GlobalId, CatalogEntry>,
        ops: &mut Vec<Op>,
        seen: &mut HashSet<GlobalId>,
    ) {
        if !seen.contains(&id) {
            seen.insert(id);
            for &u in &by_id[&id].used_by {
                Self::drop_item_cascade(u, by_id, ops, seen)
            }
            ops.push(Op::DropItem(id));
        }
    }

    /// Returns the [`Op`]s necessary to enable an index.
    ///
    /// # Panics
    /// Panics if `id` is not the `id` of a [`CatalogItem::Index`].
    pub fn enable_index_ops(&mut self, id: GlobalId) -> Result<Vec<Op>, Error> {
        Ok(match &self.get_by_id(&id).item {
            // no-op
            CatalogItem::Index(index) if index.enabled => vec![],
            CatalogItem::Index(index) => {
                if let CatalogItem::Table(_) = self.get_by_id(&index.on).item() {
                    let default_idx_id = self
                        .default_index_for(index.on)
                        .expect("table must have default index");
                    if id != default_idx_id {
                        self.ensure_default_index_enabled(index.on)?;
                    }
                }

                vec![Op::UpdateItem {
                    id,
                    to_item: CatalogItem::Index(Index {
                        enabled: true,
                        ..index.clone()
                    }),
                }]
            }
            _ => unreachable!("cannot enable non-indexes"),
        })
    }

    /// Gets GlobalIds of temporary items to be created, checks for name collisions
    /// within a connection id.
    fn temporary_ids(
        &mut self,
        ops: &[Op],
        temporary_drops: HashSet<(u32, String)>,
    ) -> Result<Vec<GlobalId>, Error> {
        let mut creating = HashSet::with_capacity(ops.len());
        let mut temporary_ids = Vec::with_capacity(ops.len());
        for op in ops.iter() {
            if let Op::CreateItem {
                id,
                oid: _,
                name,
                item,
            } = op
            {
                if let Some(conn_id) = item.conn_id() {
                    if self.item_exists_in_temp_schemas(conn_id, &name.item)
                        && !temporary_drops.contains(&(conn_id, name.item.clone()))
                        || creating.contains(&(conn_id, &name.item))
                    {
                        return Err(Error::new(ErrorKind::ItemAlreadyExists(name.item.clone())));
                    } else {
                        creating.insert((conn_id, &name.item));
                        temporary_ids.push(id.clone());
                    }
                }
            }
        }
        Ok(temporary_ids)
    }

    /// Insert timestamp bindings into SQLite, and ignores duplicate timestamp bindings.
    ///
    /// Each individual binding is listed as (source_id, partition_id, timestamp, offset)
    /// and it indicates that all data from (source, partition) for offsets < `offset`, can
    /// be assigned `timestamp` iff `offset` is the minimal such offset (this is a way to encode
    /// a [start, end) offset interval without having to duplicate adjacent starts and ends in
    /// storage).
    /// TODO: we intentionally ignore duplicates because BYO sources can send multiple
    /// copies of the same timestamp.
    pub fn insert_timestamp_bindings(
        &mut self,
        timestamps: impl IntoIterator<Item = (GlobalId, String, Timestamp, i64)>,
    ) -> Result<(), Error> {
        let mut storage = self.storage();
        let tx = storage.transaction()?;

        for (sid, pid, ts, offset) in timestamps.into_iter() {
            tx.insert_timestamp_binding(&sid, &pid, ts, offset)?;
        }
        tx.commit()?;

        Ok(())
    }

    /// Read all available timestamp bindings for a source
    ///
    /// Returns its output sorted by (partition, timestamp)
    pub fn load_timestamp_bindings(
        &mut self,
        source_id: GlobalId,
    ) -> Result<Vec<(PartitionId, Timestamp, MzOffset)>, Error> {
        let mut storage = self.storage();
        let tx = storage.transaction()?;

        let ret = tx.load_timestamp_bindings(source_id)?;
        tx.commit()?;

        Ok(ret)
    }

    /// Compact timestamp bindings for a source
    ///
    /// In practice this ends up being "remove all bindings less than a given timestamp"
    /// because all offsets are then assigned to the next available binding.
    pub fn compact_timestamp_bindings(
        &mut self,
        source_id: GlobalId,
        frontier: Timestamp,
    ) -> Result<(), Error> {
        let mut storage = self.storage();
        let tx = storage.transaction()?;

        tx.compact_timestamp_bindings(source_id, frontier)?;
        tx.commit()?;

        Ok(())
    }

    pub fn transact<F, T>(
        &mut self,
        ops: Vec<Op>,
        f: F,
    ) -> Result<(Vec<BuiltinTableUpdate>, T), CoordError>
    where
        F: FnOnce(&CatalogState) -> Result<T, CoordError>,
    {
        trace!("transact: {:?}", ops);

        #[derive(Debug, Clone)]
        enum Action {
            CreateDatabase {
                id: i64,
                oid: u32,
                name: String,
            },
            CreateSchema {
                id: i64,
                oid: u32,
                database_name: String,
                schema_name: String,
            },
            CreateRole {
                id: i64,
                oid: u32,
                name: String,
            },
            CreateItem {
                id: GlobalId,
                oid: u32,
                name: FullName,
                item: CatalogItem,
            },

            DropDatabase {
                name: String,
            },
            DropSchema {
                database_name: String,
                schema_name: String,
            },
            DropRole {
                name: String,
            },
            DropItem(GlobalId),
            UpdateItem {
                id: GlobalId,
                to_name: FullName,
                to_item: CatalogItem,
            },
        }

        let drop_ids: HashSet<_> = ops
            .iter()
            .filter_map(|op| match op {
                Op::DropItem(id) => Some(*id),
                _ => None,
            })
            .collect();
        let temporary_drops = drop_ids
            .iter()
            .filter_map(|id| {
                let entry = self.get_by_id(id);
                match entry.item.conn_id() {
                    Some(conn_id) => Some((conn_id, entry.name().item.clone())),
                    None => None,
                }
            })
            .collect();
        let temporary_ids = self.temporary_ids(&ops, temporary_drops)?;
        let mut builtin_table_updates = vec![];
        let mut actions = Vec::with_capacity(ops.len());
        let mut storage = self.storage();
        let mut tx = storage.transaction()?;
        for op in ops {
            actions.extend(match op {
                Op::CreateDatabase { name, oid } => vec![Action::CreateDatabase {
                    id: tx.insert_database(&name)?,
                    oid,
                    name,
                }],
                Op::CreateSchema {
                    database_name,
                    schema_name,
                    oid,
                } => {
                    if is_reserved_name(&schema_name) {
                        return Err(CoordError::Catalog(Error::new(
                            ErrorKind::ReservedSchemaName(schema_name),
                        )));
                    }
                    let (database_id, database_name) = match database_name {
                        DatabaseSpecifier::Name(name) => (tx.load_database_id(&name)?, name),
                        DatabaseSpecifier::Ambient => {
                            return Err(CoordError::Catalog(Error::new(
                                ErrorKind::ReadOnlySystemSchema(schema_name),
                            )));
                        }
                    };
                    vec![Action::CreateSchema {
                        id: tx.insert_schema(database_id, &schema_name)?,
                        oid,
                        database_name,
                        schema_name,
                    }]
                }
                Op::CreateRole { name, oid } => {
                    if is_reserved_name(&name) {
                        return Err(CoordError::Catalog(Error::new(
                            ErrorKind::ReservedRoleName(name),
                        )));
                    }
                    vec![Action::CreateRole {
                        id: tx.insert_role(&name)?,
                        oid,
                        name,
                    }]
                }
                Op::CreateItem {
                    id,
                    oid,
                    name,
                    item,
                } => {
                    if item.is_temporary() {
                        if name.database != DatabaseSpecifier::Ambient
                            || name.schema != MZ_TEMP_SCHEMA
                        {
                            return Err(CoordError::Catalog(Error::new(
                                ErrorKind::InvalidTemporarySchema,
                            )));
                        }
                    } else {
                        if let Some(temp_id) =
                            item.uses()
                                .iter()
                                .find(|id| match self.try_get_by_id(**id) {
                                    Some(entry) => entry.item().is_temporary(),
                                    None => temporary_ids.contains(&id),
                                })
                        {
                            let temp_item = self.get_by_id(temp_id);
                            return Err(CoordError::Catalog(Error::new(
                                ErrorKind::InvalidTemporaryDependency(
                                    temp_item.name().item.clone(),
                                ),
                            )));
                        }
                        let database_id = match &name.database {
                            DatabaseSpecifier::Name(name) => tx.load_database_id(&name)?,
                            DatabaseSpecifier::Ambient => {
                                return Err(CoordError::Catalog(Error::new(
                                    ErrorKind::ReadOnlySystemSchema(name.to_string()),
                                )));
                            }
                        };
                        if let CatalogItem::Type(Type {
                            inner: TypeInner::Base { .. },
                            ..
                        }) = item
                        {
                            return Err(CoordError::Catalog(Error::new(ErrorKind::ReadOnlyItem(
                                name.item,
                            ))));
                        }

                        let schema_id = tx.load_schema_id(database_id, &name.schema)?;
                        let serialized_item = self.serialize_item(&item);
                        tx.insert_item(id, schema_id, &name.item, &serialized_item)?;
                    }

                    vec![Action::CreateItem {
                        id,
                        oid,
                        name,
                        item,
                    }]
                }
                Op::DropDatabase { name } => {
                    tx.remove_database(&name)?;
                    builtin_table_updates.push(self.state.pack_database_update(&name, -1));
                    vec![Action::DropDatabase { name }]
                }
                Op::DropSchema {
                    database_name,
                    schema_name,
                } => {
                    let (database_id, database_name) = match database_name {
                        DatabaseSpecifier::Name(name) => (tx.load_database_id(&name)?, name),
                        DatabaseSpecifier::Ambient => {
                            return Err(CoordError::Catalog(Error::new(
                                ErrorKind::ReadOnlySystemSchema(schema_name),
                            )));
                        }
                    };
                    tx.remove_schema(database_id, &schema_name)?;
                    builtin_table_updates.push(self.state.pack_schema_update(
                        &DatabaseSpecifier::Name(database_name.clone()),
                        &schema_name,
                        -1,
                    ));
                    vec![Action::DropSchema {
                        database_name,
                        schema_name,
                    }]
                }
                Op::DropRole { name } => {
                    tx.remove_role(&name)?;
                    builtin_table_updates.push(self.state.pack_role_update(&name, -1));
                    vec![Action::DropRole { name }]
                }
                Op::DropItem(id) => {
                    let entry = self.get_by_id(&id);
                    // Prevent dropping a table's default index unless the table
                    // is being dropped too.
                    match entry.item() {
                        CatalogItem::Index(Index { on, .. }) => {
                            if self.get_by_id(on).is_table()
                                && self.default_index_for(*on) == Some(id)
                                && !drop_ids.contains(on)
                            {
                                return Err(CoordError::Catalog(Error::new(
                                    ErrorKind::MandatoryTableIndex(entry.name().to_string()),
                                )));
                            }
                        }
                        CatalogItem::Source(_) => {
                            tx.delete_timestamp_bindings(id)?;
                        }
                        _ => {}
                    }
                    if !entry.item().is_temporary() {
                        tx.remove_item(id)?;
                    }
                    builtin_table_updates.extend(self.state.pack_item_update(id, -1));
                    vec![Action::DropItem(id)]
                }
                Op::RenameItem { id, to_name } => {
                    let mut actions = Vec::new();

                    let entry = self.state.by_id.get(&id).unwrap();
                    if let CatalogItem::Type(_) = entry.item() {
                        return Err(CoordError::Catalog(Error::new(ErrorKind::TypeRename(
                            entry.name().to_string(),
                        ))));
                    }

                    let mut to_full_name = entry.name.clone();
                    to_full_name.item = to_name;

                    // Rename item itself.
                    let item = entry
                        .item
                        .rename_item_refs(entry.name.clone(), to_full_name.item.clone(), true)
                        .map_err(|e| {
                            Error::new(ErrorKind::AmbiguousRename {
                                depender: entry.name.to_string(),
                                dependee: entry.name.to_string(),
                                message: e,
                            })
                        })?;
                    let serialized_item = self.serialize_item(&item);

                    for id in entry.used_by() {
                        let dependent_item = self.state.by_id.get(&id).unwrap();
                        let to_item = dependent_item
                            .item
                            .rename_item_refs(entry.name.clone(), to_full_name.item.clone(), false)
                            .map_err(|e| {
                                Error::new(ErrorKind::AmbiguousRename {
                                    depender: dependent_item.name.to_string(),
                                    dependee: entry.name.to_string(),
                                    message: e,
                                })
                            })?;

                        if !item.is_temporary() {
                            let serialized_item = self.serialize_item(&to_item);
                            tx.update_item(*id, &dependent_item.name.item, &serialized_item)?;
                        }
                        builtin_table_updates.extend(self.state.pack_item_update(*id, -1));

                        actions.push(Action::UpdateItem {
                            id: id.clone(),
                            to_name: dependent_item.name.clone(),
                            to_item,
                        });
                    }
                    if !item.is_temporary() {
                        tx.update_item(id, &to_full_name.item, &serialized_item)?;
                    }
                    builtin_table_updates.extend(self.state.pack_item_update(id, -1));
                    actions.push(Action::UpdateItem {
                        id,
                        to_name: to_full_name,
                        to_item: item,
                    });
                    actions
                }
                Op::UpdateItem { id, to_item } => {
                    let entry = self.get_by_id(&id);

                    if !to_item.is_temporary() {
                        let serialized_item = self.serialize_item(&to_item);
                        tx.update_item(id, &entry.name().item, &serialized_item)?;
                    }

                    builtin_table_updates.extend(self.state.pack_item_update(id, -1));

                    vec![Action::UpdateItem {
                        id,
                        to_name: entry.name().clone(),
                        to_item,
                    }]
                }
            });
        }

        // Prepare a candidate catalog state.
        let mut state = self.state.clone();

        for action in actions {
            match action {
                Action::CreateDatabase { id, oid, name } => {
                    info!("create database {}", name);
                    state.by_name.insert(
                        name.clone(),
                        Database {
                            name: name.clone(),
                            id,
                            oid,
                            schemas: BTreeMap::new(),
                        },
                    );
                    builtin_table_updates.push(state.pack_database_update(&name, 1));
                }

                Action::CreateSchema {
                    id,
                    oid,
                    database_name,
                    schema_name,
                } => {
                    info!("create schema {}.{}", database_name, schema_name);
                    let db = state.by_name.get_mut(&database_name).unwrap();
                    db.schemas.insert(
                        schema_name.clone(),
                        Schema {
                            name: SchemaName {
                                database: DatabaseSpecifier::Name(database_name.clone()),
                                schema: schema_name.clone(),
                            },
                            id,
                            oid,
                            items: BTreeMap::new(),
                            functions: BTreeMap::new(),
                        },
                    );
                    builtin_table_updates.push(state.pack_schema_update(
                        &DatabaseSpecifier::Name(database_name.clone()),
                        &schema_name,
                        1,
                    ));
                }

                Action::CreateRole { id, oid, name } => {
                    info!("create role {}", name);
                    state.roles.insert(
                        name.clone(),
                        Role {
                            name: name.clone(),
                            id,
                            oid,
                        },
                    );
                    builtin_table_updates.push(state.pack_role_update(&name, 1));
                }

                Action::CreateItem {
                    id,
                    oid,
                    name,
                    item,
                } => {
                    state.insert_item(id, oid, name, item);
                    builtin_table_updates.extend(state.pack_item_update(id, 1));
                }

                Action::DropDatabase { name } => {
                    state.by_name.remove(&name);
                }

                Action::DropSchema {
                    database_name,
                    schema_name,
                } => {
                    let db = state.by_name.get_mut(&database_name).unwrap();
                    db.schemas.remove(&schema_name);
                }

                Action::DropRole { name } => {
                    if state.roles.remove(&name).is_some() {
                        info!("drop role {}", name);
                    }
                }

                Action::DropItem(id) => {
                    let metadata = state.by_id.remove(&id).unwrap();
                    if !metadata.item.is_placeholder() {
                        info!("drop {} {} ({})", metadata.item_type(), metadata.name, id);
                    }
                    for u in metadata.uses() {
                        if let Some(dep_metadata) = state.by_id.get_mut(&u) {
                            dep_metadata.used_by.retain(|u| *u != metadata.id)
                        }
                    }

                    let conn_id = metadata.item.conn_id().unwrap_or(SYSTEM_CONN_ID);
                    let schema = state
                        .get_schema_mut(&metadata.name.database, &metadata.name.schema, conn_id)
                        .expect("catalog out of sync");
                    schema
                        .items
                        .remove(&metadata.name.item)
                        .expect("catalog out of sync");
                    if let CatalogItem::Index(index) = &metadata.item {
                        let indexes = state
                            .enabled_indexes
                            .get_mut(&index.on)
                            .expect("catalog out of sync");
                        let i = indexes.iter().position(|(idx_id, _keys)| *idx_id == id);
                        match i {
                            Some(i) => {
                                indexes.remove(i);
                            }
                            None if !index.enabled => {}
                            None => panic!("catalog out of sync"),
                        };
                    }
                    state.enabled_indexes.remove(&id);
                }

                Action::UpdateItem {
                    id,
                    to_name,
                    to_item,
                } => {
                    let old_entry = state.by_id.remove(&id).unwrap();
                    info!(
                        "update {} {} ({})",
                        old_entry.item_type(),
                        old_entry.name,
                        id
                    );
                    assert_eq!(old_entry.uses(), to_item.uses());

                    // Handle updating any indexes. n.b. only supports enabling
                    // indexes; does not support disabling indexes.
                    state.populate_enabled_indexes(id, &to_item);

                    let conn_id = old_entry.item().conn_id().unwrap_or(SYSTEM_CONN_ID);
                    let schema = &mut state
                        .get_schema_mut(&old_entry.name.database, &old_entry.name.schema, conn_id)
                        .expect("catalog out of sync");
                    schema.items.remove(&old_entry.name.item);
                    let mut new_entry = old_entry.clone();
                    new_entry.name = to_name;
                    new_entry.item = to_item;
                    schema.items.insert(new_entry.name.item.clone(), id);
                    state.by_id.insert(id, new_entry.clone());
                    builtin_table_updates.extend(state.pack_item_update(id, 1));
                }
            }
        }

        let result = f(&state)?;

        // The user closure was successful, apply the updates.
        tx.commit().map_err(|err| CoordError::Catalog(err.into()))?;
        // Dropping here keeps the mutable borrow on self, preventing us accidentally
        // mutating anything until after f is executed.
        drop(storage);
        self.state = state;
        self.transient_revision += 1;

        Ok((builtin_table_updates, result))
    }

    fn serialize_item(&self, item: &CatalogItem) -> Vec<u8> {
        let item = match item {
            CatalogItem::Table(table) => SerializedCatalogItem::V1 {
                create_sql: table.create_sql.clone(),
                eval_env: None,
                table_persist_name: table.persist_name.clone(),
                source_persist_details: None,
            },
            CatalogItem::Source(source) => SerializedCatalogItem::V1 {
                create_sql: source.create_sql.clone(),
                eval_env: None,
                table_persist_name: None,
                source_persist_details: source.persist_details.clone(),
            },
            CatalogItem::View(view) => SerializedCatalogItem::V1 {
                create_sql: view.create_sql.clone(),
                eval_env: None,
                table_persist_name: None,
                source_persist_details: None,
            },
            CatalogItem::Index(index) => SerializedCatalogItem::V1 {
                create_sql: index.create_sql.clone(),
                eval_env: None,
                table_persist_name: None,
                source_persist_details: None,
            },
            CatalogItem::Sink(sink) => SerializedCatalogItem::V1 {
                create_sql: sink.create_sql.clone(),
                eval_env: None,
                table_persist_name: None,
                source_persist_details: None,
            },
            CatalogItem::Type(typ) => SerializedCatalogItem::V1 {
                create_sql: typ.create_sql.clone(),
                eval_env: None,
                table_persist_name: None,
                source_persist_details: None,
            },
            CatalogItem::Func(_) => unreachable!("cannot serialize functions yet"),
        };
        serde_json::to_vec(&item).expect("catalog serialization cannot fail")
    }

    fn deserialize_item(&self, id: GlobalId, bytes: Vec<u8>) -> Result<CatalogItem, anyhow::Error> {
        let SerializedCatalogItem::V1 {
            create_sql,
            eval_env: _,
            table_persist_name,
            source_persist_details,
        } = serde_json::from_slice(&bytes)?;
        self.parse_item(
            id,
            create_sql,
            Some(&PlanContext::zero()),
            table_persist_name,
            source_persist_details,
        )
    }

    // Parses the given SQL string into a `CatalogItem`.
    //
    // The given `persist_details` are an optional description of the persisted streams that this
    // source uses, if it is a persisted source.
    fn parse_item(
        &self,
        id: GlobalId,
        create_sql: String,
        pcx: Option<&PlanContext>,
        table_persist_name: Option<String>,
        source_persist_details: Option<SerializedSourcePersistDetails>,
    ) -> Result<CatalogItem, anyhow::Error> {
        let stmt = sql::parse::parse(&create_sql)?.into_element();
        let plan = sql::plan::plan(pcx, &self.for_system_session(), stmt, &Params::empty())?;
        Ok(match plan {
            Plan::CreateTable(CreateTablePlan { table, .. }) => {
                assert!(
                    source_persist_details.is_none(),
                    "got some source_persist_details while we didn't expect them for a table"
                );
                CatalogItem::Table(Table {
                    create_sql: table.create_sql,
                    desc: table.desc,
                    defaults: table.defaults,
                    conn_id: None,
                    depends_on: table.depends_on,
                    persist_name: table_persist_name,
                })
            }
            Plan::CreateSource(CreateSourcePlan { source, .. }) => {
                assert!(
                    table_persist_name.is_none(),
                    "got some table_persist_name while we didn't expect them for a source"
                );
                CatalogItem::Source(Source {
                    create_sql: source.create_sql,
                    connector: source.connector,
                    persist_details: source_persist_details,
                    desc: source.desc,
                })
            }
            Plan::CreateView(CreateViewPlan { view, .. }) => {
                let mut optimizer = Optimizer::logical_optimizer();
                let optimized_expr = optimizer.optimize(view.expr)?;
                let desc = RelationDesc::new(optimized_expr.typ(), view.column_names);
                CatalogItem::View(View {
                    create_sql: view.create_sql,
                    optimized_expr,
                    desc,
                    conn_id: None,
                    depends_on: view.depends_on,
                })
            }
            Plan::CreateIndex(CreateIndexPlan { index, .. }) => CatalogItem::Index(Index {
                create_sql: index.create_sql,
                on: index.on,
                keys: index.keys,
                conn_id: None,
                depends_on: index.depends_on,
                enabled: self.index_enabled_by_default(&id),
            }),
            Plan::CreateSink(CreateSinkPlan {
                sink,
                with_snapshot,
                ..
            }) => CatalogItem::Sink(Sink {
                create_sql: sink.create_sql,
                from: sink.from,
                connector: SinkConnectorState::Pending(sink.connector_builder),
                envelope: sink.envelope,
                with_snapshot,
                depends_on: sink.depends_on,
            }),
            Plan::CreateType(CreateTypePlan { typ, .. }) => CatalogItem::Type(Type {
                create_sql: typ.create_sql,
                inner: typ.inner.into(),
                depends_on: typ.depends_on,
            }),
            _ => bail!("catalog entry generated inappropriate plan"),
        })
    }

    /// Returns the default value for an [`Index`]'s `enabled` field.
    ///
    /// Note that it is the caller's responsibility to ensure that the `id` is
    /// used for an `Index`.
    pub fn index_enabled_by_default(&self, id: &GlobalId) -> bool {
        !self.config.disable_user_indexes || !id.is_user()
    }

    /// Returns a mapping that indicates all indices that are available for each
    /// item in the catalog.
    ///
    /// Note that when `self.config.disable_user_indexes` is `true`, this does
    /// not include any user indexes.
    pub fn enabled_indexes(&self) -> &HashMap<GlobalId, Vec<(GlobalId, Vec<MirScalarExpr>)>> {
        &self.state.enabled_indexes
    }

    /// Returns whether or not an index is enabled.
    ///
    /// # Panics
    /// Panics if `id` does not belong to a [`CatalogItem::Index`].
    pub fn is_index_enabled(&self, id: &GlobalId) -> bool {
        let index_entry = self.get_by_id(&id);
        match index_entry.item() {
            CatalogItem::Index(index) => index.enabled,
            _ => unreachable!("cannot call is_index_enabled on non-idex"),
        }
    }

    /// Returns all indexes on this object known in the catalog.
    pub fn get_indexes_on(&self, id: GlobalId) -> Vec<GlobalId> {
        self.get_by_id(&id)
            .used_by()
            .iter()
            .filter(|uses_id| match self.get_by_id(uses_id).item() {
                CatalogItem::Index(index) => index.on == id,
                _ => false,
            })
            .cloned()
            .collect()
    }

    /// Returns the default index for the specified `id`.
    ///
    /// Panics if `id` does not exist, or if `id` is not an object on which
    /// indexes can be built.
    pub fn default_index_for(&self, id: GlobalId) -> Option<GlobalId> {
        // The default index is the index with the smallest ID, i.e. the one
        // created in closest temporal proximity to the object itself.
        self.get_indexes_on(id).iter().min().cloned()
    }

    /// Returns an error if the object's default index is disabled.
    ///
    /// Note that this function is really only meant to be used with tables.
    ///
    /// # Panics
    /// Panics if the object identified with `id` does not have a default index.
    pub fn ensure_default_index_enabled(&self, id: GlobalId) -> Result<(), Error> {
        let default_idx_id = self
            .default_index_for(id)
            .expect("object must have default index");
        if !self.is_index_enabled(&default_idx_id) {
            return Err(Error::new(ErrorKind::DefaultIndexDisabled {
                idx_on: self.get_by_id(&id).name().to_string(),
                default_idx: self.get_by_id(&default_idx_id).name().to_string(),
            }));
        }
        Ok(())
    }

    pub fn nearest_indexes(&self, ids: &[GlobalId]) -> (Vec<GlobalId>, Vec<GlobalId>) {
        self.state.nearest_indexes(ids)
    }

    pub fn uses_tables(&self, id: GlobalId) -> bool {
        self.state.uses_tables(id)
    }

    /// Serializes the catalog's in-memory state.
    ///
    /// There are no guarantees about the format of the serialized state, except
    /// that the serialized state for two identical catalogs will compare
    /// identically.
    pub fn dump(&self) -> String {
        serde_json::to_string(&self.state.by_name).expect("serialization cannot fail")
    }

    pub fn config(&self) -> &sql::catalog::CatalogConfig {
        &self.config
    }

    pub fn entries(&self) -> impl Iterator<Item = &CatalogEntry> {
        self.state.by_id.values()
    }

    /// Returns all tables, views, and sources in the same schemas as a set of
    /// input ids. The indexes of all relations are included.
    pub fn schema_adjacent_indexed_relations(
        &self,
        ids: &[GlobalId],
        conn_id: u32,
    ) -> Vec<GlobalId> {
        // Find all relations referenced by the expression. Find their parent schemas
        // and add all tables, views, and sources in those schemas to a set.
        let mut relations: HashSet<GlobalId> = HashSet::new();
        let mut schemas = HashSet::new();
        let mut schema_ids = VecDeque::new();
        for id in ids {
            // Always add in the user-specified ids.
            relations.insert(*id);
            let entry = self.get_by_id(&id);
            let name = entry.name();
            schemas.insert((&name.database, &*name.schema));
        }

        // If any of the system schemas is specified, add the rest of the
        // system schemas.
        let system_schemas = &[
            (&DatabaseSpecifier::Ambient, MZ_CATALOG_SCHEMA),
            (&DatabaseSpecifier::Ambient, PG_CATALOG_SCHEMA),
            (&DatabaseSpecifier::Ambient, INFORMATION_SCHEMA),
        ];
        if system_schemas.iter().any(|s| schemas.contains(s)) {
            schemas.extend(system_schemas);
        }

        for (db, schema) in schemas {
            if let Some(schema) = self.get_schema(db, schema, conn_id) {
                schema_ids.extend(schema.items.values());
                while let Some(id) = schema_ids.pop_front() {
                    let entry = self.get_by_id(id);
                    let ty = entry.item_type();
                    match ty {
                        SqlCatalogItemType::Table => {
                            relations.insert(*id);
                        }
                        SqlCatalogItemType::View | SqlCatalogItemType::Source => {
                            let (indexes, unmaterialized) = self.nearest_indexes(&[*id]);
                            relations.extend(indexes);
                            // Add in the view/source if fully materialized.
                            if unmaterialized.is_empty() {
                                relations.insert(*id);
                                if let SqlCatalogItemType::View = ty {
                                    // Add transitive items from views.
                                    if let CatalogItem::View(view) = entry.item() {
                                        schema_ids.extend(&view.depends_on);
                                    }
                                }
                            }
                        }
                        _ => {}
                    }
                }
            }
        }
        relations.into_iter().collect()
    }
}

fn is_reserved_name(name: &str) -> bool {
    name.starts_with("mz_") || name.starts_with("pg_")
}

#[derive(Debug, Clone)]
pub enum Op {
    CreateDatabase {
        name: String,
        oid: u32,
    },
    CreateSchema {
        database_name: DatabaseSpecifier,
        schema_name: String,
        oid: u32,
    },
    CreateRole {
        name: String,
        oid: u32,
    },
    CreateItem {
        id: GlobalId,
        oid: u32,
        name: FullName,
        item: CatalogItem,
    },
    DropDatabase {
        name: String,
    },
    DropSchema {
        database_name: DatabaseSpecifier,
        schema_name: String,
    },
    DropRole {
        name: String,
    },
    /// Unconditionally removes the identified items. It is required that the
    /// IDs come from the output of `plan_remove`; otherwise consistency rules
    /// may be violated.
    DropItem(GlobalId),
    RenameItem {
        id: GlobalId,
        to_name: String,
    },
    UpdateItem {
        id: GlobalId,
        to_item: CatalogItem,
    },
}

#[derive(Debug, Clone, Serialize, Deserialize)]
enum SerializedCatalogItem {
    V1 {
        create_sql: String,
        // The name "eval_env" is historical.
        eval_env: Option<SerializedPlanContext>,
        // Previous versions used "persist_name" as the field name here.
        #[serde(alias = "persist_name")]
        table_persist_name: Option<String>,
        source_persist_details: Option<SerializedSourcePersistDetails>,
    },
}

/// Serialized source persistence details. See `SourcePersistDesc` for an explanation of the
/// fields.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct SerializedSourcePersistDetails {
    /// Name of the primary persisted stream of this source. This is what a consumer of the
    /// persisted data would be interested in while the secondary stream(s) of the source are an
    /// internal implementation detail.
    pub primary_stream: String,

    /// Persisted stream of timestamp bindings.
    pub timestamp_bindings_stream: String,

    /// Any additional details that we need to make the envelope logic stateful.
    pub envelope_details: SerializedEnvelopePersistDetails,
}

/// See `EnvelopePersistDesc` for an explanation of the fields.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub enum SerializedEnvelopePersistDetails {
    Upsert,
    None,
}

impl From<SourcePersistDesc> for SerializedSourcePersistDetails {
    fn from(source_persist_desc: SourcePersistDesc) -> Self {
        SerializedSourcePersistDetails {
            primary_stream: source_persist_desc.primary_stream,
            timestamp_bindings_stream: source_persist_desc.timestamp_bindings_stream,
            envelope_details: source_persist_desc.envelope_desc.into(),
        }
    }
}

impl From<EnvelopePersistDesc> for SerializedEnvelopePersistDetails {
    fn from(persist_desc: EnvelopePersistDesc) -> Self {
        match persist_desc {
            EnvelopePersistDesc::Upsert => SerializedEnvelopePersistDetails::Upsert,
            EnvelopePersistDesc::None => SerializedEnvelopePersistDetails::None,
        }
    }
}

#[derive(Debug, Clone, Serialize, Deserialize)]
struct SerializedPlanContext {
    pub logical_time: Option<u64>,
    pub wall_time: Option<DateTime<Utc>>,
}

impl From<SerializedPlanContext> for PlanContext {
    fn from(cx: SerializedPlanContext) -> PlanContext {
        PlanContext {
            wall_time: cx.wall_time.unwrap_or_else(|| Utc.timestamp(0, 0)),
            qgm_optimizations: false,
        }
    }
}

impl From<PlanContext> for SerializedPlanContext {
    fn from(cx: PlanContext) -> SerializedPlanContext {
        SerializedPlanContext {
            logical_time: None,
            wall_time: Some(cx.wall_time),
        }
    }
}

impl ConnCatalog<'_> {
    fn resolve_item_name(&self, name: &PartialName) -> Result<&FullName, SqlCatalogError> {
        self.resolve_item(name).map(|entry| entry.name())
    }

    fn minimal_qualification(&self, full_name: &FullName) -> PartialName {
        let database = match &full_name.database {
            DatabaseSpecifier::Ambient => None,
            DatabaseSpecifier::Name(n) if *n == self.database => None,
            DatabaseSpecifier::Name(n) => Some(n.clone()),
        };

        let schema = if database.is_none()
            && self.resolve_item_name(&PartialName {
                database: None,
                schema: None,
                item: full_name.item.clone(),
            }) == Ok(full_name)
        {
            None
        } else {
            // If `search_path` does not contain `full_name.schema`, the
            // `PartialName` must contain it.
            Some(full_name.schema.clone())
        };

        let res = PartialName {
            database,
            schema,
            item: full_name.item.clone(),
        };
        assert_eq!(self.resolve_item_name(&res), Ok(full_name));
        res
    }
}

impl ExprHumanizer for ConnCatalog<'_> {
    fn humanize_id(&self, id: GlobalId) -> Option<String> {
        self.catalog
            .state
            .by_id
            .get(&id)
            .map(|entry| entry.name.to_string())
    }

    fn humanize_scalar_type(&self, typ: &ScalarType) -> String {
        use ScalarType::*;

        match typ {
            Array(t) => format!("{}[]", self.humanize_scalar_type(t)),
            List { custom_oid, .. } | Map { custom_oid, .. } if custom_oid.is_some() => {
                let full_name = self.get_item_by_oid(&custom_oid.unwrap()).name();
                self.minimal_qualification(full_name).to_string()
            }
            List { element_type, .. } => {
                format!("{} list", self.humanize_scalar_type(element_type))
            }
            Map { value_type, .. } => format!(
                "map[{}=>{}]",
                self.humanize_scalar_type(&ScalarType::String),
                self.humanize_scalar_type(value_type)
            ),
            Record { fields, .. } => format!(
                "record({})",
                fields
                    .iter()
                    .map(|f| format!("{}: {}", f.0, self.humanize_column_type(&f.1)))
                    .join(",")
            ),
            ty => {
                let pgrepr_type = pgrepr::Type::from(ty);
                let res = if self
                    .search_path
                    .iter()
                    .any(|schema| schema == &PG_CATALOG_SCHEMA)
                {
                    pgrepr_type.name().to_string()
                } else {
                    // If PG_CATALOG_SCHEMA is not in search path, you need
                    // qualified object name to refer to type.
                    self.get_item_by_oid(&pgrepr_type.oid()).name().to_string()
                };
                res
            }
        }
    }
}

impl SessionCatalog for ConnCatalog<'_> {
    fn search_path(&self, include_system_schemas: bool) -> Vec<&str> {
        if include_system_schemas {
            self.search_path.to_vec()
        } else {
            self.search_path
                .iter()
                .filter(|s| {
                    (**s != PG_CATALOG_SCHEMA)
                        && (**s != INFORMATION_SCHEMA)
                        && (**s != MZ_CATALOG_SCHEMA)
                        && (**s != MZ_TEMP_SCHEMA)
                        && (**s != MZ_INTERNAL_SCHEMA)
                })
                .cloned()
                .collect()
        }
    }

    fn user(&self) -> &str {
        &self.user
    }

    fn get_prepared_statement_desc(&self, name: &str) -> Option<&StatementDesc> {
        self.prepared_statements
            .map(|ps| ps.get(name).map(|ps| ps.desc()))
            .flatten()
    }

    fn default_database(&self) -> &str {
        &self.database
    }

    fn resolve_database(
        &self,
        database_name: &str,
    ) -> Result<&dyn sql::catalog::CatalogDatabase, SqlCatalogError> {
        match self.catalog.state.by_name.get(database_name) {
            Some(database) => Ok(database),
            None => Err(SqlCatalogError::UnknownDatabase(database_name.into())),
        }
    }

    fn resolve_schema(
        &self,
        database: Option<String>,
        schema_name: &str,
    ) -> Result<&dyn sql::catalog::CatalogSchema, SqlCatalogError> {
        Ok(self
            .catalog
            .resolve_schema(&self.database, database, schema_name, self.conn_id)?)
    }

    fn resolve_role(
        &self,
        role_name: &str,
    ) -> Result<&dyn sql::catalog::CatalogRole, SqlCatalogError> {
        match self.catalog.state.roles.get(role_name) {
            Some(role) => Ok(role),
            None => Err(SqlCatalogError::UnknownRole(role_name.into())),
        }
    }

    fn resolve_item(
        &self,
        name: &PartialName,
    ) -> Result<&dyn sql::catalog::CatalogItem, SqlCatalogError> {
        Ok(self
            .catalog
            .resolve_item(&self.database, self.search_path, name, self.conn_id)?)
    }

    fn resolve_function(
        &self,
        name: &PartialName,
    ) -> Result<&dyn sql::catalog::CatalogItem, SqlCatalogError> {
        Ok(self
            .catalog
            .resolve_function(&self.database, self.search_path, name, self.conn_id)?)
    }

    fn try_get_item_by_id(&self, id: &GlobalId) -> Option<&dyn sql::catalog::CatalogItem> {
        self.catalog
            .try_get_by_id(*id)
            .map(|item| item as &dyn sql::catalog::CatalogItem)
    }

    fn get_item_by_id(&self, id: &GlobalId) -> &dyn sql::catalog::CatalogItem {
        self.catalog.get_by_id(id)
    }

    fn get_item_by_oid(&self, oid: &u32) -> &dyn sql::catalog::CatalogItem {
        let id = self.catalog.state.by_oid[oid];
        self.catalog.get_by_id(&id)
    }

    fn item_exists(&self, name: &FullName) -> bool {
        self.catalog.try_get(name, self.conn_id).is_some()
    }

    fn try_get_lossy_scalar_type_by_id(&self, id: &GlobalId) -> Option<ScalarType> {
        let entry = self.catalog.get_by_id(id);
        let t = match entry.item() {
            CatalogItem::Type(t) => t,
            _ => return None,
        };

        Some(match t.inner {
            TypeInner::Array { element_id } => {
                let element_type = self
                    .try_get_lossy_scalar_type_by_id(&element_id)
                    .expect("array's element_id refers to a valid type");
                ScalarType::Array(Box::new(element_type))
            }
            TypeInner::Base => pgrepr::Type::from_oid(entry.oid())?.to_scalar_type_lossy(),
            TypeInner::List { element_id } => {
                let element_type = self
                    .try_get_lossy_scalar_type_by_id(&element_id)
                    .expect("list's element_id refers to a valid type");
                ScalarType::List {
                    element_type: Box::new(element_type),
                    custom_oid: Some(entry.oid),
                }
            }
            TypeInner::Map { key_id, value_id } => {
                let key_type = self
                    .try_get_lossy_scalar_type_by_id(&key_id)
                    .expect("map's key_id refers to a valid type");
                assert!(matches!(key_type, ScalarType::String));
                let value_type = Box::new(
                    self.try_get_lossy_scalar_type_by_id(&value_id)
                        .expect("map's value_id refers to a valid type"),
                );
                ScalarType::Map {
                    value_type,
                    custom_oid: Some(entry.oid),
                }
            }
            TypeInner::Pseudo => return None,
        })
    }

    fn config(&self) -> &sql::catalog::CatalogConfig {
        &self.catalog.config
    }

    fn now(&self) -> EpochMillis {
        (self.catalog.config.now)()
    }
}

impl sql::catalog::CatalogDatabase for Database {
    fn name(&self) -> &str {
        &self.name
    }

    fn id(&self) -> i64 {
        self.id
    }

    fn has_schemas(&self) -> bool {
        !self.schemas.is_empty()
    }
}

impl sql::catalog::CatalogSchema for Schema {
    fn name(&self) -> &SchemaName {
        &self.name
    }

    fn id(&self) -> i64 {
        self.id
    }

    fn has_items(&self) -> bool {
        !self.items.is_empty()
    }
}

impl sql::catalog::CatalogRole for Role {
    fn name(&self) -> &str {
        &self.name
    }

    fn id(&self) -> i64 {
        self.id
    }
}

impl sql::catalog::CatalogItem for CatalogEntry {
    fn name(&self) -> &FullName {
        self.name()
    }

    fn id(&self) -> GlobalId {
        self.id()
    }

    fn oid(&self) -> u32 {
        self.oid()
    }

    fn desc(&self) -> Result<&RelationDesc, SqlCatalogError> {
        Ok(self.desc()?)
    }

    fn func(&self) -> Result<&'static sql::func::Func, SqlCatalogError> {
        Ok(self.func()?)
    }

    fn source_connector(&self) -> Result<&SourceConnector, SqlCatalogError> {
        Ok(self.source_connector()?)
    }

    fn create_sql(&self) -> &str {
        match self.item() {
            CatalogItem::Table(Table { create_sql, .. }) => create_sql,
            CatalogItem::Source(Source { create_sql, .. }) => create_sql,
            CatalogItem::Sink(Sink { create_sql, .. }) => create_sql,
            CatalogItem::View(View { create_sql, .. }) => create_sql,
            CatalogItem::Index(Index { create_sql, .. }) => create_sql,
            CatalogItem::Type(Type { create_sql, .. }) => create_sql,
            CatalogItem::Func(_) => "TODO",
        }
    }

    fn item_type(&self) -> SqlCatalogItemType {
        self.item().typ()
    }

    fn index_details(&self) -> Option<(&[MirScalarExpr], GlobalId)> {
        if let CatalogItem::Index(Index { keys, on, .. }) = self.item() {
            Some((keys, *on))
        } else {
            None
        }
    }

    fn table_details(&self) -> Option<&[Expr<Raw>]> {
        if let CatalogItem::Table(Table { defaults, .. }) = self.item() {
            Some(defaults)
        } else {
            None
        }
    }

    fn uses(&self) -> &[GlobalId] {
        self.uses()
    }

    fn used_by(&self) -> &[GlobalId] {
        self.used_by()
    }
}

#[cfg(test)]
mod tests {
    use tempfile::NamedTempFile;

    use ore::now::NOW_ZERO;
    use sql::names::{DatabaseSpecifier, FullName, PartialName};

    use crate::catalog::{Catalog, Op, MZ_CATALOG_SCHEMA, PG_CATALOG_SCHEMA};
    use crate::session::Session;

    /// System sessions have an empty `search_path` so it's necessary to
    /// schema-qualify all referenced items.
    ///
    /// Dummy (and ostensibly client) sessions contain system schemas in their
    /// search paths, so do not require schema qualification on system objects such
    /// as types.
    #[tokio::test]
    async fn test_minimal_qualification() -> Result<(), anyhow::Error> {
        struct TestCase {
            input: FullName,
            system_output: PartialName,
            normal_output: PartialName,
        }

        let test_cases = vec![
            TestCase {
                input: FullName {
                    database: DatabaseSpecifier::Ambient,
                    schema: PG_CATALOG_SCHEMA.to_string(),
                    item: "numeric".to_string(),
                },
                system_output: PartialName {
                    database: None,
                    schema: Some(PG_CATALOG_SCHEMA.to_string()),
                    item: "numeric".to_string(),
                },
                normal_output: PartialName {
                    database: None,
                    schema: None,
                    item: "numeric".to_string(),
                },
            },
            TestCase {
                input: FullName {
                    database: DatabaseSpecifier::Ambient,
                    schema: MZ_CATALOG_SCHEMA.to_string(),
                    item: "mz_array_types".to_string(),
                },
                system_output: PartialName {
                    database: None,
                    schema: Some(MZ_CATALOG_SCHEMA.to_string()),
                    item: "mz_array_types".to_string(),
                },
                normal_output: PartialName {
                    database: None,
                    schema: None,
                    item: "mz_array_types".to_string(),
                },
            },
        ];

        let catalog_file = NamedTempFile::new()?;
        let catalog = Catalog::open_debug(catalog_file.path(), NOW_ZERO.clone()).await?;
        for tc in test_cases {
            assert_eq!(
                catalog
                    .for_system_session()
                    .minimal_qualification(&tc.input),
                tc.system_output
            );
            assert_eq!(
                catalog
                    .for_session(&Session::dummy())
                    .minimal_qualification(&tc.input),
                tc.normal_output
            );
        }
        Ok(())
    }

    #[tokio::test]
    async fn test_catalog_revision() -> Result<(), anyhow::Error> {
        let catalog_file = NamedTempFile::new()?;
        let mut catalog = Catalog::open_debug(catalog_file.path(), NOW_ZERO.clone()).await?;
        assert_eq!(catalog.transient_revision(), 1);
        catalog
            .transact(
                vec![Op::CreateDatabase {
                    name: "test".to_string(),
                    oid: 1,
                }],
                |_catalog| Ok(()),
            )
            .unwrap();
        assert_eq!(catalog.transient_revision(), 2);
        drop(catalog);

        let catalog = Catalog::open_debug(catalog_file.path(), NOW_ZERO.clone()).await?;
        assert_eq!(catalog.transient_revision(), 1);

        Ok(())
    }
}