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
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258
3259
3260
3261
3262
3263
3264
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274
3275
3276
3277
3278
3279
3280
3281
3282
3283
3284
3285
3286
3287
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311
3312
3313
3314
3315
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363
3364
3365
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375
3376
3377
3378
3379
3380
3381
3382
3383
3384
3385
3386
3387
3388
3389
3390
3391
3392
3393
3394
3395
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405
3406
3407
3408
3409
3410
3411
3412
3413
3414
3415
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
3426
3427
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442
3443
3444
3445
3446
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458
3459
3460
3461
3462
3463
3464
3465
3466
3467
3468
3469
3470
3471
3472
3473
3474
3475
3476
3477
3478
3479
3480
3481
3482
3483
3484
3485
3486
3487
3488
3489
3490
3491
3492
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516
3517
3518
3519
3520
3521
3522
3523
3524
3525
3526
3527
3528
3529
3530
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548
3549
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599
3600
3601
3602
3603
3604
3605
3606
3607
3608
3609
3610
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636
3637
3638
3639
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651
3652
3653
3654
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691
3692
3693
3694
3695
3696
3697
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711
3712
3713
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745
3746
3747
3748
3749
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
3767
3768
3769
3770
3771
3772
3773
3774
3775
3776
3777
3778
3779
3780
3781
3782
3783
3784
3785
3786
3787
3788
3789
3790
3791
3792
3793
3794
3795
3796
3797
3798
3799
3800
3801
3802
3803
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818
3819
3820
3821
3822
3823
3824
3825
3826
3827
3828
3829
3830
3831
3832
3833
3834
3835
3836
3837
3838
3839
3840
3841
3842
3843
3844
3845
3846
3847
3848
3849
3850
3851
3852
3853
3854
3855
3856
3857
3858
3859
3860
3861
3862
3863
3864
3865
3866
3867
3868
3869
3870
3871
3872
3873
3874
3875
3876
3877
3878
3879
3880
3881
3882
3883
3884
3885
3886
3887
3888
3889
3890
3891
3892
3893
3894
3895
3896
3897
3898
3899
3900
3901
3902
3903
3904
3905
3906
3907
3908
3909
3910
3911
3912
3913
3914
3915
3916
3917
3918
3919
3920
3921
3922
3923
3924
3925
3926
3927
3928
3929
3930
3931
3932
3933
3934
3935
3936
3937
3938
3939
3940
3941
3942
3943
3944
3945
3946
3947
3948
3949
3950
3951
3952
3953
3954
3955
3956
3957
3958
3959
3960
3961
3962
3963
3964
3965
3966
3967
3968
3969
3970
3971
3972
3973
3974
3975
3976
3977
3978
3979
3980
3981
3982
3983
3984
3985
3986
3987
3988
3989
3990
3991
3992
3993
3994
3995
3996
3997
3998
3999
4000
4001
4002
4003
4004
4005
4006
4007
4008
4009
4010
4011
4012
4013
4014
4015
4016
4017
4018
4019
4020
4021
4022
4023
4024
4025
4026
4027
4028
4029
4030
4031
4032
4033
4034
4035
4036
4037
4038
4039
4040
4041
4042
4043
4044
4045
4046
4047
4048
4049
4050
4051
4052
4053
4054
4055
4056
4057
4058
4059
4060
4061
4062
4063
4064
4065
4066
4067
4068
4069
4070
4071
4072
4073
4074
4075
4076
4077
4078
4079
4080
4081
4082
4083
4084
4085
4086
4087
4088
4089
4090
4091
4092
4093
4094
4095
4096
4097
4098
4099
4100
4101
4102
4103
4104
4105
4106
4107
4108
4109
4110
4111
4112
4113
4114
4115
4116
4117
4118
4119
4120
4121
4122
4123
4124
4125
4126
4127
4128
4129
4130
4131
4132
4133
4134
4135
4136
4137
4138
4139
4140
4141
4142
4143
4144
4145
4146
4147
4148
4149
4150
4151
4152
4153
4154
4155
4156
4157
4158
4159
4160
4161
4162
4163
4164
4165
4166
4167
4168
4169
4170
4171
4172
4173
4174
4175
4176
4177
4178
4179
4180
4181
4182
4183
4184
4185
4186
4187
4188
4189
4190
4191
4192
4193
4194
4195
4196
4197
4198
4199
4200
4201
4202
4203
4204
4205
4206
4207
4208
4209
4210
4211
4212
4213
4214
4215
4216
4217
4218
4219
4220
4221
4222
4223
4224
4225
4226
4227
4228
4229
4230
4231
4232
4233
4234
4235
4236
4237
4238
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248
4249
4250
4251
4252
4253
4254
4255
4256
4257
4258
4259
4260
4261
4262
4263
4264
4265
4266
4267
4268
4269
4270
4271
4272
4273
4274
4275
4276
4277
4278
4279
4280
4281
4282
4283
4284
4285
4286
4287
4288
4289
4290
4291
4292
4293
4294
4295
4296
4297
4298
4299
4300
4301
4302
4303
4304
4305
4306
4307
4308
4309
4310
4311
4312
4313
4314
4315
4316
4317
4318
4319
4320
4321
4322
4323
4324
4325
4326
4327
4328
4329
4330
4331
4332
4333
4334
4335
4336
4337
4338
4339
4340
4341
4342
4343
4344
4345
4346
4347
4348
4349
4350
4351
4352
4353
4354
4355
4356
4357
4358
4359
4360
4361
4362
4363
4364
4365
4366
4367
4368
4369
4370
4371
4372
4373
4374
4375
4376
4377
4378
4379
4380
4381
4382
4383
4384
4385
4386
4387
4388
4389
4390
4391
4392
4393
4394
4395
4396
4397
4398
4399
4400
4401
4402
4403
4404
4405
4406
4407
4408
4409
4410
4411
4412
4413
4414
4415
4416
4417
4418
4419
4420
4421
4422
4423
4424
4425
4426
4427
4428
4429
4430
4431
4432
4433
4434
4435
4436
4437
4438
4439
4440
4441
4442
4443
4444
4445
4446
4447
4448
4449
4450
4451
4452
4453
4454
4455
4456
4457
4458
4459
4460
4461
4462
4463
4464
4465
4466
4467
4468
4469
4470
4471
4472
4473
4474
4475
4476
4477
4478
4479
4480
4481
4482
4483
4484
4485
4486
4487
4488
4489
4490
4491
4492
4493
4494
4495
4496
4497
4498
4499
4500
4501
4502
4503
4504
4505
4506
4507
4508
4509
4510
4511
4512
4513
4514
4515
4516
4517
4518
4519
4520
4521
4522
4523
4524
4525
4526
4527
4528
4529
4530
4531
4532
4533
4534
4535
4536
4537
4538
4539
4540
4541
4542
4543
4544
4545
4546
4547
4548
4549
4550
4551
4552
4553
4554
4555
4556
4557
4558
4559
4560
4561
4562
4563
4564
4565
4566
4567
4568
4569
4570
4571
4572
4573
4574
4575
4576
4577
4578
4579
4580
4581
4582
4583
4584
4585
4586
4587
4588
4589
4590
4591
4592
4593
4594
4595
4596
4597
4598
4599
4600
4601
4602
4603
4604
4605
4606
4607
4608
4609
4610
4611
4612
4613
4614
4615
4616
4617
4618
4619
4620
4621
4622
4623
4624
4625
4626
4627
4628
4629
4630
4631
4632
4633
4634
4635
4636
4637
4638
4639
4640
4641
4642
4643
4644
4645
4646
4647
4648
4649
4650
4651
4652
4653
4654
4655
4656
4657
4658
4659
4660
4661
4662
4663
4664
4665
4666
4667
4668
4669
4670
4671
4672
4673
4674
4675
4676
4677
4678
4679
4680
4681
4682
4683
4684
4685
4686
4687
4688
4689
4690
4691
4692
4693
4694
4695
4696
4697
4698
4699
4700
4701
4702
4703
4704
4705
4706
4707
4708
4709
4710
4711
4712
4713
4714
4715
4716
4717
4718
4719
4720
4721
4722
4723
4724
4725
4726
4727
4728
4729
4730
4731
4732
4733
4734
4735
4736
4737
4738
4739
4740
4741
4742
4743
4744
4745
4746
4747
4748
4749
4750
4751
4752
4753
4754
4755
4756
4757
4758
4759
4760
4761
4762
4763
4764
4765
4766
4767
4768
4769
4770
4771
4772
4773
4774
4775
4776
4777
4778
4779
4780
4781
4782
4783
4784
4785
4786
4787
4788
4789
4790
4791
4792
4793
4794
4795
4796
4797
4798
4799
4800
4801
4802
4803
4804
4805
4806
4807
4808
4809
4810
4811
4812
4813
4814
4815
4816
4817
4818
4819
4820
4821
4822
4823
4824
4825
4826
4827
4828
4829
4830
4831
4832
4833
4834
4835
4836
4837
4838
4839
4840
4841
4842
4843
4844
4845
4846
4847
4848
4849
4850
4851
4852
4853
4854
4855
4856
4857
4858
4859
4860
4861
4862
4863
4864
4865
4866
4867
4868
4869
4870
4871
4872
4873
4874
4875
4876
4877
4878
4879
4880
4881
4882
4883
4884
4885
4886
4887
4888
4889
4890
4891
4892
4893
4894
4895
4896
4897
4898
4899
4900
4901
4902
4903
4904
4905
4906
4907
4908
4909
4910
4911
4912
4913
4914
4915
4916
4917
4918
4919
4920
4921
4922
4923
4924
4925
4926
4927
4928
4929
4930
4931
4932
4933
4934
4935
4936
4937
4938
4939
4940
4941
4942
4943
4944
4945
4946
4947
4948
4949
4950
4951
4952
4953
4954
4955
4956
4957
4958
4959
4960
4961
4962
4963
4964
4965
4966
4967
4968
4969
4970
4971
4972
4973
4974
4975
4976
4977
4978
4979
4980
4981
4982
4983
4984
4985
4986
4987
4988
4989
4990
4991
4992
4993
4994
4995
4996
4997
4998
4999
5000
5001
5002
5003
5004
5005
5006
5007
5008
5009
5010
5011
5012
5013
5014
5015
5016
5017
5018
5019
5020
5021
5022
5023
5024
5025
5026
5027
5028
5029
5030
5031
5032
5033
5034
5035
5036
5037
5038
5039
5040
5041
5042
5043
5044
5045
5046
5047
5048
5049
5050
5051
5052
5053
5054
5055
5056
5057
5058
5059
5060
5061
5062
5063
5064
5065
5066
5067
5068
5069
5070
5071
5072
5073
5074
5075
5076
5077
5078
5079
5080
5081
5082
5083
5084
5085
5086
5087
5088
5089
5090
5091
5092
5093
5094
5095
5096
5097
5098
5099
5100
5101
5102
5103
5104
5105
5106
5107
5108
5109
5110
5111
5112
5113
5114
5115
5116
5117
5118
5119
5120
5121
5122
5123
5124
5125
5126
5127
5128
5129
5130
5131
5132
5133
5134
5135
5136
5137
5138
5139
5140
5141
5142
5143
5144
5145
5146
5147
5148
5149
// 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.

//! Coordination of installed views, available timestamps, compacted
//! timestamps, and transactions.
//!
//! The command coordinator maintains a view of the installed views, and for
//! each tracks the frontier of available times
//! ([`upper`](arrangement_state::Frontiers::upper)) and the frontier of
//! compacted times ([`since`](arrangement_state::Frontiers::since)). The upper
//! frontier describes times that may not return immediately, as any timestamps
//! in advance of the frontier are still open. The since frontier constrains
//! those times for which the maintained view will be correct, as any
//! timestamps in advance of the frontier must accumulate to the same value as
//! would an un-compacted trace. The since frontier cannot be directly mutated,
//! but instead can have multiple handles to it which forward changes from an
//! internal MutableAntichain to the since.
//!
//! The [`Coordinator`] tracks various compaction frontiers so that source,
//! indexes, compaction, and transactions can work together.
//! [`determine_timestamp()`](Coordinator::determine_timestamp)
//! returns the least valid since of its sources. Any new transactions should
//! thus always be >= the current compaction frontier and so should never change
//! the frontier when being added to [`txn_reads`](Coordinator::txn_reads). The
//! compaction frontier may change when a transaction ends (if it was the oldest
//! transaction and the since was advanced after the transaction started) or
//! when [`update_upper()`](Coordinator::update_upper) is run (if there
//! are no in progress transactions before the new since). When it does, it is
//! added to [`index_since_updates`](Coordinator::index_since_updates) or
//! [`source_since_updates`](Coordinator::source_since_updates) and will be
//! processed during the next [`maintenance()`](Coordinator::maintenance) call.
//!
//! ## Frontiers another way
//!
//! If the above description of frontiers left you with questions, this
//! repackaged explanation might help.
//!
//! - `since` is the least recent time (i.e. oldest time) that you can read
//!   from sources and be guaranteed that the returned data is accurate as of
//!   that time.
//!
//!   Reads at times less than `since` may return values that were not actually
//!   seen at the specified time, but arrived later (i.e. the results are
//!   compacted).
//!
//!   For correctness' sake, the coordinator never chooses to read at a time
//!   less than an arrangement's `since`.
//!
//! - `upper` is the first time after the most recent time that you can read
//!   from sources and receive an immediate response. Alternately, it is the
//!   least time at which the data may still change (that is the reason we may
//!   not be able to respond immediately).
//!
//!   Reads at times >= `upper` may not immediately return because the answer
//!   isn't known yet. However, once the `upper` is > the specified read time,
//!   the read can return.
//!
//!   For the sake of returned values' freshness, the coordinator prefers
//!   performing reads at an arrangement's `upper`. However, because we more
//!   strongly prefer correctness, the coordinator will choose timestamps
//!   greater than an object's `upper` if it is also being accessed alongside
//!   objects whose `since` times are >= its `upper`.
//!
//! This illustration attempts to show, with time moving left to right, the
//! relationship between `since` and `upper`.
//!
//! - `#`: possibly inaccurate results
//! - `-`: immediate, correct response
//! - `?`: not yet known
//! - `s`: since
//! - `u`: upper
//! - `|`: eligible for coordinator to select
//!
//! ```nofmt
//! ####s----u?????
//!     |||||||||||
//! ```
//!

use std::cell::RefCell;
use std::collections::{BTreeMap, HashMap, HashSet, VecDeque};
use std::future::Future;
use std::rc::Rc;
use std::sync::Arc;
use std::thread;
use std::time::Duration;

use anyhow::{anyhow, Context};
use chrono::{DateTime, Utc};
use derivative::Derivative;
use differential_dataflow::lattice::Lattice;
use futures::future::{self, FutureExt, TryFutureExt};
use futures::stream::StreamExt;
use itertools::Itertools;
use rand::Rng;
use repr::adt::interval::Interval;
use timely::order::PartialOrder;
use timely::progress::frontier::MutableAntichain;
use timely::progress::{Antichain, ChangeBatch, Timestamp as _};
use tokio::runtime::Handle as TokioHandle;
use tokio::select;
use tokio::sync::{mpsc, oneshot, watch};

use build_info::BuildInfo;
use dataflow_types::client::{ComputeClient, StorageClient, DEFAULT_COMPUTE_INSTANCE_ID};
use dataflow_types::client::{ComputeResponse, TimestampBindingFeedback};
use dataflow_types::client::{Response as DataflowResponse, StorageResponse};
use dataflow_types::logging::LoggingConfig as DataflowLoggingConfig;
use dataflow_types::sinks::{SinkAsOf, SinkConnector, SinkDesc, TailSinkConnector};
use dataflow_types::sources::{
    ExternalSourceConnector, PostgresSourceConnector, SourceConnector, Timeline,
};
use dataflow_types::{DataflowDesc, DataflowDescription, IndexDesc, PeekResponse, Update};
use expr::{
    permutation_for_arrangement, GlobalId, MirRelationExpr, MirScalarExpr, NullaryFunc,
    OptimizedMirRelationExpr, RowSetFinishing,
};
use ore::metrics::MetricsRegistry;
use ore::now::{to_datetime, NowFn};
use ore::retry::Retry;
use ore::soft_assert_eq;
use ore::task;
use ore::thread::JoinHandleExt;
use repr::adt::numeric;
use repr::{Datum, Diff, RelationDesc, Row, RowArena, Timestamp};
use sql::ast::display::AstDisplay;
use sql::ast::{
    ConnectorType, CreateIndexStatement, CreateSinkStatement, CreateSourceStatement, ExplainStage,
    FetchStatement, Ident, InsertSource, ObjectType, Query, Raw, SetExpr, Statement,
};
use sql::catalog::{CatalogError, SessionCatalog as _};
use sql::names::{DatabaseSpecifier, FullName};
use sql::plan::{
    AlterIndexEnablePlan, AlterIndexResetOptionsPlan, AlterIndexSetOptionsPlan,
    AlterItemRenamePlan, CreateDatabasePlan, CreateIndexPlan, CreateRolePlan, CreateSchemaPlan,
    CreateSinkPlan, CreateSourcePlan, CreateTablePlan, CreateTypePlan, CreateViewPlan,
    CreateViewsPlan, DropDatabasePlan, DropItemsPlan, DropRolesPlan, DropSchemaPlan, ExecutePlan,
    ExplainPlan, FetchPlan, HirRelationExpr, IndexOption, IndexOptionName, InsertPlan,
    MutationKind, Params, PeekPlan, PeekWhen, Plan, ReadThenWritePlan, SendDiffsPlan,
    SetVariablePlan, ShowVariablePlan, TailFrom, TailPlan,
};
use sql::plan::{OptimizerConfig, StatementDesc, View};
use transform::Optimizer;

use self::arrangement_state::{ArrangementFrontiers, Frontiers, SinkWrites};
use self::prometheus::Scraper;
use crate::catalog::builtin::{BUILTINS, MZ_VIEW_FOREIGN_KEYS, MZ_VIEW_KEYS};
use crate::catalog::{
    self, storage, BuiltinTableUpdate, Catalog, CatalogItem, CatalogState, SinkConnectorState,
};
use crate::client::{Client, Handle};
use crate::command::{
    Canceled, Command, ExecuteResponse, Response, StartupMessage, StartupResponse,
};
use crate::coord::antichain::AntichainToken;
use crate::coord::dataflow_builder::DataflowBuilder;
use crate::error::CoordError;
use crate::persistcfg::PersisterWithConfig;
use crate::session::{
    EndTransactionAction, PreparedStatement, Session, Transaction, TransactionOps,
    TransactionStatus, WriteOp,
};
use crate::sink_connector;
use crate::tail::PendingTail;
use crate::util::ClientTransmitter;

mod antichain;
mod arrangement_state;
mod dataflow_builder;
mod prometheus;

#[derive(Debug)]
pub enum Message {
    Command(Command),
    Worker(dataflow_types::client::Response),
    StatementReady(StatementReady),
    SinkConnectorReady(SinkConnectorReady),
    ScrapeMetrics,
    SendDiffs(SendDiffs),
    WriteLockGrant(tokio::sync::OwnedMutexGuard<()>),
    AdvanceLocalInputs,
}

#[derive(Derivative)]
#[derivative(Debug)]
pub struct SendDiffs {
    session: Session,
    #[derivative(Debug = "ignore")]
    tx: ClientTransmitter<ExecuteResponse>,
    pub id: GlobalId,
    pub diffs: Result<Vec<(Row, Diff)>, CoordError>,
    pub kind: MutationKind,
}

#[derive(Derivative)]
#[derivative(Debug)]
pub struct StatementReady {
    pub session: Session,
    #[derivative(Debug = "ignore")]
    pub tx: ClientTransmitter<ExecuteResponse>,
    pub result: Result<sql::ast::Statement<Raw>, CoordError>,
    pub params: Params,
}

/// This is the struct meant to be paired with [`Message::WriteLockGrant`], but
/// could theoretically be used to queue any deferred plan.
#[derive(Derivative)]
#[derivative(Debug)]
pub struct DeferredPlan {
    #[derivative(Debug = "ignore")]
    pub tx: ClientTransmitter<ExecuteResponse>,
    pub session: Session,
    pub plan: Plan,
}

#[derive(Derivative)]
#[derivative(Debug)]
pub struct SinkConnectorReady {
    pub session: Session,
    #[derivative(Debug = "ignore")]
    pub tx: ClientTransmitter<ExecuteResponse>,
    pub id: GlobalId,
    pub oid: u32,
    pub result: Result<SinkConnector, CoordError>,
}

#[derive(Debug)]
pub struct TimestampedUpdate {
    pub updates: Vec<BuiltinTableUpdate>,
    pub timestamp_offset: u64,
}

/// Configures dataflow worker logging.
#[derive(Clone, Debug)]
pub struct LoggingConfig {
    pub granularity: Duration,
    pub log_logging: bool,
    pub retain_readings_for: Duration,
    pub metrics_scraping_interval: Option<Duration>,
}

/// Configures a coordinator.
pub struct Config {
    pub dataflow_client: Box<dyn dataflow_types::client::Client>,
    pub logging: Option<LoggingConfig>,
    pub storage: storage::Connection,
    pub timestamp_frequency: Duration,
    pub logical_compaction_window: Option<Duration>,
    pub experimental_mode: bool,
    pub disable_user_indexes: bool,
    pub safe_mode: bool,
    pub build_info: &'static BuildInfo,
    pub aws_external_id: Option<String>,
    pub metrics_registry: MetricsRegistry,
    pub persister: PersisterWithConfig,
    pub now: NowFn,
}

/// Glues the external world to the Timely workers.
pub struct Coordinator {
    /// A client to a running dataflow cluster.
    dataflow_client: dataflow_types::client::Controller<Box<dyn dataflow_types::client::Client>>,
    /// Optimizer instance for logical optimization of views.
    view_optimizer: Optimizer,
    catalog: Catalog,
    /// A runtime for the `persist` crate alongside its configuration.
    persister: PersisterWithConfig,

    /// Maps (global Id of arrangement) -> (frontier information). This tracks the
    /// `upper` and computed `since` of the indexes. The `since` is the time at
    /// which we are willing to compact up to. `determine_timestamp()` uses this as
    /// part of its heuristic when determining a viable timestamp for queries.
    indexes: ArrangementFrontiers<Timestamp>,
    /// Map of frontier information for sources
    sources: ArrangementFrontiers<Timestamp>,
    /// Delta from leading edge of an arrangement from which we allow compaction.
    logical_compaction_window_ms: Option<Timestamp>,
    /// Whether base sources are enabled.
    logging_enabled: bool,
    /// Channel to manange internal commands from the coordinator to itself.
    internal_cmd_tx: mpsc::UnboundedSender<Message>,
    /// Channel to communicate source status updates to the timestamper thread.
    metric_scraper: Scraper,

    /// The last known timestamp that was considered "open" (i.e. where writes
    /// may occur). However, this timestamp is _not_ open when
    /// `read_writes_at_open_ts == true`; in this case, reads will occur at
    /// `last_open_local_ts`, and the Coordinator must open a new timestamp
    /// for writes.
    ///
    /// Indirectly, this value aims to represent the Coordinator's desired value
    /// for `upper` for table frontiers, as long as we know it is open.
    last_open_local_ts: Timestamp,
    /// Whether or not we have written at the open timestamp.
    writes_at_open_ts: bool,
    /// Whether or not we have read the writes that have occurred at the open
    /// timestamp. When this is `true`, it signals we need to open a new
    /// timestamp to support future writes.
    read_writes_at_open_ts: bool,

    transient_id_counter: u64,
    /// A map from connection ID to metadata about that connection for all
    /// active connections.
    active_conns: HashMap<u32, ConnMeta>,

    /// Holds pending compaction messages to be sent to the dataflow workers. When
    /// `since_handles` are advanced or `txn_reads` are dropped, this can advance.
    index_since_updates: Rc<RefCell<HashMap<GlobalId, Antichain<Timestamp>>>>,
    /// Holds pending compaction messages to be sent to the dataflow workers. When
    /// `since_handles` are advanced or `txn_reads` are dropped, this can advance.
    source_since_updates: Rc<RefCell<HashMap<GlobalId, Antichain<Timestamp>>>>,
    /// Holds handles to ids that are advanced by update_upper.
    since_handles: HashMap<GlobalId, AntichainToken<Timestamp>>,
    /// Tracks active read transactions so that we don't compact any indexes beyond
    /// an in-progress transaction.
    // TODO(mjibson): Should this live on a Session?
    txn_reads: HashMap<u32, TxnReads>,
    /// Tracks write frontiers for active exactly-once sinks.
    sink_writes: HashMap<GlobalId, SinkWrites<Timestamp>>,

    /// A map from pending peeks to the queue into which responses are sent, and
    /// the IDs of workers who have responded.
    pending_peeks: HashMap<u32, mpsc::UnboundedSender<PeekResponse>>,
    /// A map from pending tails to the tail description.
    pending_tails: HashMap<GlobalId, PendingTail>,

    /// Serializes accesses to write critical sections.
    write_lock: Arc<tokio::sync::Mutex<()>>,
    /// Holds plans deferred due to write lock.
    write_lock_wait_group: VecDeque<DeferredPlan>,
}

/// Metadata about an active connection.
struct ConnMeta {
    /// A watch channel shared with the client to inform the client of
    /// cancellation requests. The coordinator sets the contained value to
    /// `Canceled::Canceled` whenever it receives a cancellation request that
    /// targets this connection. It is the client's responsibility to check this
    /// value when appropriate and to reset the value to
    /// `Canceled::NotCanceled` before starting a new operation.
    cancel_tx: Arc<watch::Sender<Canceled>>,
    /// Pgwire specifies that every connection have a 32-bit secret associated
    /// with it, that is known to both the client and the server. Cancellation
    /// requests are required to authenticate with the secret of the connection
    /// that they are targeting.
    secret_key: u32,
}

struct TxnReads {
    // True iff all statements run so far in the transaction are independent
    // of the chosen logical timestamp (not the PlanContext walltime). This
    // happens if both 1) there are no referenced sources or indexes and 2)
    // `mz_logical_timestamp()` is not present.
    timestamp_independent: bool,
    timedomain_ids: HashSet<GlobalId>,
    _handles: Vec<AntichainToken<Timestamp>>,
}

/// Enforces critical section invariants for functions that perform writes to
/// tables, e.g. `INSERT`, `UPDATE`.
///
/// If the provided session doesn't currently hold the write lock, attempts to
/// grant it. If the coord cannot immediately grant the write lock, defers
/// executing the provided plan until the write lock is available, and exits the
/// function.
///
/// # Parameters
/// - `$coord: &mut Coord`
/// - `$tx: ClientTransmitter<ExecuteResponse>`
/// - `mut $session: Session`
/// - `$plan_to_defer: Plan`
///
/// Note that making this a macro rather than a function lets us avoid taking
/// ownership of e.g. session and lets us unilaterally enforce the return when
/// deferring work.
macro_rules! guard_write_critical_section {
    ($coord:expr, $tx:expr, $session:expr, $plan_to_defer: expr) => {
        if !$session.has_write_lock() {
            if $coord.try_grant_session_write_lock(&mut $session).is_err() {
                $coord.defer_write($tx, $session, $plan_to_defer);
                return;
            }
        }
    };
}

impl Coordinator {
    /// Assign a timestamp for a read from a local input. Reads following writes
    /// must be at a time >= the write's timestamp; we choose "equal to" for
    /// simplicity's sake and to open as few new timestamps as possible.
    fn get_local_read_ts(&mut self) -> Timestamp {
        if self.writes_at_open_ts {
            // If you have pending writes, you will need to read those writes,
            // which happened at the last known open time. This also means you
            // will need to advance to those writes, i.e. close over
            // `last_open_local_ts`.
            self.read_writes_at_open_ts = true;
            self.last_open_local_ts
        } else {
            // If there are no writes at the open timestamp, we know we can read
            // at one unit of time less than the open time (which will always be
            // closed).
            self.last_open_local_ts - 1
        }
    }

    /// Assign a timestamp for a write to a local input. Writes following reads
    /// must ensure that they are assigned a strictly larger timestamp to ensure
    /// they are not visible to any real-time earlier reads.
    fn get_local_write_ts(&mut self) -> Timestamp {
        // This assert is valid because:
        // - Whenever a write precedes a read, the read sets
        //   `read_writes_at_open_ts = true`, which will advance the
        //   `last_open_local_ts`.
        // - The Coordinator always has the opportunity to check the state of
        //   `read_writes_at_open_ts` after a read, even in the case of
        //   `ReadThenWrite` plans, which dictates when we advance the
        //   timestamp.
        // - Advancing the timestamp sets `read_writes_at_open_ts = false`.
        assert!(
            !self.read_writes_at_open_ts,
            "do not perform writes at time where tables want to read"
        );

        self.writes_at_open_ts = true;

        self.last_open_local_ts
    }

    /// Opens a new timestamp for local inputs at which writes may occur, and
    /// where reads should return quickly at a value 1 less.
    fn open_new_local_ts(&mut self) {
        // This is a hack. In a perfect world we would represent time as having a "real" dimension
        // and a "coordinator" dimension so that clients always observed linearizability from
        // things the coordinator did without being related to the real dimension.
        let ts = (self.catalog.config().now)();

        // We cannot depend on `self.catalog.config().now`'s value to increase
        // (in addition to the normal considerations around clocks in computers,
        // this feature enables us to drive the Coordinator's time when using a
        // test harness). Instead, we must manually increment
        // `last_open_local_ts` if `now` appears non-increasing.
        self.last_open_local_ts = std::cmp::max(ts, self.last_open_local_ts + 1);

        // Opening a new timestamp means that there cannot be new writes at the
        // open timestamp.
        self.writes_at_open_ts = false;
        self.read_writes_at_open_ts = false;
    }

    fn now_datetime(&self) -> DateTime<Utc> {
        to_datetime((self.catalog.config().now)())
    }

    /// Generate a new frontiers object that forwards since changes to `index_since_updates`.
    ///
    /// # Panics
    ///
    /// This function panics if called twice with the same `id`.
    fn new_index_frontiers<I>(
        &mut self,
        id: GlobalId,
        initial: I,
        compaction_window_ms: Option<Timestamp>,
    ) -> Frontiers<Timestamp>
    where
        I: IntoIterator<Item = Timestamp>,
    {
        let index_since_updates = Rc::clone(&self.index_since_updates);
        let (frontier, handle) = Frontiers::new(initial, compaction_window_ms, move |frontier| {
            index_since_updates.borrow_mut().insert(id, frontier);
        });
        let prev = self.since_handles.insert(id, handle);
        // Ensure we don't double-register ids.
        assert!(prev.is_none());
        frontier
    }
    ///
    /// Generate a new frontiers object that forwards since changes to `source_since_updates`.
    ///
    /// # Panics
    ///
    /// This function panics if called twice with the same `id`.
    fn new_source_frontiers<I>(
        &mut self,
        id: GlobalId,
        initial: I,
        compaction_window_ms: Option<Timestamp>,
    ) -> Frontiers<Timestamp>
    where
        I: IntoIterator<Item = Timestamp>,
    {
        let storage_since_updates = Rc::clone(&self.source_since_updates);
        let (frontier, handle) = Frontiers::new(initial, compaction_window_ms, move |frontier| {
            storage_since_updates.borrow_mut().insert(id, frontier);
        });
        let prev = self.since_handles.insert(id, handle);
        // Ensure we don't double-register ids.
        assert!(prev.is_none());
        frontier
    }

    /// Initializes coordinator state based on the contained catalog. Must be
    /// called after creating the coordinator and before calling the
    /// `Coordinator::serve` method.
    async fn bootstrap(
        &mut self,
        builtin_table_updates: Vec<BuiltinTableUpdate>,
    ) -> Result<(), CoordError> {
        let entries: Vec<_> = self.catalog.entries().cloned().collect();

        // Sources and indexes may be depended upon by other catalog items,
        // insert them first.
        for entry in &entries {
            match entry.item() {
                // Currently catalog item rebuild assumes that sinks and
                // indexes are always built individually and does not store information
                // about how it was built. If we start building multiple sinks and/or indexes
                // using a single dataflow, we have to make sure the rebuild process re-runs
                // the same multiple-build dataflow.
                CatalogItem::Source(source) => {
                    // Inform the timestamper about this source.
                    self.update_timestamper(entry.id(), true).await;
                    let since_ts = self
                        .persister
                        .load_source_persist_desc(&source)
                        .map_err(CoordError::Persistence)?
                        .map(|p| p.since_ts)
                        .unwrap_or(0);

                    let frontiers = self.new_source_frontiers(
                        entry.id(),
                        [since_ts],
                        self.logical_compaction_window_ms,
                    );
                    self.sources.insert(entry.id(), frontiers);
                    // Re-announce the source description.
                    let source_description = self
                        .catalog
                        .state()
                        .source_description_for(entry.id())
                        .unwrap();
                    self.dataflow_client
                        .create_sources(vec![(
                            entry.id(),
                            (source_description, Antichain::from_elem(since_ts)),
                        )])
                        .await;
                }
                CatalogItem::Table(table) => {
                    self.persister
                        .add_table(entry.id(), &table)
                        .map_err(CoordError::Persistence)?;

                    let since_ts = self
                        .persister
                        .table_details
                        .get(&entry.id())
                        .map(|td| td.since_ts)
                        .unwrap_or(0);

                    let frontiers = self.new_source_frontiers(
                        entry.id(),
                        [since_ts],
                        self.logical_compaction_window_ms,
                    );

                    // NOTE: Tables are not sources, but to a large part of the system they look
                    // like they are, e.g. they are rendered as a SourceConnector::Local.
                    self.sources.insert(entry.id(), frontiers);
                    // Re-announce the source description.
                    let source_description = self
                        .catalog
                        .state()
                        .source_description_for(entry.id())
                        .unwrap();
                    self.dataflow_client
                        .create_sources(vec![(
                            entry.id(),
                            (source_description, Antichain::from_elem(since_ts)),
                        )])
                        .await;
                }
                CatalogItem::Index(_) => {
                    if BUILTINS.logs().any(|log| log.index_id == entry.id()) {
                        // Indexes on logging views are special, as they are
                        // already installed in the dataflow plane via
                        // `dataflow_types::client::Command::EnableLogging`. Just teach the
                        // coordinator of their existence, without creating a
                        // dataflow for the index.
                        //
                        // TODO(benesch): why is this hardcoded to 1000?
                        // Should it not be the same logical compaction window
                        // that everything else uses?
                        let frontiers = self.new_index_frontiers(entry.id(), Some(0), Some(1_000));
                        self.indexes.insert(entry.id(), frontiers);
                    } else {
                        let index_id = entry.id();
                        if let Some((name, description)) =
                            Self::prepare_index_build(self.catalog.state(), &index_id)
                        {
                            let df = self.dataflow_builder().build_index_dataflow(
                                name,
                                index_id,
                                description,
                            )?;
                            self.ship_dataflow(df).await;
                        }
                    }
                }
                _ => (), // Handled in next loop.
            }
        }

        for entry in entries {
            match entry.item() {
                CatalogItem::View(_) => (),
                CatalogItem::Sink(sink) => {
                    let builder = match &sink.connector {
                        SinkConnectorState::Pending(builder) => builder,
                        SinkConnectorState::Ready(_) => {
                            panic!("sink already initialized during catalog boot")
                        }
                    };
                    let connector = sink_connector::build(builder.clone(), entry.id())
                        .await
                        .with_context(|| format!("recreating sink {}", entry.name()))?;
                    self.handle_sink_connector_ready(entry.id(), entry.oid(), connector)
                        .await?;
                }
                _ => (), // Handled in prior loop.
            }
        }

        self.send_builtin_table_updates(builtin_table_updates).await;

        // Announce primary and foreign key relationships.
        if self.logging_enabled {
            for log in BUILTINS.logs() {
                let log_id = &log.id.to_string();
                self.send_builtin_table_updates(
                    log.variant
                        .desc()
                        .typ()
                        .keys
                        .iter()
                        .enumerate()
                        .flat_map(move |(index, key)| {
                            key.iter().map(move |k| {
                                let row = Row::pack_slice(&[
                                    Datum::String(log_id),
                                    Datum::Int64(*k as i64),
                                    Datum::Int64(index as i64),
                                ]);
                                BuiltinTableUpdate {
                                    id: MZ_VIEW_KEYS.id,
                                    row,
                                    diff: 1,
                                }
                            })
                        })
                        .collect(),
                )
                .await;

                self.send_builtin_table_updates(
                    log.variant
                        .foreign_keys()
                        .into_iter()
                        .enumerate()
                        .flat_map(move |(index, (parent, pairs))| {
                            let parent_id = BUILTINS
                                .logs()
                                .find(|src| src.variant == parent)
                                .unwrap()
                                .id
                                .to_string();
                            pairs.into_iter().map(move |(c, p)| {
                                let row = Row::pack_slice(&[
                                    Datum::String(&log_id),
                                    Datum::Int64(c as i64),
                                    Datum::String(&parent_id),
                                    Datum::Int64(p as i64),
                                    Datum::Int64(index as i64),
                                ]);
                                BuiltinTableUpdate {
                                    id: MZ_VIEW_FOREIGN_KEYS.id,
                                    row,
                                    diff: 1,
                                }
                            })
                        })
                        .collect(),
                )
                .await;
            }
        }

        Ok(())
    }

    /// Serves the coordinator, receiving commands from users over `cmd_rx`
    /// and feedback from dataflow workers over `feedback_rx`.
    ///
    /// You must call `bootstrap` before calling this method.
    async fn serve(
        mut self,
        mut internal_cmd_rx: mpsc::UnboundedReceiver<Message>,
        mut cmd_rx: mpsc::UnboundedReceiver<Command>,
    ) {
        {
            // An explicit SELECT or INSERT on a table will bump the table's timestamps,
            // but there are cases where timestamps are not bumped but we expect the closed
            // timestamps to advance (`AS OF now()`, TAILing views over RT sources and
            // tables). To address these, spawn a task that forces table timestamps to
            // close on a regular interval. This roughly tracks the behaivor of realtime
            // sources that close off timestamps on an interval.
            let internal_cmd_tx = self.internal_cmd_tx.clone();
            task::spawn(|| "coordinator_serve", async move {
                let mut interval = tokio::time::interval(tokio::time::Duration::from_millis(1_000));
                loop {
                    interval.tick().await;
                    // If sending fails, the main thread has shutdown.
                    if internal_cmd_tx.send(Message::AdvanceLocalInputs).is_err() {
                        break;
                    }
                }
            });
        }

        // Create the default cluster so that subsequent default cluster commands succeed.
        self.dataflow_client
            .create_instance(DEFAULT_COMPUTE_INSTANCE_ID)
            .await;

        let mut metric_scraper_stream = self.metric_scraper.tick_stream();

        loop {
            let msg = select! {
                // Order matters here. We want to process internal commands
                // before processing external commands.
                biased;

                Some(m) = internal_cmd_rx.recv() => m,
                Some(m) = self.dataflow_client.recv() => Message::Worker(m),
                Some(m) = metric_scraper_stream.next() => m,
                m = cmd_rx.recv() => match m {
                    None => break,
                    Some(m) => Message::Command(m),
                },
            };

            match msg {
                Message::Command(cmd) => self.message_command(cmd).await,
                Message::Worker(worker) => self.message_worker(worker).await,
                Message::StatementReady(ready) => self.message_statement_ready(ready).await,
                Message::SinkConnectorReady(ready) => {
                    self.message_sink_connector_ready(ready).await
                }
                Message::WriteLockGrant(write_lock_guard) => {
                    // It's possible to have more incoming write lock grants
                    // than pending writes because of cancellations.
                    if let Some(mut ready) = self.write_lock_wait_group.pop_front() {
                        ready.session.grant_write_lock(write_lock_guard);
                        self.sequence_plan(ready.tx, ready.session, ready.plan)
                            .await;
                    }
                    // N.B. if no deferred plans, write lock is released by drop
                    // here.
                }
                Message::SendDiffs(diffs) => self.message_send_diffs(diffs),
                Message::ScrapeMetrics => self.message_scrape_metrics().await,
                Message::AdvanceLocalInputs => {
                    // Convince the coordinator it needs to open a new timestamp
                    // and advance inputs.
                    self.read_writes_at_open_ts = true;
                }
            }

            if self.read_writes_at_open_ts {
                self.advance_local_inputs().await;
            }
        }
    }

    // Advance all local inputs (tables) to the current wall clock or at least
    // a time greater than any previous table read (if wall clock has gone
    // backward). This downgrades the capabilities of all tables, which means that
    // all tables can no longer produce new data before this timestamp.
    async fn advance_local_inputs(&mut self) {
        self.open_new_local_ts();

        // Close the stream up to the newly opened timestamp.
        let advance_to = self.last_open_local_ts;

        // Ensure that the persister is aware of exactly the set of tables for
        // which persistence is enabled.
        soft_assert_eq!(
            self.catalog
                .entries()
                .filter(|entry| matches!(
                    entry.item(),
                    CatalogItem::Table(catalog::Table {
                        persist_name: Some(_),
                        ..
                    })
                ))
                .map(|entry| entry.id())
                .collect::<Vec<_>>(),
            self.persister
                .table_details
                .keys()
                .copied()
                .collect::<Vec<_>>(),
        );

        if let Some(table_writer) = &mut self.persister.table_writer {
            // Close out the timestamp for persisted tables.
            //
            // NB: Keep this method call outside the tokio::spawn. We're
            // guaranteed by persist that writes and seals happen in order,
            // but only if we synchronously wait for the (fast) registration
            // of that work to return.
            let seal_fut = table_writer.seal(&self.persister.all_table_ids, advance_to);
            let _ = task::spawn(
                || format!("advance_local_inputs:{advance_to}"),
                async move {
                    if let Err(err) = seal_fut.await {
                        // TODO: Linearizability relies on this, bubble up the
                        // error instead.
                        //
                        // EDIT: On further consideration, I think it doesn't
                        // affect correctness if this fails, just availability
                        // of the table.
                        tracing::error!(
                            "failed to seal persisted stream to ts {}: {}",
                            advance_to,
                            err
                        );
                    }
                },
            );
        }

        self.dataflow_client
            .advance_all_table_timestamps(advance_to)
            .await;
    }

    async fn message_worker(&mut self, message: DataflowResponse) {
        match message {
            DataflowResponse::Compute(ComputeResponse::PeekResponse(conn_id, response)) => {
                // We expect exactly one peek response, which we forward.
                self.pending_peeks
                    .remove(&conn_id)
                    .expect("no more PeekResponses after closing peek channel")
                    .send(response)
                    .expect("Peek endpoint terminated prematurely");
            }
            DataflowResponse::Compute(ComputeResponse::TailResponse(sink_id, response)) => {
                // We use an `if let` here because the peek could have been canceled already.
                // We can also potentially receive multiple `Complete` responses, followed by
                // a `Dropped` response.
                if let Some(pending_tail) = self.pending_tails.get_mut(&sink_id) {
                    let remove = pending_tail.process_response(response);
                    if remove {
                        self.pending_tails.remove(&sink_id);
                    }
                }
            }
            DataflowResponse::Compute(ComputeResponse::FrontierUppers(updates)) => {
                for (name, changes) in updates {
                    self.update_upper(&name, changes);
                }
                self.maintenance().await;
            }
            DataflowResponse::Storage(StorageResponse::TimestampBindings(
                TimestampBindingFeedback { bindings, changes },
            )) => {
                self.catalog
                    .insert_timestamp_bindings(
                        bindings
                            .into_iter()
                            .map(|(id, pid, ts, offset)| (id, pid.to_string(), ts, offset.offset)),
                    )
                    .expect("inserting timestamp bindings cannot fail");

                let mut durability_updates = Vec::new();
                for (source_id, mut changes) in changes {
                    if let Some(source_state) = self.sources.get_mut(&source_id) {
                        // Apply the updates the dataflow worker sent over, and check if there
                        // were any changes to the source's upper frontier.
                        let changes: Vec<_> =
                            source_state.upper.update_iter(changes.drain()).collect();

                        if !changes.is_empty() {
                            // The source's durability frontier changed as a result of the updates sent over
                            // by the dataflow workers. Advance the durability frontier known to the dataflow worker
                            // to indicate that these bindings have been persisted.
                            durability_updates
                                .push((source_id, source_state.upper.frontier().to_owned()));

                            // Allow compaction to advance.
                            if let Some(compaction_window_ms) = source_state.compaction_window_ms {
                                if !source_state.upper.frontier().is_empty() {
                                    self.since_handles
                                        .get_mut(&source_id)
                                        .unwrap()
                                        .maybe_advance(source_state.upper.frontier().iter().map(
                                            |time| {
                                                compaction_window_ms
                                                    * (time.saturating_sub(compaction_window_ms)
                                                        / compaction_window_ms)
                                            },
                                        ));
                                }
                            }
                        }

                        // Let's also check to see if we can compact any of the bindings we've received.
                        let compaction_ts = *source_state
                            .since
                            .borrow()
                            .frontier()
                            .first()
                            .expect("known to exist");

                        self.catalog
                            .compact_timestamp_bindings(source_id, compaction_ts)
                            .expect("compacting timestamp bindings cannot fail");
                    }
                }

                // Announce the new frontiers that have been durably persisted.
                if !durability_updates.is_empty() {
                    self.dataflow_client
                        .update_durability_frontiers(durability_updates)
                        .await;
                }
            }
        }
    }

    async fn message_statement_ready(
        &mut self,
        StatementReady {
            mut session,
            tx,
            result,
            params,
        }: StatementReady,
    ) {
        match future::ready(result)
            .and_then(|stmt| self.handle_statement(&mut session, stmt, &params))
            .await
        {
            Ok(plan) => self.sequence_plan(tx, session, plan).await,
            Err(e) => tx.send(Err(e), session),
        }
    }

    async fn message_sink_connector_ready(
        &mut self,
        SinkConnectorReady {
            session,
            tx,
            id,
            oid,
            result,
        }: SinkConnectorReady,
    ) {
        match result {
            Ok(connector) => {
                // NOTE: we must not fail from here on out. We have a
                // connector, which means there is external state (like
                // a Kafka topic) that's been created on our behalf. If
                // we fail now, we'll leak that external state.
                if self.catalog.try_get_by_id(id).is_some() {
                    // TODO(benesch): this `expect` here is possibly scary, but
                    // no better solution presents itself. Possibly sinks should
                    // have an error bit, and an error here would set the error
                    // bit on the sink.
                    self.handle_sink_connector_ready(id, oid, connector)
                        .await
                        .expect("sinks should be validated by sequence_create_sink");
                } else {
                    // Another session dropped the sink while we were
                    // creating the connector. Report to the client that
                    // we created the sink, because from their
                    // perspective we did, as there is state (e.g. a
                    // Kafka topic) they need to clean up.
                }
                tx.send(Ok(ExecuteResponse::CreatedSink { existed: false }), session);
            }
            Err(e) => {
                // Drop the placeholder sink if still present.
                if self.catalog.try_get_by_id(id).is_some() {
                    self.catalog_transact(vec![catalog::Op::DropItem(id)], |_builder| Ok(()))
                        .await
                        .expect("deleting placeholder sink cannot fail");
                } else {
                    // Another session may have dropped the placeholder sink while we were
                    // attempting to create the connector, in which case we don't need to do
                    // anything.
                }
                tx.send(Err(e), session);
            }
        }
    }

    fn message_send_diffs(
        &mut self,
        SendDiffs {
            mut session,
            tx,
            id,
            diffs,
            kind,
        }: SendDiffs,
    ) {
        match diffs {
            Ok(diffs) => {
                tx.send(
                    self.sequence_send_diffs(
                        &mut session,
                        SendDiffsPlan {
                            id,
                            updates: diffs,
                            kind,
                        },
                    ),
                    session,
                );
            }
            Err(e) => {
                tx.send(Err(e), session);
            }
        }
    }

    async fn message_scrape_metrics(&mut self) {
        let scraped_metrics = self.metric_scraper.scrape_once();
        self.send_builtin_table_updates_at_offset(scraped_metrics)
            .await;
    }

    async fn message_command(&mut self, cmd: Command) {
        match cmd {
            Command::Startup {
                session,
                cancel_tx,
                tx,
            } => {
                if let Err(e) = self.catalog.create_temporary_schema(session.conn_id()) {
                    let _ = tx.send(Response {
                        result: Err(e.into()),
                        session,
                    });
                    return;
                }

                let catalog = self.catalog.for_session(&session);
                if catalog.resolve_role(session.user()).is_err() {
                    let _ = tx.send(Response {
                        result: Err(CoordError::UnknownLoginRole(session.user().into())),
                        session,
                    });
                    return;
                }

                let mut messages = vec![];
                if catalog
                    .resolve_database(catalog.default_database())
                    .is_err()
                {
                    messages.push(StartupMessage::UnknownSessionDatabase(
                        catalog.default_database().into(),
                    ));
                }

                let secret_key = rand::thread_rng().gen();

                self.active_conns.insert(
                    session.conn_id(),
                    ConnMeta {
                        cancel_tx,
                        secret_key,
                    },
                );

                ClientTransmitter::new(tx).send(
                    Ok(StartupResponse {
                        messages,
                        secret_key,
                    }),
                    session,
                )
            }

            Command::Execute {
                portal_name,
                session,
                tx,
            } => {
                let result = session
                    .get_portal(&portal_name)
                    .ok_or(CoordError::UnknownCursor(portal_name));
                let portal = match result {
                    Ok(portal) => portal,
                    Err(e) => {
                        let _ = tx.send(Response {
                            result: Err(e),
                            session,
                        });
                        return;
                    }
                };
                let stmt = portal.stmt.clone();
                let params = portal.parameters.clone();

                match stmt {
                    Some(stmt) => {
                        // Verify that this statetement type can be executed in the current
                        // transaction state.
                        match session.transaction() {
                            // By this point we should be in a running transaction.
                            &TransactionStatus::Default => unreachable!(),

                            // Started is almost always safe (started means there's a single statement
                            // being executed). Failed transactions have already been checked in pgwire for
                            // a safe statement (COMMIT, ROLLBACK, etc.) and can also proceed.
                            &TransactionStatus::Started(_) | &TransactionStatus::Failed(_) => {
                                if let Statement::Declare(_) = stmt {
                                    // Declare is an exception. Although it's not against any spec to execute
                                    // it, it will always result in nothing happening, since all portals will be
                                    // immediately closed. Users don't know this detail, so this error helps them
                                    // understand what's going wrong. Postgres does this too.
                                    let _ = tx.send(Response {
                                        result: Err(CoordError::OperationRequiresTransaction(
                                            "DECLARE CURSOR".into(),
                                        )),
                                        session,
                                    });
                                    return;
                                }
                            }

                            // Implicit or explicit transactions.
                            //
                            // Implicit transactions happen when a multi-statement query is executed
                            // (a "simple query"). However if a "BEGIN" appears somewhere in there,
                            // then the existing implicit transaction will be upgraded to an explicit
                            // transaction. Thus, we should not separate what implicit and explicit
                            // transactions can do unless there's some additional checking to make sure
                            // something disallowed in explicit transactions did not previously take place
                            // in the implicit portion.
                            &TransactionStatus::InTransactionImplicit(_)
                            | &TransactionStatus::InTransaction(_) => match stmt {
                                // Statements that are safe in a transaction. We still need to verify that we
                                // don't interleave reads and writes since we can't perform those serializably.
                                Statement::Close(_)
                                | Statement::Commit(_)
                                | Statement::Copy(_)
                                | Statement::Deallocate(_)
                                | Statement::Declare(_)
                                | Statement::Discard(_)
                                | Statement::Execute(_)
                                | Statement::Explain(_)
                                | Statement::Fetch(_)
                                | Statement::Prepare(_)
                                | Statement::Rollback(_)
                                | Statement::Select(_)
                                | Statement::SetTransaction(_)
                                | Statement::ShowColumns(_)
                                | Statement::ShowCreateIndex(_)
                                | Statement::ShowCreateSink(_)
                                | Statement::ShowCreateSource(_)
                                | Statement::ShowCreateTable(_)
                                | Statement::ShowCreateView(_)
                                | Statement::ShowDatabases(_)
                                | Statement::ShowIndexes(_)
                                | Statement::ShowObjects(_)
                                | Statement::ShowVariable(_)
                                | Statement::SetVariable(_)
                                | Statement::StartTransaction(_)
                                | Statement::Tail(_) => {
                                    // Always safe.
                                }

                                Statement::Insert(ref insert_statment)
                                    if matches!(
                                        insert_statment.source,
                                        InsertSource::Query(Query {
                                            body: SetExpr::Values(..),
                                            ..
                                        }) | InsertSource::DefaultValues
                                    ) =>
                                {
                                    // Inserting from default? values statements
                                    // is always safe.
                                }

                                // Statements below must by run singly (in Started).
                                Statement::AlterIndex(_)
                                | Statement::AlterObjectRename(_)
                                | Statement::CreateDatabase(_)
                                | Statement::CreateIndex(_)
                                | Statement::CreateRole(_)
                                | Statement::CreateSchema(_)
                                | Statement::CreateSink(_)
                                | Statement::CreateSource(_)
                                | Statement::CreateTable(_)
                                | Statement::CreateType(_)
                                | Statement::CreateView(_)
                                | Statement::CreateViews(_)
                                | Statement::Delete(_)
                                | Statement::DropDatabase(_)
                                | Statement::DropObjects(_)
                                | Statement::Insert(_)
                                | Statement::Update(_) => {
                                    let _ = tx.send(Response {
                                        result: Err(CoordError::OperationProhibitsTransaction(
                                            stmt.to_string(),
                                        )),
                                        session,
                                    });
                                    return;
                                }
                            },
                        }

                        if self.catalog.config().safe_mode {
                            if let Err(e) = check_statement_safety(&stmt) {
                                let _ = tx.send(Response {
                                    result: Err(e),
                                    session,
                                });
                                return;
                            }
                        }

                        let internal_cmd_tx = self.internal_cmd_tx.clone();
                        let catalog = self.catalog.for_session(&session);
                        let purify_fut = sql::pure::purify(&catalog, stmt);
                        let conn_id = session.conn_id();
                        task::spawn(|| format!("purify:{conn_id}"), async move {
                            let result = purify_fut.await.map_err(|e| e.into());
                            internal_cmd_tx
                                .send(Message::StatementReady(StatementReady {
                                    session,
                                    tx: ClientTransmitter::new(tx),
                                    result,
                                    params,
                                }))
                                .expect("sending to internal_cmd_tx cannot fail");
                        });
                    }
                    None => {
                        let _ = tx.send(Response {
                            result: Ok(ExecuteResponse::EmptyQuery),
                            session,
                        });
                    }
                }
            }

            Command::Declare {
                name,
                stmt,
                param_types,
                mut session,
                tx,
            } => {
                let result = self.handle_declare(&mut session, name, stmt, param_types);
                let _ = tx.send(Response { result, session });
            }

            Command::Describe {
                name,
                stmt,
                param_types,
                mut session,
                tx,
            } => {
                let result = self.handle_describe(&mut session, name, stmt, param_types);
                let _ = tx.send(Response { result, session });
            }

            Command::CancelRequest {
                conn_id,
                secret_key,
            } => {
                self.handle_cancel(conn_id, secret_key).await;
            }

            Command::DumpCatalog { session, tx } => {
                // TODO(benesch): when we have RBAC, dumping the catalog should
                // require superuser permissions.

                let _ = tx.send(Response {
                    result: Ok(self.catalog.dump()),
                    session,
                });
            }

            Command::CopyRows {
                id,
                columns,
                rows,
                mut session,
                tx,
            } => {
                let result = self.sequence_copy_rows(&mut session, id, columns, rows);
                let _ = tx.send(Response { result, session });
            }

            Command::Terminate { mut session } => {
                self.handle_terminate(&mut session).await;
            }

            Command::StartTransaction {
                implicit,
                session,
                tx,
            } => {
                let now = self.now_datetime();
                let session = match implicit {
                    None => session.start_transaction(now, None),
                    Some(stmts) => session.start_transaction_implicit(now, stmts),
                };
                let _ = tx.send(Response {
                    result: Ok(()),
                    session,
                });
            }

            Command::Commit {
                action,
                session,
                tx,
            } => {
                let tx = ClientTransmitter::new(tx);
                self.sequence_end_transaction(tx, session, action).await;
            }

            Command::VerifyPreparedStatement {
                name,
                mut session,
                tx,
            } => {
                let result = self.handle_verify_prepared_statement(&mut session, &name);
                let _ = tx.send(Response { result, session });
            }
        }
    }

    /// Validate that all upper frontier updates obey the following invariants:
    ///
    /// 1. The `upper` frontier for each source, index and sink does not go backwards with
    /// upper updates
    /// 2. `upper` never contains any times with negative multiplicity.
    /// 3. `upper` never contains any times with multiplicity greater than `1`.
    /// 4. No updates increase the sum of all multiplicities in `upper`.
    ///
    /// Note that invariants 2 - 4 require single dimensional time, and a fixed number of
    /// dataflow workers. If we migrate to multidimensional time then 2 no longer holds, and
    /// 3. relaxes to "the longest chain in `upper` has to have <= n_workers elements" and
    /// 4. relaxes to "no comparable updates increase the sum of all multiplicities in `upper`".
    /// If we ever switch to dynamically scaling the number of dataflow workers then 3 and 4 no
    /// longer hold.
    fn validate_update_iter(
        upper: &mut MutableAntichain<Timestamp>,
        mut changes: ChangeBatch<Timestamp>,
    ) -> Vec<(Timestamp, i64)> {
        let old_frontier = upper.frontier().to_owned();

        // Validate that no changes correspond to a net addition in the sum of all multiplicities.
        // All updates have to relinquish a time, and optionally, acquire another time.
        // TODO: generalize this to multidimensional times.
        let total_changes = changes
            .iter()
            .map(|(_, change)| *change)
            .fold(0, |acc, x| acc + x);
        assert!(total_changes <= 0);

        let frontier_changes = upper.update_iter(changes.clone().drain()).collect();

        // Make sure no times in `upper` have a negative multiplicity
        for (t, _) in changes.into_inner() {
            let count = upper.count_for(&t);
            assert!(count >= 0);
            assert!(count as usize <= 1);
        }

        assert!(<_ as PartialOrder>::less_equal(
            &old_frontier.borrow(),
            &upper.frontier(),
        ));

        frontier_changes
    }

    /// Updates the upper frontier of a maintained arrangement or sink.
    fn update_upper(&mut self, name: &GlobalId, changes: ChangeBatch<Timestamp>) {
        if let Some(index_state) = self.indexes.get_mut(name) {
            let changes = Self::validate_update_iter(&mut index_state.upper, changes);

            if !changes.is_empty() {
                // Advance the compaction frontier to trail the new frontier.
                // If the compaction latency is `None` compaction messages are
                // not emitted, and the trace should be broadly useable.
                // TODO: If the frontier advances surprisingly quickly, e.g. in
                // the case of a constant collection, this compaction is actively
                // harmful. We should reconsider compaction policy with an eye
                // towards minimizing unexpected screw-ups.
                if let Some(compaction_window_ms) = index_state.compaction_window_ms {
                    // Decline to compact complete collections. This would have the
                    // effect of making the collection unusable. Instead, we would
                    // prefer to compact collections only when we believe it would
                    // reduce the volume of the collection, but we don't have that
                    // information here.
                    if !index_state.upper.frontier().is_empty() {
                        // The since_handle for this GlobalId should have already been registered with
                        // an AntichainToken. Advance it. Changes to the AntichainToken's frontier
                        // will propagate to the Frontiers' since, and changes to that will propate to
                        // self.since_updates.
                        self.since_handles.get_mut(name).unwrap().maybe_advance(
                            index_state.upper.frontier().iter().map(|time| {
                                compaction_window_ms
                                    * (time.saturating_sub(compaction_window_ms)
                                        / compaction_window_ms)
                            }),
                        );
                    }
                }
            }
        } else if self.sources.get_mut(name).is_some() {
            panic!(
                "expected an update for an index or sink, instead got update for source {}",
                name
            );
        } else if let Some(sink_state) = self.sink_writes.get_mut(name) {
            // Only one dataflow worker should give updates for sinks
            let changes = Self::validate_update_iter(&mut sink_state.frontier, changes);

            if !changes.is_empty() {
                sink_state.advance_source_handles();
            }
        }
    }

    /// Forward the subset of since updates that belong to persisted tables'
    /// primary indexes to the persisted tables themselves.
    ///
    /// TODO: In the future the coordinator should perhaps track a table's upper and
    /// since frontiers directly as it currently does for sources.
    fn persisted_table_allow_compaction(
        &mut self,
        since_updates: &[(GlobalId, Antichain<Timestamp>)],
    ) {
        let mut table_since_updates = vec![];

        // Updates for the persistence source that is backing a table.
        let mut table_source_since_updates = vec![];

        for (id, frontier) in since_updates.iter() {
            // HACK: Avoid the "failed to compact persisted tables" error log at
            // restart, by not trying to allow compaction on the minimum
            // timestamp.
            if !frontier
                .elements()
                .iter()
                .any(|x| *x > Timestamp::minimum())
            {
                continue;
            }

            // Not all ids will be present in the catalog however, those that are
            // in the catalog must also have their dependencies in the catalog as
            // well.
            let item = self.catalog.try_get_by_id(*id).map(|e| e.item());
            if let Some(CatalogItem::Index(catalog::Index { on, .. })) = item {
                // We only want to forward since updates to persist if they:
                //  - are from the primary index on a table
                //  - and that table itself is currently persisted
                if let Some(persist) = self.persister.table_details.get(&on) {
                    if self.catalog.default_index_for(*on) == Some(*id) {
                        table_since_updates.push((persist.stream_id, frontier.clone()));
                        table_source_since_updates.push((*on, frontier.clone()));
                    }
                }
            }
        }

        // The persistence source that is backing a table on workers does not send frontier updates
        // back to the coordinator. We update our internal bookkeeping here, because we also
        // forward the compaction frontier here and therefore know that the since advances.
        for (id, frontier) in table_source_since_updates {
            let since_handle = self
                .since_handles
                .get_mut(&id)
                .expect("missing since handle");

            since_handle.maybe_advance(frontier);
        }

        if !table_since_updates.is_empty() {
            let persist_multi = match &mut self.persister.table_writer {
                Some(multi) => multi,
                None => {
                    tracing::error!("internal error: persist_multi_details invariant violated");
                    return;
                }
            };

            let compaction_fut = persist_multi.allow_compaction(&table_since_updates);
            let _ = task::spawn(
                // TODO(guswynn): Add more relevant info here
                || "compaction",
                async move {
                    if let Err(err) = compaction_fut.await {
                        // TODO: Do something smarter here
                        tracing::error!("failed to compact persisted tables: {}", err);
                    }
                },
            );
        }
    }

    /// Perform maintenance work associated with the coordinator.
    ///
    /// Primarily, this involves sequencing compaction commands, which should be
    /// issued whenever available.
    async fn maintenance(&mut self) {
        // Take this opportunity to drain `since_update` commands.
        // Don't try to compact to an empty frontier. There may be a good reason to do this
        // in principle, but not in any current Mz use case.
        // (For background, see: https://github.com/MaterializeInc/materialize/pull/1113#issuecomment-559281990)

        let index_since_updates: Vec<_> = self
            .index_since_updates
            .borrow_mut()
            .drain()
            .filter(|(_, frontier)| frontier != &Antichain::new())
            .collect();

        if !index_since_updates.is_empty() {
            // TEMPORARY HACK: When since_updates got split into
            // index_since_updates and source_since_updates, it looks like the
            // ones persist was looking for ended up here (the wrong one). This
            // means the compaction frontier for persisted tables never
            // advances. Since we run a background test of persist on two system
            // tables, this means unbounded storage (disk) usage, which is very
            // much not correct. Putting this call to persist back in as a
            // workaround while we fix the bug the proper way, which is making
            // sure the messages end up in source_since_updates instead.
            //
            // See #10300 for context.
            self.persisted_table_allow_compaction(&index_since_updates);
            self.dataflow_client
                .allow_index_compaction(DEFAULT_COMPUTE_INSTANCE_ID, index_since_updates)
                .await;
        }

        let source_since_updates: Vec<_> = self
            .source_since_updates
            .borrow_mut()
            .drain()
            .filter(|(_, frontier)| frontier != &Antichain::new())
            .collect();

        if !source_since_updates.is_empty() {
            self.persisted_table_allow_compaction(&source_since_updates);
            self.dataflow_client
                .allow_source_compaction(source_since_updates)
                .await;
        }
    }

    async fn handle_statement(
        &mut self,
        session: &mut Session,
        stmt: sql::ast::Statement<Raw>,
        params: &sql::plan::Params,
    ) -> Result<sql::plan::Plan, CoordError> {
        let pcx = session.pcx();
        let plan = sql::plan::plan(Some(&pcx), &self.catalog.for_session(session), stmt, params)?;
        Ok(plan)
    }

    fn handle_declare(
        &self,
        session: &mut Session,
        name: String,
        stmt: Statement<Raw>,
        param_types: Vec<Option<pgrepr::Type>>,
    ) -> Result<(), CoordError> {
        let desc = describe(&self.catalog, stmt.clone(), &param_types, session)?;
        let params = vec![];
        let result_formats = vec![pgrepr::Format::Text; desc.arity()];
        session.set_portal(name, desc, Some(stmt), params, result_formats)?;
        Ok(())
    }

    fn handle_describe(
        &self,
        session: &mut Session,
        name: String,
        stmt: Option<Statement<Raw>>,
        param_types: Vec<Option<pgrepr::Type>>,
    ) -> Result<(), CoordError> {
        let desc = self.describe(session, stmt.clone(), param_types)?;
        session.set_prepared_statement(
            name,
            PreparedStatement::new(stmt, desc, self.catalog.transient_revision()),
        );
        Ok(())
    }

    fn describe(
        &self,
        session: &Session,
        stmt: Option<Statement<Raw>>,
        param_types: Vec<Option<pgrepr::Type>>,
    ) -> Result<StatementDesc, CoordError> {
        if let Some(stmt) = stmt {
            describe(&self.catalog, stmt, &param_types, session)
        } else {
            Ok(StatementDesc::new(None))
        }
    }

    /// Verify a prepared statement is still valid.
    fn handle_verify_prepared_statement(
        &self,
        session: &mut Session,
        name: &str,
    ) -> Result<(), CoordError> {
        let ps = match session.get_prepared_statement_unverified(&name) {
            Some(ps) => ps,
            None => return Err(CoordError::UnknownPreparedStatement(name.to_string())),
        };
        if ps.catalog_revision != self.catalog.transient_revision() {
            let desc = self.describe(
                session,
                ps.sql().cloned(),
                ps.desc()
                    .param_types
                    .iter()
                    .map(|ty| Some(ty.clone()))
                    .collect(),
            )?;
            if &desc != ps.desc() {
                Err(CoordError::ChangedPlan)
            } else {
                // If the descs are the same, we can bump our version to declare that ps is
                // correct as of now.
                let ps = session
                    .get_prepared_statement_mut_unverified(name)
                    .expect("known to exist");
                ps.catalog_revision = self.catalog.transient_revision();
                Ok(())
            }
        } else {
            Ok(())
        }
    }

    /// Instruct the dataflow layer to cancel any ongoing, interactive work for
    /// the named `conn_id`.
    async fn handle_cancel(&mut self, conn_id: u32, secret_key: u32) {
        if let Some(conn_meta) = self.active_conns.get(&conn_id) {
            // If the secret key specified by the client doesn't match the
            // actual secret key for the target connection, we treat this as a
            // rogue cancellation request and ignore it.
            if conn_meta.secret_key != secret_key {
                return;
            }

            // Cancel deferred writes. There is at most one pending write per session.
            if let Some(idx) = self
                .write_lock_wait_group
                .iter()
                .position(|ready| ready.session.conn_id() == conn_id)
            {
                let ready = self.write_lock_wait_group.remove(idx).unwrap();
                ready.tx.send(Ok(ExecuteResponse::Canceled), ready.session);
            }

            // Inform the target session (if it asks) about the cancellation.
            let _ = conn_meta.cancel_tx.send(Canceled::Canceled);

            // Allow dataflow to cancel any pending peeks.
            self.dataflow_client
                .cancel_peek(DEFAULT_COMPUTE_INSTANCE_ID, conn_id)
                .await;
        }
    }

    /// Handle termination of a client session.
    ///
    /// This cleans up any state in the coordinator associated with the session.
    async fn handle_terminate(&mut self, session: &mut Session) {
        self.clear_transaction(session).await;

        self.drop_temp_items(session.conn_id()).await;
        self.catalog
            .drop_temporary_schema(session.conn_id())
            .expect("unable to drop temporary schema");
        self.active_conns.remove(&session.conn_id());
    }

    /// Handle removing in-progress transaction state regardless of the end action
    /// of the transaction.
    async fn clear_transaction(&mut self, session: &mut Session) -> TransactionStatus {
        let (drop_sinks, txn) = session.clear_transaction();
        self.drop_sinks(drop_sinks).await;

        // Allow compaction of sources from this transaction.
        self.txn_reads.remove(&session.conn_id());

        txn
    }

    /// Removes all temporary items created by the specified connection, though
    /// not the temporary schema itself.
    async fn drop_temp_items(&mut self, conn_id: u32) {
        let ops = self.catalog.drop_temp_item_ops(conn_id);
        self.catalog_transact(ops, |_builder| Ok(()))
            .await
            .expect("unable to drop temporary items for conn_id");
    }

    async fn handle_sink_connector_ready(
        &mut self,
        id: GlobalId,
        oid: u32,
        connector: SinkConnector,
    ) -> Result<(), CoordError> {
        // Update catalog entry with sink connector.
        let entry = self.catalog.get_by_id(&id);
        let name = entry.name().clone();
        let mut sink = match entry.item() {
            CatalogItem::Sink(sink) => sink.clone(),
            _ => unreachable!(),
        };
        sink.connector = catalog::SinkConnectorState::Ready(connector.clone());
        let as_of = SinkAsOf {
            frontier: self.determine_frontier(&[sink.from]),
            strict: !sink.with_snapshot,
        };
        let ops = vec![
            catalog::Op::DropItem(id),
            catalog::Op::CreateItem {
                id,
                oid,
                name: name.clone(),
                item: CatalogItem::Sink(sink.clone()),
            },
        ];
        let df = self
            .catalog_transact(ops, |mut builder| {
                let sink_description = dataflow_types::sinks::SinkDesc {
                    from: sink.from,
                    from_desc: builder
                        .catalog
                        .get_by_id(&sink.from)
                        .desc()
                        .unwrap()
                        .clone(),
                    connector: connector.clone(),
                    envelope: Some(sink.envelope),
                    as_of,
                };
                Ok(builder.build_sink_dataflow(name.to_string(), id, sink_description)?)
            })
            .await?;

        // For some sinks, we need to block compaction of each timestamp binding
        // until all sinks that depend on a given source have finished writing out that timestamp.
        // To achieve that, each sink will hold a AntichainToken for all of the sources it depends
        // on, and will advance all of its source dependencies' compaction frontiers as it completes
        // writes.
        if connector.requires_source_compaction_holdback() {
            let mut tokens = Vec::new();

            // Collect AntichainTokens from all of the sources that have them.
            for id in connector.transitive_source_dependencies() {
                if let Some(token) = self.since_handles.get(&id) {
                    tokens.push(token.clone());
                }
            }

            let sink_writes = SinkWrites::new(tokens);
            self.sink_writes.insert(id, sink_writes);
        }
        Ok(self.ship_dataflow(df).await)
    }

    async fn sequence_plan(
        &mut self,
        tx: ClientTransmitter<ExecuteResponse>,
        mut session: Session,
        plan: Plan,
    ) {
        match plan {
            Plan::CreateDatabase(plan) => {
                tx.send(self.sequence_create_database(plan).await, session);
            }
            Plan::CreateSchema(plan) => {
                tx.send(self.sequence_create_schema(plan).await, session);
            }
            Plan::CreateRole(plan) => {
                tx.send(self.sequence_create_role(plan).await, session);
            }
            Plan::CreateTable(plan) => {
                tx.send(self.sequence_create_table(&session, plan).await, session);
            }
            Plan::CreateSource(plan) => {
                tx.send(
                    self.sequence_create_source(&mut session, plan).await,
                    session,
                );
            }
            Plan::CreateSink(plan) => {
                self.sequence_create_sink(session, plan, tx).await;
            }
            Plan::CreateView(plan) => {
                tx.send(self.sequence_create_view(&session, plan).await, session);
            }
            Plan::CreateViews(plan) => {
                tx.send(
                    self.sequence_create_views(&mut session, plan).await,
                    session,
                );
            }
            Plan::CreateIndex(plan) => {
                tx.send(self.sequence_create_index(plan).await, session);
            }
            Plan::CreateType(plan) => {
                tx.send(self.sequence_create_type(plan).await, session);
            }
            Plan::DropDatabase(plan) => {
                tx.send(self.sequence_drop_database(plan).await, session);
            }
            Plan::DropSchema(plan) => {
                tx.send(self.sequence_drop_schema(plan).await, session);
            }
            Plan::DropRoles(plan) => {
                tx.send(self.sequence_drop_roles(plan).await, session);
            }
            Plan::DropItems(plan) => {
                tx.send(self.sequence_drop_items(plan).await, session);
            }
            Plan::EmptyQuery => {
                tx.send(Ok(ExecuteResponse::EmptyQuery), session);
            }
            Plan::ShowAllVariables => {
                tx.send(self.sequence_show_all_variables(&session), session);
            }
            Plan::ShowVariable(plan) => {
                tx.send(self.sequence_show_variable(&session, plan), session);
            }
            Plan::SetVariable(plan) => {
                tx.send(self.sequence_set_variable(&mut session, plan), session);
            }
            Plan::StartTransaction(plan) => {
                let duplicated =
                    matches!(session.transaction(), TransactionStatus::InTransaction(_));
                let session = session.start_transaction(self.now_datetime(), plan.access);
                tx.send(
                    Ok(ExecuteResponse::StartedTransaction { duplicated }),
                    session,
                )
            }

            Plan::CommitTransaction | Plan::AbortTransaction => {
                let action = match plan {
                    Plan::CommitTransaction => EndTransactionAction::Commit,
                    Plan::AbortTransaction => EndTransactionAction::Rollback,
                    _ => unreachable!(),
                };
                self.sequence_end_transaction(tx, session, action).await;
            }
            Plan::Peek(plan) => {
                tx.send(self.sequence_peek(&mut session, plan).await, session);
            }
            Plan::Tail(plan) => {
                tx.send(self.sequence_tail(&mut session, plan).await, session);
            }
            Plan::SendRows(plan) => {
                tx.send(Ok(send_immediate_rows(plan.rows)), session);
            }

            Plan::CopyFrom(plan) => {
                tx.send(
                    Ok(ExecuteResponse::CopyFrom {
                        id: plan.id,
                        columns: plan.columns,
                        params: plan.params,
                    }),
                    session,
                );
            }
            Plan::Explain(plan) => {
                tx.send(self.sequence_explain(&session, plan), session);
            }
            Plan::SendDiffs(plan) => {
                tx.send(self.sequence_send_diffs(&mut session, plan), session);
            }
            Plan::Insert(plan) => {
                self.sequence_insert(tx, session, plan).await;
            }
            Plan::ReadThenWrite(plan) => {
                self.sequence_read_then_write(tx, session, plan).await;
            }
            Plan::AlterNoop(plan) => {
                tx.send(
                    Ok(ExecuteResponse::AlteredObject(plan.object_type)),
                    session,
                );
            }
            Plan::AlterItemRename(plan) => {
                tx.send(self.sequence_alter_item_rename(plan).await, session);
            }
            Plan::AlterIndexSetOptions(plan) => {
                tx.send(self.sequence_alter_index_set_options(plan), session);
            }
            Plan::AlterIndexResetOptions(plan) => {
                tx.send(self.sequence_alter_index_reset_options(plan), session);
            }
            Plan::AlterIndexEnable(plan) => {
                tx.send(self.sequence_alter_index_enable(plan).await, session);
            }
            Plan::DiscardTemp => {
                self.drop_temp_items(session.conn_id()).await;
                tx.send(Ok(ExecuteResponse::DiscardedTemp), session);
            }
            Plan::DiscardAll => {
                let ret = if let TransactionStatus::Started(_) = session.transaction() {
                    self.drop_temp_items(session.conn_id()).await;
                    let drop_sinks = session.reset();
                    self.drop_sinks(drop_sinks).await;
                    Ok(ExecuteResponse::DiscardedAll)
                } else {
                    Err(CoordError::OperationProhibitsTransaction(
                        "DISCARD ALL".into(),
                    ))
                };
                tx.send(ret, session);
            }
            Plan::Declare(plan) => {
                let param_types = vec![];
                let res = self
                    .handle_declare(&mut session, plan.name, plan.stmt, param_types)
                    .map(|()| ExecuteResponse::DeclaredCursor);
                tx.send(res, session);
            }
            Plan::Fetch(FetchPlan {
                name,
                count,
                timeout,
            }) => {
                tx.send(
                    Ok(ExecuteResponse::Fetch {
                        name,
                        count,
                        timeout,
                    }),
                    session,
                );
            }
            Plan::Close(plan) => {
                if session.remove_portal(&plan.name) {
                    tx.send(Ok(ExecuteResponse::ClosedCursor), session);
                } else {
                    tx.send(Err(CoordError::UnknownCursor(plan.name)), session);
                }
            }
            Plan::Prepare(plan) => {
                if session
                    .get_prepared_statement_unverified(&plan.name)
                    .is_some()
                {
                    tx.send(Err(CoordError::PreparedStatementExists(plan.name)), session);
                } else {
                    session.set_prepared_statement(
                        plan.name,
                        PreparedStatement::new(
                            Some(plan.stmt),
                            plan.desc,
                            self.catalog.transient_revision(),
                        ),
                    );
                    tx.send(Ok(ExecuteResponse::Prepare), session);
                }
            }
            Plan::Execute(plan) => {
                let plan_name = plan.name.clone();
                match self.sequence_execute(&mut session, plan) {
                    Ok(portal_name) => {
                        let internal_cmd_tx = self.internal_cmd_tx.clone();
                        task::spawn(|| format!("execute:{plan_name}"), async move {
                            internal_cmd_tx
                                .send(Message::Command(Command::Execute {
                                    portal_name,
                                    session,
                                    tx: tx.take(),
                                }))
                                .expect("sending to internal_cmd_tx cannot fail");
                        });
                    }
                    Err(err) => tx.send(Err(err), session),
                };
            }
            Plan::Deallocate(plan) => match plan.name {
                Some(name) => {
                    if session.remove_prepared_statement(&name) {
                        tx.send(Ok(ExecuteResponse::Deallocate { all: false }), session);
                    } else {
                        tx.send(Err(CoordError::UnknownPreparedStatement(name)), session);
                    }
                }
                None => {
                    session.remove_all_prepared_statements();
                    tx.send(Ok(ExecuteResponse::Deallocate { all: true }), session);
                }
            },
        }
    }

    // Returns the name of the portal to execute.
    fn sequence_execute(
        &mut self,
        session: &mut Session,
        plan: ExecutePlan,
    ) -> Result<String, CoordError> {
        // Verify the stmt is still valid.
        self.handle_verify_prepared_statement(session, &plan.name)?;
        let ps = session.get_prepared_statement_unverified(&plan.name);
        match ps {
            Some(ps) => {
                let sql = ps.sql().cloned();
                let desc = ps.desc().clone();
                session.create_new_portal(sql, desc, plan.params, Vec::new())
            }
            None => Err(CoordError::UnknownPreparedStatement(plan.name)),
        }
    }

    async fn sequence_create_database(
        &mut self,
        plan: CreateDatabasePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let db_oid = self.catalog.allocate_oid()?;
        let schema_oid = self.catalog.allocate_oid()?;
        let ops = vec![
            catalog::Op::CreateDatabase {
                name: plan.name.clone(),
                oid: db_oid,
            },
            catalog::Op::CreateSchema {
                database_name: DatabaseSpecifier::Name(plan.name),
                schema_name: "public".into(),
                oid: schema_oid,
            },
        ];
        match self.catalog_transact(ops, |_builder| Ok(())).await {
            Ok(_) => Ok(ExecuteResponse::CreatedDatabase { existed: false }),
            Err(CoordError::Catalog(catalog::Error {
                kind: catalog::ErrorKind::DatabaseAlreadyExists(_),
                ..
            })) if plan.if_not_exists => Ok(ExecuteResponse::CreatedDatabase { existed: true }),
            Err(err) => Err(err),
        }
    }

    async fn sequence_create_schema(
        &mut self,
        plan: CreateSchemaPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let oid = self.catalog.allocate_oid()?;
        let op = catalog::Op::CreateSchema {
            database_name: plan.database_name,
            schema_name: plan.schema_name,
            oid,
        };
        match self.catalog_transact(vec![op], |_builder| Ok(())).await {
            Ok(_) => Ok(ExecuteResponse::CreatedSchema { existed: false }),
            Err(CoordError::Catalog(catalog::Error {
                kind: catalog::ErrorKind::SchemaAlreadyExists(_),
                ..
            })) if plan.if_not_exists => Ok(ExecuteResponse::CreatedSchema { existed: true }),
            Err(err) => Err(err),
        }
    }

    async fn sequence_create_role(
        &mut self,
        plan: CreateRolePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let oid = self.catalog.allocate_oid()?;
        let op = catalog::Op::CreateRole {
            name: plan.name,
            oid,
        };
        self.catalog_transact(vec![op], |_builder| Ok(()))
            .await
            .map(|_| ExecuteResponse::CreatedRole)
    }

    async fn sequence_create_table(
        &mut self,
        session: &Session,
        plan: CreateTablePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let CreateTablePlan {
            name,
            table,
            if_not_exists,
        } = plan;

        let conn_id = if table.temporary {
            Some(session.conn_id())
        } else {
            None
        };
        let table_id = self.catalog.allocate_id()?;
        let mut index_depends_on = table.depends_on.clone();
        index_depends_on.push(table_id);
        let table = catalog::Table {
            create_sql: table.create_sql,
            desc: table.desc,
            defaults: table.defaults,
            conn_id,
            depends_on: table.depends_on,
            persist_name: self
                .persister
                .new_table_persist_name(table_id, &name.to_string()),
        };
        let index_id = self.catalog.allocate_id()?;
        let mut index_name = name.clone();
        index_name.item += "_primary_idx";
        index_name = self
            .catalog
            .for_session(session)
            .find_available_name(index_name);
        let index = auto_generate_primary_idx(
            index_name.item.clone(),
            name.clone(),
            table_id,
            &table.desc,
            conn_id,
            index_depends_on,
            self.catalog.index_enabled_by_default(&index_id),
        );
        let table_oid = self.catalog.allocate_oid()?;
        let index_oid = self.catalog.allocate_oid()?;
        let df = self
            .catalog_transact(
                vec![
                    catalog::Op::CreateItem {
                        id: table_id,
                        oid: table_oid,
                        name,
                        item: CatalogItem::Table(table.clone()),
                    },
                    catalog::Op::CreateItem {
                        id: index_id,
                        oid: index_oid,
                        name: index_name,
                        item: CatalogItem::Index(index),
                    },
                ],
                |mut builder| {
                    if let Some((name, description)) =
                        Self::prepare_index_build(builder.catalog, &index_id)
                    {
                        let df = builder.build_index_dataflow(name, index_id, description)?;
                        Ok(Some(df))
                    } else {
                        Ok(None)
                    }
                },
            )
            .await;
        match df {
            Ok(df) => {
                // Determine the initial validity for the table.
                self.persister
                    .add_table(table_id, &table)
                    .map_err(CoordError::Persistence)?;
                let since_ts = self
                    .persister
                    .table_details
                    .get(&table_id)
                    .map(|td| td.since_ts)
                    .unwrap_or(0);

                // Announce the creation of the table source.
                let source_description = self
                    .catalog
                    .state()
                    .source_description_for(table_id)
                    .unwrap();
                self.dataflow_client
                    .create_sources(vec![(
                        table_id,
                        (source_description, Antichain::from_elem(since_ts)),
                    )])
                    .await;
                // Install the dataflow if so required.
                if let Some(df) = df {
                    let frontiers = self.new_source_frontiers(
                        table_id,
                        [since_ts],
                        self.logical_compaction_window_ms,
                    );

                    // NOTE: Tables are not sources, but to a large part of the system they look
                    // like they are, e.g. they are rendered as a SourceConnector::Local.
                    self.sources.insert(table_id, frontiers);

                    self.ship_dataflow(df).await;
                }
                Ok(ExecuteResponse::CreatedTable { existed: false })
            }
            Err(CoordError::Catalog(catalog::Error {
                kind: catalog::ErrorKind::ItemAlreadyExists(_),
                ..
            })) if if_not_exists => Ok(ExecuteResponse::CreatedTable { existed: true }),
            Err(err) => Err(err),
        }
    }

    async fn sequence_create_source(
        &mut self,
        session: &mut Session,
        plan: CreateSourcePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let if_not_exists = plan.if_not_exists;
        let (metadata, ops) = self.generate_create_source_ops(session, vec![plan])?;
        match self
            .catalog_transact(ops, move |mut builder| {
                let mut dfs = Vec::new();
                let mut source_ids = Vec::new();
                for (source_id, idx_id) in metadata {
                    source_ids.push(source_id);
                    if let Some(index_id) = idx_id {
                        if let Some((name, description)) =
                            Self::prepare_index_build(builder.catalog, &index_id)
                        {
                            let df = builder.build_index_dataflow(name, index_id, description)?;
                            dfs.push(df);
                        }
                    }
                }
                Ok((dfs, source_ids))
            })
            .await
        {
            Ok((dfs, source_ids)) => {
                // Do everything to instantiate the source at the coordinator and
                // inform the timestamper and dataflow workers of its existence before
                // shipping any dataflows that depend on its existence.
                let catalog_state = self.catalog.state();

                // Ask persistence if it has a since timestamps for any
                // of the new sources.
                let since_timestamps = source_ids
                    .iter()
                    .map(|id| {
                        let source = catalog_state.get_by_id(&id).source().ok_or_else(|| {
                            CoordError::Internal(format!("ID {} unexpectedly not a source", id))
                        })?;
                        let since_ts = self
                            .persister
                            .load_source_persist_desc(&source)
                            .map_err(CoordError::Persistence)?
                            .map(|p| p.since_ts)
                            .unwrap_or(0);
                        Ok::<_, CoordError>(since_ts)
                    })
                    .collect::<Result<Vec<_>, _>>()?;

                let descriptions = source_ids
                    .iter()
                    .map(|id| catalog_state.source_description_for(*id).unwrap())
                    .collect::<Vec<_>>();

                // Continue to do those things.
                let mut source_descriptions = Vec::with_capacity(source_ids.len());
                for ((source_id, since_ts), description) in source_ids
                    .into_iter()
                    .zip_eq(since_timestamps)
                    .zip_eq(descriptions)
                {
                    self.update_timestamper(source_id, true).await;

                    let frontiers = self.new_source_frontiers(
                        source_id,
                        [since_ts],
                        self.logical_compaction_window_ms,
                    );
                    self.sources.insert(source_id, frontiers);
                    source_descriptions
                        .push((source_id, (description, Antichain::from_elem(since_ts))));
                }

                self.dataflow_client
                    .create_sources(source_descriptions)
                    .await;
                self.ship_dataflows(dfs).await;
                Ok(ExecuteResponse::CreatedSource { existed: false })
            }
            Err(CoordError::Catalog(catalog::Error {
                kind: catalog::ErrorKind::ItemAlreadyExists(_),
                ..
            })) if if_not_exists => Ok(ExecuteResponse::CreatedSource { existed: true }),
            Err(err) => Err(err),
        }
    }

    fn generate_create_source_ops(
        &mut self,
        session: &mut Session,
        plans: Vec<CreateSourcePlan>,
    ) -> Result<(Vec<(GlobalId, Option<GlobalId>)>, Vec<catalog::Op>), CoordError> {
        let mut metadata = vec![];
        let mut ops = vec![];
        for plan in plans {
            let CreateSourcePlan {
                name,
                source,
                materialized,
                ..
            } = plan;
            let source_id = self.catalog.allocate_id()?;
            let source_oid = self.catalog.allocate_oid()?;

            let persist_details = self.persister.new_serialized_source_persist_details(
                source_id,
                &source.connector,
                &name.to_string(),
            );

            let source = catalog::Source {
                create_sql: source.create_sql,
                connector: source.connector,
                persist_details,
                desc: source.desc,
            };
            ops.push(catalog::Op::CreateItem {
                id: source_id,
                oid: source_oid,
                name: name.clone(),
                item: CatalogItem::Source(source.clone()),
            });
            let index_id = if materialized {
                let mut index_name = name.clone();
                index_name.item += "_primary_idx";
                index_name = self
                    .catalog
                    .for_session(session)
                    .find_available_name(index_name);
                let index_id = self.catalog.allocate_id()?;
                let index = auto_generate_primary_idx(
                    index_name.item.clone(),
                    name,
                    source_id,
                    &source.desc,
                    None,
                    vec![source_id],
                    self.catalog.index_enabled_by_default(&index_id),
                );
                let index_oid = self.catalog.allocate_oid()?;
                ops.push(catalog::Op::CreateItem {
                    id: index_id,
                    oid: index_oid,
                    name: index_name,
                    item: CatalogItem::Index(index),
                });
                Some(index_id)
            } else {
                None
            };
            metadata.push((source_id, index_id))
        }
        Ok((metadata, ops))
    }

    async fn sequence_create_sink(
        &mut self,
        session: Session,
        plan: CreateSinkPlan,
        tx: ClientTransmitter<ExecuteResponse>,
    ) {
        let CreateSinkPlan {
            name,
            sink,
            with_snapshot,
            if_not_exists,
        } = plan;

        // First try to allocate an ID and an OID. If either fails, we're done.
        let id = match self.catalog.allocate_id() {
            Ok(id) => id,
            Err(e) => {
                tx.send(Err(e.into()), session);
                return;
            }
        };
        let oid = match self.catalog.allocate_oid() {
            Ok(id) => id,
            Err(e) => {
                tx.send(Err(e.into()), session);
                return;
            }
        };

        // Then try to create a placeholder catalog item with an unknown
        // connector. If that fails, we're done, though if the client specified
        // `if_not_exists` we'll tell the client we succeeded.
        //
        // This placeholder catalog item reserves the name while we create
        // the sink connector, which could take an arbitrarily long time.
        let op = catalog::Op::CreateItem {
            id,
            oid,
            name,
            item: CatalogItem::Sink(catalog::Sink {
                create_sql: sink.create_sql,
                from: sink.from,
                connector: catalog::SinkConnectorState::Pending(sink.connector_builder.clone()),
                envelope: sink.envelope,
                with_snapshot,
                depends_on: sink.depends_on,
            }),
        };

        let transact_result = self
            .catalog_transact(vec![op], |mut builder| -> Result<(), CoordError> {
                // Insert a dummy dataflow to trigger validation before we try to actually create
                // the external sink resources (e.g. Kafka Topics)
                builder
                    .build_sink_dataflow(
                        "dummy".into(),
                        id,
                        dataflow_types::sinks::SinkDesc {
                            from: sink.from,
                            from_desc: builder
                                .catalog
                                .get_by_id(&sink.from)
                                .desc()
                                .unwrap()
                                .clone(),
                            connector: SinkConnector::Tail(TailSinkConnector {}),
                            envelope: Some(sink.envelope),
                            as_of: SinkAsOf {
                                frontier: Antichain::new(),
                                strict: false,
                            },
                        },
                    )
                    .map(|_ok| ())
            })
            .await;
        match transact_result {
            Ok(()) => (),
            Err(CoordError::Catalog(catalog::Error {
                kind: catalog::ErrorKind::ItemAlreadyExists(_),
                ..
            })) if if_not_exists => {
                tx.send(Ok(ExecuteResponse::CreatedSink { existed: true }), session);
                return;
            }
            Err(e) => {
                tx.send(Err(e), session);
                return;
            }
        }

        // Now we're ready to create the sink connector. Arrange to notify the
        // main coordinator thread when the future completes.
        let connector_builder = sink.connector_builder;
        let internal_cmd_tx = self.internal_cmd_tx.clone();
        task::spawn(
            || format!("sink_connector_ready:{}", sink.from),
            async move {
                internal_cmd_tx
                    .send(Message::SinkConnectorReady(SinkConnectorReady {
                        session,
                        tx,
                        id,
                        oid,
                        result: sink_connector::build(connector_builder, id).await,
                    }))
                    .expect("sending to internal_cmd_tx cannot fail");
            },
        );
    }

    fn generate_view_ops(
        &mut self,
        session: &Session,
        name: FullName,
        view: View,
        replace: Option<GlobalId>,
        materialize: bool,
    ) -> Result<(Vec<catalog::Op>, Option<GlobalId>), CoordError> {
        self.validate_timeline(view.expr.global_uses())?;

        let mut ops = vec![];

        if let Some(id) = replace {
            ops.extend(self.catalog.drop_items_ops(&[id]));
        }
        let view_id = self.catalog.allocate_id()?;
        let view_oid = self.catalog.allocate_oid()?;
        // Optimize the expression so that we can form an accurately typed description.
        let optimized_expr = self.prep_relation_expr(view.expr, ExprPrepStyle::Static)?;
        let desc = RelationDesc::new(optimized_expr.typ(), view.column_names);
        let view = catalog::View {
            create_sql: view.create_sql,
            optimized_expr,
            desc,
            conn_id: if view.temporary {
                Some(session.conn_id())
            } else {
                None
            },
            depends_on: view.depends_on,
        };
        ops.push(catalog::Op::CreateItem {
            id: view_id,
            oid: view_oid,
            name: name.clone(),
            item: CatalogItem::View(view.clone()),
        });
        let index_id = if materialize {
            let mut index_name = name.clone();
            index_name.item += "_primary_idx";
            index_name = self
                .catalog
                .for_session(session)
                .find_available_name(index_name);
            let index_id = self.catalog.allocate_id()?;
            let index = auto_generate_primary_idx(
                index_name.item.clone(),
                name,
                view_id,
                &view.desc,
                view.conn_id,
                vec![view_id],
                self.catalog.index_enabled_by_default(&index_id),
            );
            let index_oid = self.catalog.allocate_oid()?;
            ops.push(catalog::Op::CreateItem {
                id: index_id,
                oid: index_oid,
                name: index_name,
                item: CatalogItem::Index(index),
            });
            Some(index_id)
        } else {
            None
        };

        Ok((ops, index_id))
    }

    async fn sequence_create_view(
        &mut self,
        session: &Session,
        plan: CreateViewPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let if_not_exists = plan.if_not_exists;
        let (ops, index_id) = self.generate_view_ops(
            session,
            plan.name,
            plan.view.clone(),
            plan.replace,
            plan.materialize,
        )?;

        match self
            .catalog_transact(ops, |mut builder| {
                if let Some(index_id) = index_id {
                    if let Some((name, description)) =
                        Self::prepare_index_build(&builder.catalog, &index_id)
                    {
                        let df = builder.build_index_dataflow(name, index_id, description)?;
                        return Ok(Some(df));
                    }
                }
                Ok(None)
            })
            .await
        {
            Ok(df) => {
                if let Some(df) = df {
                    self.ship_dataflow(df).await;
                }
                Ok(ExecuteResponse::CreatedView { existed: false })
            }
            Err(CoordError::Catalog(catalog::Error {
                kind: catalog::ErrorKind::ItemAlreadyExists(_),
                ..
            })) if if_not_exists => Ok(ExecuteResponse::CreatedView { existed: true }),
            Err(err) => Err(err),
        }
    }

    async fn sequence_create_views(
        &mut self,
        session: &mut Session,
        plan: CreateViewsPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let mut ops = vec![];
        let mut index_ids = vec![];

        for (name, view) in plan.views {
            let (mut view_ops, index_id) =
                self.generate_view_ops(session, name, view, None, plan.materialize)?;
            ops.append(&mut view_ops);
            if let Some(index_id) = index_id {
                index_ids.push(index_id);
            }
        }

        match self
            .catalog_transact(ops, |mut builder| {
                let mut dfs = vec![];
                for index_id in index_ids {
                    if let Some((name, description)) =
                        Self::prepare_index_build(builder.catalog, &index_id)
                    {
                        let df = builder.build_index_dataflow(name, index_id, description)?;
                        dfs.push(df);
                    }
                }
                Ok(dfs)
            })
            .await
        {
            Ok(dfs) => {
                self.ship_dataflows(dfs).await;
                Ok(ExecuteResponse::CreatedView { existed: false })
            }
            Err(_) if plan.if_not_exists => Ok(ExecuteResponse::CreatedView { existed: true }),
            Err(err) => Err(err),
        }
    }

    async fn sequence_create_index(
        &mut self,
        plan: CreateIndexPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let CreateIndexPlan {
            name,
            mut index,
            options,
            if_not_exists,
        } = plan;

        for key in &mut index.keys {
            Self::prep_scalar_expr(key, ExprPrepStyle::Static)?;
        }
        let id = self.catalog.allocate_id()?;
        let index = catalog::Index {
            create_sql: index.create_sql,
            keys: index.keys,
            on: index.on,
            conn_id: None,
            depends_on: index.depends_on,
            enabled: self.catalog.index_enabled_by_default(&id),
        };
        let oid = self.catalog.allocate_oid()?;
        let op = catalog::Op::CreateItem {
            id,
            oid,
            name,
            item: CatalogItem::Index(index),
        };
        match self
            .catalog_transact(vec![op], |mut builder| {
                if let Some((name, description)) = Self::prepare_index_build(builder.catalog, &id) {
                    let df = builder.build_index_dataflow(name, id, description)?;
                    Ok(Some(df))
                } else {
                    Ok(None)
                }
            })
            .await
        {
            Ok(df) => {
                if let Some(df) = df {
                    self.ship_dataflow(df).await;
                    self.set_index_options(id, options).expect("index enabled");
                }
                Ok(ExecuteResponse::CreatedIndex { existed: false })
            }
            Err(CoordError::Catalog(catalog::Error {
                kind: catalog::ErrorKind::ItemAlreadyExists(_),
                ..
            })) if if_not_exists => Ok(ExecuteResponse::CreatedIndex { existed: true }),
            Err(err) => Err(err),
        }
    }

    async fn sequence_create_type(
        &mut self,
        plan: CreateTypePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let typ = catalog::Type {
            create_sql: plan.typ.create_sql,
            inner: plan.typ.inner.into(),
            depends_on: plan.typ.depends_on,
        };
        let id = self.catalog.allocate_id()?;
        let oid = self.catalog.allocate_oid()?;
        let op = catalog::Op::CreateItem {
            id,
            oid,
            name: plan.name,
            item: CatalogItem::Type(typ),
        };
        match self.catalog_transact(vec![op], |_builder| Ok(())).await {
            Ok(()) => Ok(ExecuteResponse::CreatedType),
            Err(err) => Err(err),
        }
    }

    async fn sequence_drop_database(
        &mut self,
        plan: DropDatabasePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let ops = self.catalog.drop_database_ops(plan.name);
        self.catalog_transact(ops, |_builder| Ok(())).await?;
        Ok(ExecuteResponse::DroppedDatabase)
    }

    async fn sequence_drop_schema(
        &mut self,
        plan: DropSchemaPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let ops = self.catalog.drop_schema_ops(plan.name);
        self.catalog_transact(ops, |_builder| Ok(())).await?;
        Ok(ExecuteResponse::DroppedSchema)
    }

    async fn sequence_drop_roles(
        &mut self,
        plan: DropRolesPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let ops = plan
            .names
            .into_iter()
            .map(|name| catalog::Op::DropRole { name })
            .collect();
        self.catalog_transact(ops, |_builder| Ok(())).await?;
        Ok(ExecuteResponse::DroppedRole)
    }

    async fn sequence_drop_items(
        &mut self,
        plan: DropItemsPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let ops = self.catalog.drop_items_ops(&plan.items);
        self.catalog_transact(ops, |_builder| Ok(())).await?;
        Ok(match plan.ty {
            ObjectType::Schema => unreachable!(),
            ObjectType::Source => ExecuteResponse::DroppedSource,
            ObjectType::View => ExecuteResponse::DroppedView,
            ObjectType::Table => ExecuteResponse::DroppedTable,
            ObjectType::Sink => ExecuteResponse::DroppedSink,
            ObjectType::Index => ExecuteResponse::DroppedIndex,
            ObjectType::Type => ExecuteResponse::DroppedType,
            ObjectType::Role => unreachable!("DROP ROLE not supported"),
            ObjectType::Object => unreachable!("generic OBJECT cannot be dropped"),
        })
    }

    fn sequence_show_all_variables(
        &mut self,
        session: &Session,
    ) -> Result<ExecuteResponse, CoordError> {
        Ok(send_immediate_rows(
            session
                .vars()
                .iter()
                .filter(|v| !v.experimental())
                .map(|v| {
                    Row::pack_slice(&[
                        Datum::String(v.name()),
                        Datum::String(&v.value()),
                        Datum::String(v.description()),
                    ])
                })
                .collect(),
        ))
    }

    fn sequence_show_variable(
        &self,
        session: &Session,
        plan: ShowVariablePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let variable = session.vars().get(&plan.name)?;
        let row = Row::pack_slice(&[Datum::String(&variable.value())]);
        Ok(send_immediate_rows(vec![row]))
    }

    fn sequence_set_variable(
        &self,
        session: &mut Session,
        plan: SetVariablePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        session
            .vars_mut()
            .set(&plan.name, &plan.value, plan.local)?;
        Ok(ExecuteResponse::SetVariable { name: plan.name })
    }

    async fn sequence_end_transaction(
        &mut self,
        tx: ClientTransmitter<ExecuteResponse>,
        mut session: Session,
        mut action: EndTransactionAction,
    ) {
        if EndTransactionAction::Commit == action {
            let txn = session
                .transaction()
                .inner()
                .expect("must be in a transaction");
            if let Transaction {
                ops: TransactionOps::Writes(_),
                ..
            } = txn
            {
                guard_write_critical_section!(self, tx, session, Plan::CommitTransaction);
            }
        }

        // If the transaction has failed, we can only rollback.
        if let (EndTransactionAction::Commit, TransactionStatus::Failed(_)) =
            (&action, session.transaction())
        {
            action = EndTransactionAction::Rollback;
        }
        let response = ExecuteResponse::TransactionExited {
            tag: action.tag(),
            was_implicit: session.transaction().is_implicit(),
        };

        // Immediately do tasks that must be serialized in the coordinator.
        let rx = self
            .sequence_end_transaction_inner(&mut session, action)
            .await;

        // We can now wait for responses or errors and do any session/transaction
        // finalization in a separate task.
        let conn_id = session.conn_id();
        task::spawn(
            || format!("sequence_end_transaction:{conn_id}"),
            async move {
                let result = match rx {
                    // If we have more work to do, do it
                    Ok(fut) => fut.await,
                    Err(e) => Err(e),
                };

                if result.is_err() {
                    action = EndTransactionAction::Rollback;
                }
                session.vars_mut().end_transaction(action);

                match result {
                    Ok(()) => tx.send(Ok(response), session),
                    Err(err) => tx.send(Err(err), session),
                }
            },
        );
    }

    async fn sequence_end_transaction_inner(
        &mut self,
        session: &mut Session,
        action: EndTransactionAction,
    ) -> Result<impl Future<Output = Result<(), CoordError>>, CoordError> {
        let txn = self.clear_transaction(session).await;

        // Although the compaction frontier may have advanced, we do not need to
        // call `maintenance` here because it will soon be called after the next
        // `update_upper`.

        let mut write_fut = None;
        if let EndTransactionAction::Commit = action {
            if let Some(ops) = txn.into_ops() {
                match ops {
                    TransactionOps::Writes(inserts) => {
                        // Although the transaction has a wall_time in its pcx, we use a new
                        // coordinator timestamp here to provide linearizability. The wall_time does
                        // not have to relate to the write time.
                        let timestamp = self.get_local_write_ts();

                        // Separate out which updates were to tables we are
                        // persisting. In practice, we don't enable/disable this
                        // with table-level granularity so it will be all of
                        // them or none of them, which is checked below.
                        let mut persist_updates = Vec::new();
                        let mut volatile_updates = Vec::new();
                        for WriteOp { id, rows } in inserts {
                            // Re-verify this id exists.
                            let _ = self.catalog.try_get_by_id(id).ok_or_else(|| {
                                CoordError::SqlCatalog(CatalogError::UnknownItem(id.to_string()))
                            })?;
                            // This can be empty if, say, a DELETE's WHERE clause had 0 results.
                            if rows.is_empty() {
                                continue;
                            }
                            if let Some(persist) = self.persister.table_details.get(&id) {
                                let updates = rows
                                    .into_iter()
                                    .map(|(row, diff)| ((row, ()), timestamp, diff));
                                persist_updates.push((&persist.write_handle, updates));
                            } else {
                                let updates = rows
                                    .into_iter()
                                    .map(|(row, diff)| Update {
                                        row,
                                        diff,
                                        timestamp,
                                    })
                                    .collect();
                                volatile_updates.push((id, updates));
                            }
                        }

                        // Write all updates, both persistent and volatile.
                        // Persistence takes care of introducing anything it
                        // writes to the dataflow, so we only need a
                        // Command::Insert for the volatile updates.
                        if !persist_updates.is_empty() {
                            if !volatile_updates.is_empty() {
                                coord_bail!("transaction had mixed persistent and volatile writes");
                            }
                            let persist_multi =
                                self.persister.table_writer.as_mut().ok_or_else(|| {
                                    anyhow!(
                                        "internal error: persist_multi_details invariant violated"
                                    )
                                })?;
                            // NB: Keep this method call outside any
                            // tokio::spawns. We're guaranteed by persist that
                            // writes and seals happen in order, but only if we
                            // synchronously wait for the (fast) registration of
                            // that work to return.
                            write_fut = Some(
                                persist_multi
                                    .write_atomic(|builder| {
                                        for (handle, updates) in persist_updates {
                                            builder.add_write(handle, updates)?;
                                        }
                                        Ok(())
                                    })
                                    .map(|res| match res {
                                        Ok(_) => Ok(()),
                                        Err(err) => {
                                            Err(CoordError::Unstructured(anyhow!("{}", err)))
                                        }
                                    }),
                            );
                        } else {
                            for (id, updates) in volatile_updates {
                                self.dataflow_client.table_insert(id, updates).await;
                            }
                        }
                    }
                    _ => {}
                }
            }
        }
        Ok(async move {
            if let Some(fut) = write_fut {
                // Because we return an async block here, this await is not executed until
                // the containing async block is executed, so this await doesn't block the
                // coordinator task.
                fut.await
            } else {
                Ok(())
            }
        })
    }

    /// Return the set of ids in a timedomain and verify timeline correctness.
    ///
    /// When a user starts a transaction, we need to prevent compaction of anything
    /// they might read from. We use a heuristic of "anything in the same database
    /// schemas with the same timeline as whatever the first query is".
    fn timedomain_for(
        &self,
        source_ids: &[GlobalId],
        source_timeline: &Option<Timeline>,
        conn_id: u32,
    ) -> Result<Vec<GlobalId>, CoordError> {
        let mut timedomain_ids = self
            .catalog
            .schema_adjacent_indexed_relations(&source_ids, conn_id);

        // Filter out ids from different timelines. The timeline code only verifies
        // that the SELECT doesn't cross timelines. The schema-adjacent code looks
        // for other ids in the same database schema.
        timedomain_ids.retain(|&id| {
            let id_timeline = self
                .validate_timeline(vec![id])
                .expect("single id should never fail");
            match (&id_timeline, &source_timeline) {
                // If this id doesn't have a timeline, we can keep it.
                (None, _) => true,
                // If there's no source timeline, we have the option to opt into a timeline,
                // so optimistically choose epoch ms. This is useful when the first query in a
                // transaction is on a static view.
                (Some(id_timeline), None) => id_timeline == &Timeline::EpochMilliseconds,
                // Otherwise check if timelines are the same.
                (Some(id_timeline), Some(source_timeline)) => id_timeline == source_timeline,
            }
        });

        Ok(timedomain_ids)
    }

    /// Sequence a peek, determining a timestamp and the most efficient dataflow interaction.
    ///
    /// Peeks are sequenced by assigning a timestamp for evaluation, and then determining and
    /// deploying the most efficient evaluation plan. The peek could evaluate to a constant,
    /// be a simple read out of an existing arrangement, or required a new dataflow to build
    /// the results to return.
    async fn sequence_peek(
        &mut self,
        session: &mut Session,
        plan: PeekPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let PeekPlan {
            source,
            when,
            finishing,
            copy_to,
        } = plan;

        let source_ids = source.global_uses();
        let timeline = self.validate_timeline(source_ids.clone())?;
        let conn_id = session.conn_id();
        let in_transaction = matches!(
            session.transaction(),
            &TransactionStatus::InTransaction(_) | &TransactionStatus::InTransactionImplicit(_)
        );
        // For explicit or implicit transactions that do not use AS OF, get the
        // timestamp of the in-progress transaction or create one. If this is an AS OF
        // query, we don't care about any possible transaction timestamp. If this is a
        // single-statement transaction (TransactionStatus::Started), we don't need to
        // worry about preventing compaction or choosing a valid timestamp for future
        // queries.
        let timestamp = if in_transaction && when == PeekWhen::Immediately {
            // Queries are independent of the logical timestamp iff there are no referenced
            // sources or indexes and there is no reference to `mz_logical_timestamp()`
            // which we check by using a Static prep style.
            let timestamp_independent = source_ids.is_empty()
                && self
                    .prep_relation_expr(source.clone(), ExprPrepStyle::Static)
                    .is_ok();

            // If all previous statements were timestamp-independent and the current one is
            // not, clear the transaction ops so it can get a new timestamp and timedomain.
            if let Some(read_txn) = self.txn_reads.get(&conn_id) {
                if read_txn.timestamp_independent && !timestamp_independent {
                    session.clear_transaction_ops();
                }
            }

            let timestamp = session.get_transaction_timestamp(|| {
                // Determine a timestamp that will be valid for anything in any schema
                // referenced by the first query.
                let mut timedomain_ids = self.timedomain_for(&source_ids, &timeline, conn_id)?;

                // We want to prevent compaction of the indexes consulted by
                // determine_timestamp, not the ones listed in the query.
                let (timestamp, timestamp_ids) =
                    self.determine_timestamp(&timedomain_ids, PeekWhen::Immediately)?;
                // Add the used indexes to the recorded ids.
                timedomain_ids.extend(&timestamp_ids);
                let mut handles = vec![];
                for id in timestamp_ids {
                    handles.push(self.indexes.get(&id).unwrap().since_handle(vec![timestamp]));
                }
                self.txn_reads.insert(
                    conn_id,
                    TxnReads {
                        timestamp_independent,
                        timedomain_ids: timedomain_ids.into_iter().collect(),
                        _handles: handles,
                    },
                );

                Ok(timestamp)
            })?;

            // Verify that the references and indexes for this query are in the current
            // read transaction.
            let mut stmt_ids = HashSet::new();
            stmt_ids.extend(source_ids.iter().collect::<HashSet<_>>());
            // Using nearest_indexes here is a hack until #8318 is fixed. It's used because
            // that's what determine_timestamp uses.
            stmt_ids.extend(
                self.catalog
                    .nearest_indexes(&source_ids)
                    .0
                    .into_iter()
                    .collect::<HashSet<_>>(),
            );
            let read_txn = self.txn_reads.get(&conn_id).unwrap();
            // Find the first reference or index (if any) that is not in the transaction. A
            // reference could be caused by a user specifying an object in a different
            // schema than the first query. An index could be caused by a CREATE INDEX
            // after the transaction started.
            let outside: Vec<_> = stmt_ids.difference(&read_txn.timedomain_ids).collect();
            if !outside.is_empty() {
                let mut names: Vec<_> = read_txn
                    .timedomain_ids
                    .iter()
                    // This could filter out a view that has been replaced in another transaction.
                    .filter_map(|id| self.catalog.try_get_by_id(*id))
                    .map(|item| item.name().to_string())
                    .collect();
                let mut outside: Vec<_> = outside
                    .into_iter()
                    .filter_map(|id| self.catalog.try_get_by_id(*id))
                    .map(|item| item.name().to_string())
                    .collect();
                // Sort so error messages are deterministic.
                names.sort();
                outside.sort();
                return Err(CoordError::RelationOutsideTimeDomain {
                    relations: outside,
                    names,
                });
            }

            timestamp
        } else {
            self.determine_timestamp(&source_ids, when)?.0
        };

        let source = self.prep_relation_expr(
            source,
            ExprPrepStyle::OneShot {
                logical_time: timestamp,
            },
        )?;

        // We create a dataflow and optimize it, to determine if we can avoid building it.
        // This can happen if the result optimizes to a constant, or to a `Get` expression
        // around a maintained arrangement.
        let typ = source.typ();
        let key: Vec<MirScalarExpr> = typ
            .default_key()
            .iter()
            .map(|k| MirScalarExpr::Column(*k))
            .collect();
        let (permutation, thinning) = permutation_for_arrangement(&key, typ.arity());
        // Two transient allocations. We could reclaim these if we don't use them, potentially.
        // TODO: reclaim transient identifiers in fast path cases.
        let view_id = self.allocate_transient_id()?;
        let index_id = self.allocate_transient_id()?;
        // The assembled dataflow contains a view and an index of that view.
        let mut dataflow = DataflowDesc::new(format!("temp-view-{}", view_id));
        dataflow.set_as_of(Antichain::from_elem(timestamp));
        self.dataflow_builder()
            .import_view_into_dataflow(&view_id, &source, &mut dataflow)?;
        dataflow.export_index(
            index_id,
            IndexDesc {
                on_id: view_id,
                key: key.clone(),
            },
            typ,
        );

        // Optimize the dataflow across views, and any other ways that appeal.
        transform::optimize_dataflow(&mut dataflow, self.catalog.enabled_indexes())?;

        // Finalization optimizes the dataflow as much as possible.
        let dataflow_plan = self.finalize_dataflow(dataflow);

        // At this point, `dataflow_plan` contains our best optimized dataflow.
        // We will check the plan to see if there is a fast path to escape full dataflow construction.
        let fast_path = fast_path_peek::create_plan(
            dataflow_plan,
            view_id,
            index_id,
            key,
            permutation,
            thinning.len(),
        )?;

        // Implement the peek, and capture the response.
        let resp = self
            .implement_fast_path_peek(fast_path, timestamp, finishing, conn_id, source.arity())
            .await?;

        match copy_to {
            None => Ok(resp),
            Some(format) => Ok(ExecuteResponse::CopyTo {
                format,
                resp: Box::new(resp),
            }),
        }
    }

    async fn sequence_tail(
        &mut self,
        session: &mut Session,
        plan: TailPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let TailPlan {
            from,
            with_snapshot,
            ts,
            copy_to,
            emit_progress,
        } = plan;
        // TAIL AS OF, similar to peeks, doesn't need to worry about transaction
        // timestamp semantics.
        if ts.is_none() {
            // If this isn't a TAIL AS OF, the TAIL can be in a transaction if it's the
            // only operation.
            session.add_transaction_ops(TransactionOps::Tail)?;
        }

        let make_sink_desc = |coord: &mut Coordinator, from, from_desc, uses| {
            // Determine the frontier of updates to tail *from*.
            // Updates greater or equal to this frontier will be produced.
            let frontier = if let Some(ts) = ts {
                // If a timestamp was explicitly requested, use that.
                let ts = coord.determine_timestamp(uses, PeekWhen::AtTimestamp(ts))?;
                Antichain::from_elem(ts.0)
            } else {
                coord.determine_frontier(uses)
            };

            Ok::<_, CoordError>(SinkDesc {
                from,
                from_desc,
                connector: SinkConnector::Tail(TailSinkConnector::default()),
                envelope: None,
                as_of: SinkAsOf {
                    frontier,
                    strict: !with_snapshot,
                },
            })
        };

        let dataflow = match from {
            TailFrom::Id(from_id) => {
                let from = self.catalog.get_by_id(&from_id);
                let from_desc = from.desc().unwrap().clone();
                let sink_id = self.catalog.allocate_id()?;
                let sink_desc = make_sink_desc(self, from_id, from_desc, &[from_id])?;
                let sink_name = format!("tail-{}", sink_id);
                self.dataflow_builder()
                    .build_sink_dataflow(sink_name, sink_id, sink_desc)?
            }
            TailFrom::Query {
                expr,
                desc,
                depends_on,
            } => {
                let id = self.allocate_transient_id()?;
                let expr = self.prep_relation_expr(expr, ExprPrepStyle::Static)?;
                let desc = RelationDesc::new(expr.typ(), desc.iter_names());
                let sink_desc = make_sink_desc(self, id, desc, &depends_on)?;
                let mut dataflow = DataflowDesc::new(format!("tail-{}", id));
                let mut dataflow_builder = self.dataflow_builder();
                dataflow_builder.import_view_into_dataflow(&id, &expr, &mut dataflow)?;
                dataflow_builder.build_sink_dataflow_into(&mut dataflow, id, sink_desc)?;
                dataflow
            }
        };

        let (sink_id, sink_desc) = &dataflow.sink_exports[0];
        session.add_drop_sink(*sink_id);
        let arity = sink_desc.from_desc.arity();
        let (tx, rx) = mpsc::unbounded_channel();
        self.pending_tails
            .insert(*sink_id, PendingTail::new(tx, emit_progress, arity));
        self.ship_dataflow(dataflow).await;

        let resp = ExecuteResponse::Tailing { rx };
        match copy_to {
            None => Ok(resp),
            Some(format) => Ok(ExecuteResponse::CopyTo {
                format,
                resp: Box::new(resp),
            }),
        }
    }

    /// A policy for determining the timestamp for a peek.
    ///
    /// The Timestamp result may be `None` in the case that the `when` policy
    /// cannot be satisfied, which is possible due to the restricted validity of
    /// traces (each has a `since` and `upper` frontier, and are only valid after
    /// `since` and sure to be available not after `upper`). The set of indexes
    /// used is also returned.
    fn determine_timestamp(
        &mut self,
        uses_ids: &[GlobalId],
        when: PeekWhen,
    ) -> Result<(Timestamp, Vec<GlobalId>), CoordError> {
        // Each involved trace has a validity interval `[since, upper)`.
        // The contents of a trace are only guaranteed to be correct when
        // accumulated at a time greater or equal to `since`, and they
        // are only guaranteed to be currently present for times not
        // greater or equal to `upper`.
        //
        // The plan is to first determine a timestamp, based on the requested
        // timestamp policy, and then determine if it can be satisfied using
        // the compacted arrangements we have at hand. It remains unresolved
        // what to do if it cannot be satisfied (perhaps the query should use
        // a larger timestamp and block, perhaps the user should intervene).
        let (index_ids, unmaterialized_source_ids) = self.catalog.nearest_indexes(uses_ids);

        // Determine the valid lower bound of times that can produce correct outputs.
        // This bound is determined by the arrangements contributing to the query,
        // and does not depend on the transitive sources.
        let mut since = self.indexes.least_valid_since(index_ids.iter().cloned());
        since.join_assign(
            &self
                .sources
                .least_valid_since(unmaterialized_source_ids.iter().cloned()),
        );

        // First determine the candidate timestamp, which is either the explicitly requested
        // timestamp, or the latest timestamp known to be immediately available.
        let timestamp = match when {
            // Explicitly requested timestamps should be respected.
            PeekWhen::AtTimestamp(timestamp) => timestamp,

            // These two strategies vary in terms of which traces drive the
            // timestamp determination process: either the trace itself or the
            // original sources on which they depend.
            PeekWhen::Immediately => {
                if !unmaterialized_source_ids.is_empty() {
                    let mut unmaterialized = vec![];
                    let mut disabled_indexes = vec![];
                    for id in unmaterialized_source_ids {
                        // Determine which sources are unmaterialized and which have disabled indexes
                        let name = self.catalog.get_by_id(&id).name().to_string();
                        let indexes = self.catalog.get_indexes_on(id);
                        if indexes.is_empty() {
                            unmaterialized.push(name);
                        } else {
                            let disabled_index_names = indexes
                                .iter()
                                .filter_map(|id| {
                                    if !self.catalog.is_index_enabled(id) {
                                        Some(self.catalog.get_by_id(&id).name().to_string())
                                    } else {
                                        None
                                    }
                                })
                                .collect();
                            disabled_indexes.push((name, disabled_index_names));
                        }
                    }
                    return Err(CoordError::AutomaticTimestampFailure {
                        unmaterialized,
                        disabled_indexes,
                    });
                }

                let mut candidate = if uses_ids.iter().any(|id| self.catalog.uses_tables(*id)) {
                    // If the view depends on any tables, we enforce
                    // linearizability by choosing the latest input time.
                    self.get_local_read_ts()
                } else {
                    let upper = self.indexes.greatest_open_upper(index_ids.iter().copied());
                    // We peek at the largest element not in advance of `upper`, which
                    // involves a subtraction. If `upper` contains a zero timestamp there
                    // is no "prior" answer, and we do not want to peek at it as it risks
                    // hanging awaiting the response to data that may never arrive.
                    //
                    // The .get(0) here breaks the antichain abstraction by assuming this antichain
                    // has 0 or 1 elements in it. It happens to work because we use a timestamp
                    // type that meets that assumption, but would break if we used a more general
                    // timestamp.
                    if let Some(candidate) = upper.elements().get(0) {
                        if *candidate > 0 {
                            candidate.saturating_sub(1)
                        } else {
                            let unstarted = index_ids
                                .into_iter()
                                .filter(|id| {
                                    self.indexes
                                        .upper_of(id)
                                        .expect("id not found")
                                        .less_equal(&0)
                                })
                                .collect::<Vec<_>>();
                            return Err(CoordError::IncompleteTimestamp(unstarted));
                        }
                    } else {
                        // A complete trace can be read in its final form with this time.
                        //
                        // This should only happen for literals that have no sources
                        Timestamp::max_value()
                    }
                };
                // If the candidate is not beyond the valid `since` frontier,
                // force it to become so as best as we can. If `since` is empty
                // this will be a no-op, as there is no valid time, but that should
                // then be caught below.
                if !since.less_equal(&candidate) {
                    candidate.advance_by(since.borrow());
                }
                candidate
            }
        };

        // If the timestamp is greater or equal to some element in `since` we are
        // assured that the answer will be correct.
        if since.less_equal(&timestamp) {
            Ok((timestamp, index_ids))
        } else {
            let invalid_indexes = index_ids.iter().filter_map(|id| {
                let since = self.indexes.since_of(id).expect("id not found");
                if since.less_equal(&timestamp) {
                    None
                } else {
                    Some(since)
                }
            });
            let invalid_sources = unmaterialized_source_ids.iter().filter_map(|id| {
                let since = self.sources.since_of(id).expect("id not found");
                if since.less_equal(&timestamp) {
                    None
                } else {
                    Some(since)
                }
            });
            let invalid = invalid_indexes.chain(invalid_sources).collect::<Vec<_>>();
            coord_bail!(
                "Timestamp ({}) is not valid for all inputs: {:?}",
                timestamp,
                invalid
            );
        }
    }

    /// Determine the frontier of updates to start *from* for a sink based on
    /// `source_id`.
    ///
    /// Updates greater or equal to this frontier will be produced.
    fn determine_frontier(&mut self, source_ids: &[GlobalId]) -> Antichain<Timestamp> {
        // This function differs from determine_timestamp because sinks/tail don't care
        // about indexes existing or timestamps being complete. If data don't exist
        // yet (upper = 0), it is not a problem for the sink to wait for it. If the
        // timestamp we choose isn't as fresh as possible, that's also fine because we
        // produce timestamps describing when the diff occurred, so users can determine
        // if that's fresh enough.

        // If source_id is already indexed, then nearest_indexes will return the
        // same index that default_index_for does, so we can stick with only using
        // nearest_indexes. We don't care about the indexes being incomplete because
        // callers of this function (CREATE SINK and TAIL) are responsible for creating
        // indexes if needed.
        let (index_ids, unmaterialized_source_ids) = self.catalog.nearest_indexes(source_ids);
        let mut since = self.indexes.least_valid_since(index_ids.iter().copied());
        since.join_assign(
            &self
                .sources
                .least_valid_since(unmaterialized_source_ids.iter().copied()),
        );

        let mut candidate = if index_ids.iter().any(|id| self.catalog.uses_tables(*id)) {
            // If the sink depends on any tables, we enforce linearizability by choosing
            // the latest input time.
            self.get_local_read_ts()
        } else if unmaterialized_source_ids.is_empty() && !index_ids.is_empty() {
            // If the sink does not need to create any indexes and requires at least 1
            // index, use the upper. For something like a static view, the indexes are
            // complete but the index count is 0, and we want 0 instead of max for the
            // time, so we should fall through to the else in that case.
            let upper = self.indexes.greatest_open_upper(index_ids);
            if let Some(ts) = upper.elements().get(0) {
                // We don't need to worry about `ts == 0` like determine_timestamp, because
                // it's fine to not have any timestamps completed yet, which will just cause
                // this sink to wait.
                ts.saturating_sub(1)
            } else {
                Timestamp::max_value()
            }
        } else {
            // If the sink does need to create an index, use 0, which will cause the since
            // to be used below.
            Timestamp::min_value()
        };

        // Ensure that the timestamp is >= since. This is necessary because when a
        // Frontiers is created, its upper = 0, but the since is > 0 until update_upper
        // has run.
        if !since.less_equal(&candidate) {
            candidate.advance_by(since.borrow());
        }
        Antichain::from_elem(candidate)
    }

    fn sequence_explain(
        &mut self,
        session: &Session,
        plan: ExplainPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let ExplainPlan {
            raw_plan,
            row_set_finishing,
            stage,
            options,
        } = plan;
        use std::time::Instant;

        struct Timings {
            decorrelation: Option<Duration>,
            optimization: Option<Duration>,
        }

        let mut timings = Timings {
            decorrelation: None,
            optimization: None,
        };

        let decorrelate = |timings: &mut Timings, raw_plan: HirRelationExpr| -> MirRelationExpr {
            let start = Instant::now();
            let decorrelated_plan = raw_plan.optimize_and_lower(&OptimizerConfig {
                qgm_optimizations: session.vars().qgm_optimizations(),
            });
            timings.decorrelation = Some(start.elapsed());
            decorrelated_plan
        };

        let optimize =
            |timings: &mut Timings,
             coord: &mut Self,
             decorrelated_plan: MirRelationExpr|
             -> Result<DataflowDescription<OptimizedMirRelationExpr>, CoordError> {
                let start = Instant::now();
                let optimized_plan =
                    coord.prep_relation_expr(decorrelated_plan, ExprPrepStyle::Explain)?;
                let mut dataflow = DataflowDesc::new(format!("explanation"));
                coord.dataflow_builder().import_view_into_dataflow(
                    // TODO: If explaining a view, pipe the actual id of the view.
                    &GlobalId::Explain,
                    &optimized_plan,
                    &mut dataflow,
                )?;
                transform::optimize_dataflow(&mut dataflow, coord.catalog.enabled_indexes())?;
                timings.optimization = Some(start.elapsed());
                Ok(dataflow)
            };

        let mut explanation_string = match stage {
            ExplainStage::RawPlan => {
                let catalog = self.catalog.for_session(session);
                let mut explanation = sql::plan::Explanation::new(&raw_plan, &catalog);
                if let Some(row_set_finishing) = row_set_finishing {
                    explanation.explain_row_set_finishing(row_set_finishing);
                }
                if options.typed {
                    explanation.explain_types(&BTreeMap::new());
                }
                explanation.to_string()
            }
            ExplainStage::QueryGraph => {
                // TODO add type information to the output graph
                let model = sql::query_model::Model::from(raw_plan);
                model.as_dot("")?
            }
            ExplainStage::OptimizedQueryGraph => {
                // TODO add type information to the output graph
                let mut model = sql::query_model::Model::from(raw_plan);
                model.optimize();
                model.as_dot("")?
            }
            ExplainStage::DecorrelatedPlan => {
                let decorrelated_plan = OptimizedMirRelationExpr::declare_optimized(decorrelate(
                    &mut timings,
                    raw_plan,
                ));
                let catalog = self.catalog.for_session(session);
                let formatter =
                    dataflow_types::DataflowGraphFormatter::new(&catalog, options.typed);
                let mut explanation =
                    dataflow_types::Explanation::new(&decorrelated_plan, &catalog, &formatter);
                if let Some(row_set_finishing) = row_set_finishing {
                    explanation.explain_row_set_finishing(row_set_finishing);
                }
                explanation.to_string()
            }
            ExplainStage::OptimizedPlan => {
                let decorrelated_plan = decorrelate(&mut timings, raw_plan);
                self.validate_timeline(decorrelated_plan.global_uses())?;
                let dataflow = optimize(&mut timings, self, decorrelated_plan)?;
                let catalog = self.catalog.for_session(session);
                let formatter =
                    dataflow_types::DataflowGraphFormatter::new(&catalog, options.typed);
                let mut explanation =
                    dataflow_types::Explanation::new_from_dataflow(&dataflow, &catalog, &formatter);
                if let Some(row_set_finishing) = row_set_finishing {
                    explanation.explain_row_set_finishing(row_set_finishing);
                }
                explanation.to_string()
            }
            ExplainStage::PhysicalPlan => {
                let decorrelated_plan = decorrelate(&mut timings, raw_plan);
                self.validate_timeline(decorrelated_plan.global_uses())?;
                let dataflow = optimize(&mut timings, self, decorrelated_plan)?;
                let dataflow_plan = dataflow_types::Plan::finalize_dataflow(dataflow)
                    .expect("Dataflow planning failed; unrecoverable error");
                let catalog = self.catalog.for_session(session);
                let mut explanation = dataflow_types::Explanation::new_from_dataflow(
                    &dataflow_plan,
                    &catalog,
                    &dataflow_types::JsonViewFormatter {},
                );
                if let Some(row_set_finishing) = row_set_finishing {
                    explanation.explain_row_set_finishing(row_set_finishing);
                }
                explanation.to_string()
            }
        };
        if options.timing {
            if let Some(decorrelation) = &timings.decorrelation {
                explanation_string.push_str(&format!(
                    "\nDecorrelation time: {}",
                    Interval {
                        months: 0,
                        duration: decorrelation.as_nanos() as i128
                    }
                ));
            }
            if let Some(optimization) = &timings.optimization {
                explanation_string.push_str(&format!(
                    "\nOptimization time: {}",
                    Interval {
                        months: 0,
                        duration: optimization.as_nanos() as i128
                    }
                ));
            }
            if timings.decorrelation.is_some() || timings.optimization.is_some() {
                explanation_string.push_str("\n");
            }
        }
        let rows = vec![Row::pack_slice(&[Datum::from(&*explanation_string)])];
        Ok(send_immediate_rows(rows))
    }

    fn sequence_send_diffs(
        &mut self,
        session: &mut Session,
        mut plan: SendDiffsPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        if self.catalog.config().disable_user_indexes {
            self.catalog.ensure_default_index_enabled(plan.id)?;
        }

        let affected_rows = {
            let mut affected_rows = 0isize;
            let mut all_positive_diffs = true;
            // If all diffs are positive, the number of affected rows is just the
            // sum of all unconsolidated diffs.
            for (_, diff) in plan.updates.iter() {
                if *diff < 0 {
                    all_positive_diffs = false;
                    break;
                }

                affected_rows += diff;
            }

            if !all_positive_diffs {
                // Consolidate rows. This is useful e.g. for an UPDATE where the row
                // doesn't change, and we need to reflect that in the number of
                // affected rows.
                differential_dataflow::consolidation::consolidate(&mut plan.updates);

                affected_rows = 0;
                // With retractions, the number of affected rows is not the number
                // of rows we see, but the sum of the absolute value of their diffs,
                // e.g. if one row is retracted and another is added, the total
                // number of rows affected is 2.
                for (_, diff) in plan.updates.iter() {
                    affected_rows += diff.abs();
                }
            }

            usize::try_from(affected_rows).expect("positive isize must fit")
        };

        session.add_transaction_ops(TransactionOps::Writes(vec![WriteOp {
            id: plan.id,
            rows: plan.updates,
        }]))?;
        Ok(match plan.kind {
            MutationKind::Delete => ExecuteResponse::Deleted(affected_rows),
            MutationKind::Insert => ExecuteResponse::Inserted(affected_rows),
            MutationKind::Update => ExecuteResponse::Updated(affected_rows / 2),
        })
    }

    async fn sequence_insert(
        &mut self,
        tx: ClientTransmitter<ExecuteResponse>,
        mut session: Session,
        plan: InsertPlan,
    ) {
        let optimized_mir = match self.prep_relation_expr(plan.values, ExprPrepStyle::Write) {
            Ok(m) => m,
            Err(e) => {
                tx.send(Err(e), session);
                return;
            }
        };

        match optimized_mir.into_inner() {
            constants @ MirRelationExpr::Constant { .. } => tx.send(
                self.sequence_insert_constant(&mut session, plan.id, constants),
                session,
            ),
            // All non-constant values must be planned as read-then-writes.
            selection => {
                let desc_arity = match self.catalog.try_get_by_id(plan.id) {
                    Some(table) => table.desc().expect("desc called on table").arity(),
                    None => {
                        tx.send(
                            Err(CoordError::SqlCatalog(CatalogError::UnknownItem(
                                plan.id.to_string(),
                            ))),
                            session,
                        );
                        return;
                    }
                };

                let finishing = RowSetFinishing {
                    order_by: vec![],
                    limit: None,
                    offset: 0,
                    project: (0..desc_arity).collect(),
                };

                let read_then_write_plan = ReadThenWritePlan {
                    id: plan.id,
                    selection,
                    finishing,
                    assignments: HashMap::new(),
                    kind: MutationKind::Insert,
                };

                self.sequence_read_then_write(tx, session, read_then_write_plan)
                    .await;
            }
        }
    }

    fn sequence_insert_constant(
        &mut self,
        session: &mut Session,
        id: GlobalId,
        constants: MirRelationExpr,
    ) -> Result<ExecuteResponse, CoordError> {
        // Insert can be queued, so we need to re-verify the id exists.
        let desc = match self.catalog.try_get_by_id(id) {
            Some(table) => table.desc()?,
            None => {
                return Err(CoordError::SqlCatalog(CatalogError::UnknownItem(
                    id.to_string(),
                )))
            }
        };

        match constants {
            MirRelationExpr::Constant { rows, typ: _ } => {
                let rows = rows?;
                for (row, _) in &rows {
                    for (i, datum) in row.iter().enumerate() {
                        desc.constraints_met(i, &datum)?;
                    }
                }
                let diffs_plan = SendDiffsPlan {
                    id,
                    updates: rows,
                    kind: MutationKind::Insert,
                };
                self.sequence_send_diffs(session, diffs_plan)
            }
            o => panic!(
                "tried using sequence_insert_constant on non-constant MirRelationExpr {:?}",
                o
            ),
        }
    }

    fn sequence_copy_rows(
        &mut self,
        session: &mut Session,
        id: GlobalId,
        columns: Vec<usize>,
        rows: Vec<Row>,
    ) -> Result<ExecuteResponse, CoordError> {
        let catalog = self.catalog.for_session(session);
        let values = sql::plan::plan_copy_from(&session.pcx(), &catalog, id, columns, rows)?;

        let constants = self
            .prep_relation_expr(values.lower(), ExprPrepStyle::Write)?
            .into_inner();

        // Copied rows must always be constants.
        self.sequence_insert_constant(session, id, constants)
    }

    // ReadThenWrite is a plan whose writes depend on the results of a
    // read. This works by doing a Peek then queuing a SendDiffs. No writes
    // or read-then-writes can occur between the Peek and SendDiff otherwise a
    // serializability violation could occur.
    async fn sequence_read_then_write(
        &mut self,
        tx: ClientTransmitter<ExecuteResponse>,
        mut session: Session,
        plan: ReadThenWritePlan,
    ) {
        guard_write_critical_section!(self, tx, session, Plan::ReadThenWrite(plan));

        let ReadThenWritePlan {
            id,
            kind,
            selection,
            assignments,
            finishing,
        } = plan;

        // Read then writes can be queued, so re-verify the id exists.
        let desc = match self.catalog.try_get_by_id(id) {
            Some(table) => table.desc().expect("desc called on table").clone(),
            None => {
                tx.send(
                    Err(CoordError::SqlCatalog(CatalogError::UnknownItem(
                        id.to_string(),
                    ))),
                    session,
                );
                return;
            }
        };

        // Ensure selection targets are valid, i.e. user-defined tables, or
        // objects local to the dataflow.
        for id in selection.global_uses() {
            let valid = match self.catalog.try_get_by_id(id) {
                // TODO: Widen this check when supporting temporary tables.
                Some(entry) if id.is_user() => entry.is_table(),
                _ => false,
            };
            if !valid {
                tx.send(Err(CoordError::InvalidTableMutationSelection), session);
                return;
            }
        }

        let ts = self.get_local_read_ts();
        let peek_response = match self
            .sequence_peek(
                &mut session,
                PeekPlan {
                    source: selection,
                    when: PeekWhen::AtTimestamp(ts),
                    finishing,
                    copy_to: None,
                },
            )
            .await
        {
            Ok(resp) => resp,
            Err(e) => {
                tx.send(Err(e), session);
                return;
            }
        };

        let internal_cmd_tx = self.internal_cmd_tx.clone();
        task::spawn(|| format!("sequence_read_then_write:{id}"), async move {
            let arena = RowArena::new();
            let diffs = match peek_response {
                ExecuteResponse::SendingRows(batch) => match batch.await {
                    PeekResponse::Rows(rows) => {
                        |rows: Vec<Row>| -> Result<Vec<(Row, Diff)>, CoordError> {
                            // Use 2x row len incase there's some assignments.
                            let mut diffs = Vec::with_capacity(rows.len() * 2);
                            let mut datum_vec = repr::DatumVec::new();
                            for row in rows {
                                if !assignments.is_empty() {
                                    assert!(
                                        matches!(kind, MutationKind::Update),
                                        "only updates support assignments"
                                    );
                                    let mut datums = datum_vec.borrow_with(&row);
                                    let mut updates = vec![];
                                    for (idx, expr) in &assignments {
                                        let updated = match expr.eval(&datums, &arena) {
                                            Ok(updated) => updated,
                                            Err(e) => {
                                                return Err(CoordError::Unstructured(anyhow!(e)))
                                            }
                                        };
                                        desc.constraints_met(*idx, &updated)?;
                                        updates.push((*idx, updated));
                                    }
                                    for (idx, new_value) in updates {
                                        datums[idx] = new_value;
                                    }
                                    let updated = Row::pack_slice(&datums);
                                    diffs.push((updated, 1));
                                }
                                match kind {
                                    // Updates and deletes always remove the
                                    // current row. Updates will also add an
                                    // updated value.
                                    MutationKind::Update | MutationKind::Delete => {
                                        diffs.push((row, -1))
                                    }
                                    MutationKind::Insert => diffs.push((row, 1)),
                                }
                            }
                            Ok(diffs)
                        }(rows)
                    }
                    PeekResponse::Canceled => {
                        Err(CoordError::Unstructured(anyhow!("execution canceled")))
                    }
                    PeekResponse::Error(e) => Err(CoordError::Unstructured(anyhow!(e))),
                },
                _ => Err(CoordError::Unstructured(anyhow!("expected SendingRows"))),
            };
            internal_cmd_tx
                .send(Message::SendDiffs(SendDiffs {
                    session,
                    tx,
                    id,
                    diffs,
                    kind,
                }))
                .expect("sending to internal_cmd_tx cannot fail");
        });
    }

    async fn sequence_alter_item_rename(
        &mut self,
        plan: AlterItemRenamePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let op = catalog::Op::RenameItem {
            id: plan.id,
            to_name: plan.to_name,
        };
        match self.catalog_transact(vec![op], |_builder| Ok(())).await {
            Ok(()) => Ok(ExecuteResponse::AlteredObject(plan.object_type)),
            Err(err) => Err(err),
        }
    }

    fn sequence_alter_index_set_options(
        &mut self,
        plan: AlterIndexSetOptionsPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        self.set_index_options(plan.id, plan.options)?;
        Ok(ExecuteResponse::AlteredObject(ObjectType::Index))
    }

    fn sequence_alter_index_reset_options(
        &mut self,
        plan: AlterIndexResetOptionsPlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let options = plan
            .options
            .into_iter()
            .map(|o| match o {
                IndexOptionName::LogicalCompactionWindow => IndexOption::LogicalCompactionWindow(
                    self.logical_compaction_window_ms.map(Duration::from_millis),
                ),
            })
            .collect();
        self.set_index_options(plan.id, options)?;
        Ok(ExecuteResponse::AlteredObject(ObjectType::Index))
    }

    async fn sequence_alter_index_enable(
        &mut self,
        plan: AlterIndexEnablePlan,
    ) -> Result<ExecuteResponse, CoordError> {
        let ops = self.catalog.enable_index_ops(plan.id)?;

        // If ops is not empty, index was disabled.
        if !ops.is_empty() {
            let df = self
                .catalog_transact(ops, |mut builder| {
                    let (name, description) = Self::prepare_index_build(builder.catalog, &plan.id)
                        .expect("index enabled");
                    let df = builder.build_index_dataflow(name, plan.id, description)?;
                    Ok(df)
                })
                .await?;
            self.ship_dataflow(df).await;
        }

        Ok(ExecuteResponse::AlteredObject(ObjectType::Index))
    }

    /// Perform a catalog transaction. The closure is passed a [`DataflowBuilder`]
    /// made from the prospective [`CatalogState`] (i.e., the `Catalog` with `ops`
    /// applied but before the transaction is committed). The closure can return
    /// an error to abort the transaction, or otherwise return a value that is
    /// returned by this function. This allows callers to error while building
    /// [`DataflowDesc`]s. [`Coordinator::ship_dataflow`] must be called after this
    /// function successfully returns on any built `DataflowDesc`.
    async fn catalog_transact<F, T>(&mut self, ops: Vec<catalog::Op>, f: F) -> Result<T, CoordError>
    where
        F: FnOnce(DataflowBuilder) -> Result<T, CoordError>,
    {
        let mut sources_to_drop = vec![];
        let mut tables_to_drop = vec![];
        let mut sinks_to_drop = vec![];
        let mut indexes_to_drop = vec![];
        let mut replication_slots_to_drop: HashMap<String, Vec<String>> = HashMap::new();

        for op in &ops {
            if let catalog::Op::DropItem(id) = op {
                match self.catalog.get_by_id(id).item() {
                    CatalogItem::Table(_) => {
                        tables_to_drop.push(*id);
                    }
                    CatalogItem::Source(source) => {
                        sources_to_drop.push(*id);
                        if let SourceConnector::External {
                            connector:
                                ExternalSourceConnector::Postgres(PostgresSourceConnector {
                                    conn,
                                    slot_name,
                                    ..
                                }),
                            ..
                        } = &source.connector
                        {
                            replication_slots_to_drop
                                .entry(conn.clone())
                                .or_insert_with(Vec::new)
                                .push(slot_name.clone());
                        }
                    }
                    CatalogItem::Sink(catalog::Sink {
                        connector: SinkConnectorState::Ready(_),
                        ..
                    }) => {
                        sinks_to_drop.push(*id);
                    }
                    CatalogItem::Index(_) => {
                        indexes_to_drop.push(*id);
                    }
                    _ => (),
                }
            }
        }

        let indexes = &self.indexes;
        let persister = &self.persister;
        let storage = &self.dataflow_client;

        let (builtin_table_updates, result) = self.catalog.transact(ops, |catalog| {
            let builder = DataflowBuilder {
                catalog,
                indexes,
                persister,
                storage,
            };
            f(builder)
        })?;

        // No error returns are allowed after this point. Enforce this at compile time
        // by using this odd structure so we don't accidentally add a stray `?`.
        let _: () = async {
            self.send_builtin_table_updates(builtin_table_updates).await;

            if !sources_to_drop.is_empty() {
                for &id in &sources_to_drop {
                    self.update_timestamper(id, false).await;
                    self.sources.remove(&id);
                }
                self.dataflow_client.drop_sources(sources_to_drop).await;
            }
            if !tables_to_drop.is_empty() {
                // NOTE: When creating a persistent table we insert its compaction frontier (aka since)
                // in `self.sources` to make sure that it is taken into account when rendering
                // dataflows that use it. We must make sure to remove that here.
                for &id in &tables_to_drop {
                    self.sources.remove(&id);
                    self.persister.remove_table(id);
                }
                self.dataflow_client.drop_sources(tables_to_drop).await;
            }
            if !sinks_to_drop.is_empty() {
                for id in sinks_to_drop.iter() {
                    self.sink_writes.remove(id);
                }
                self.dataflow_client
                    .drop_sinks(DEFAULT_COMPUTE_INSTANCE_ID, sinks_to_drop)
                    .await;
            }
            if !indexes_to_drop.is_empty() {
                self.drop_indexes(indexes_to_drop).await;
            }

            // We don't want to block the coordinator on an external postgres server, so
            // move the drop slots to a separate task. This does mean that a failed drop
            // slot won't bubble up to the user as an error message. However, even if it
            // did (and how the code previously worked), mz has already dropped it from our
            // catalog, and so we wouldn't be able to retry anyway.
            if !replication_slots_to_drop.is_empty() {
                // TODO(guswynn): see if there is more relevant info to add to this name
                task::spawn(|| "drop_replication_slots", async move {
                    for (conn, slot_names) in replication_slots_to_drop {
                        // Try to drop the replication slots, but give up after a while.
                        let _ = Retry::default()
                            .retry_async(|_state| {
                                postgres_util::drop_replication_slots(&conn, &slot_names)
                            })
                            .await;
                    }
                });
            }
        }
        .await;

        Ok(result)
    }

    async fn send_builtin_table_updates_at_offset(&mut self, updates: Vec<TimestampedUpdate>) {
        // NB: This makes sure to send all records for the same id in the same
        // message so we can persist a record and its future retraction
        // atomically. Otherwise, we may end up with permanent orphans if a
        // restart/crash happens at the wrong time.
        let timestamp_base = self.get_local_write_ts();
        let mut updates_by_id = HashMap::<GlobalId, Vec<Update>>::new();
        for tu in updates.into_iter() {
            let timestamp = timestamp_base + tu.timestamp_offset;
            for u in tu.updates {
                updates_by_id.entry(u.id).or_default().push(Update {
                    row: u.row,
                    diff: u.diff,
                    timestamp,
                });
            }
        }
        for (id, updates) in updates_by_id {
            // TODO: It'd be nice to unify this with the similar logic in
            // sequence_end_transaction, but it's not initially clear how to do
            // that.
            let persist = self.persister.table_details.get(&id);
            if let Some(persist) = persist {
                let updates: Vec<((Row, ()), Timestamp, Diff)> = updates
                    .into_iter()
                    .map(|u| ((u.row, ()), u.timestamp, u.diff))
                    .collect();
                // Persistence of system table inserts is best effort, so throw
                // away the response and ignore any errors. We do, however,
                // respect the note below so we don't end up with unexpected
                // write and seal reorderings.
                //
                // NB: Keep this method call outside the tokio::spawn. We're
                // guaranteed by persist that writes and seals happen in order,
                // but only if we synchronously wait for the (fast) registration
                // of that work to return.
                let write_fut = persist.write_handle.write(&updates);
                let _ = task::spawn(|| "builtin_table_updates_write_fut:{id}", write_fut);
            } else {
                self.dataflow_client.table_insert(id, updates).await
            }
        }
    }

    async fn send_builtin_table_updates(&mut self, updates: Vec<BuiltinTableUpdate>) {
        let timestamped = TimestampedUpdate {
            updates,
            timestamp_offset: 0,
        };
        self.send_builtin_table_updates_at_offset(vec![timestamped])
            .await
    }

    async fn drop_sinks(&mut self, dataflow_names: Vec<GlobalId>) {
        if !dataflow_names.is_empty() {
            self.dataflow_client
                .drop_sinks(DEFAULT_COMPUTE_INSTANCE_ID, dataflow_names)
                .await;
        }
    }

    async fn drop_indexes(&mut self, indexes: Vec<GlobalId>) {
        let mut trace_keys = Vec::new();
        for id in indexes {
            if self.indexes.remove(&id).is_some() {
                trace_keys.push(id);
            }
        }
        if !trace_keys.is_empty() {
            self.dataflow_client
                .drop_indexes(DEFAULT_COMPUTE_INSTANCE_ID, trace_keys)
                .await
        }
    }

    fn set_index_options(
        &mut self,
        id: GlobalId,
        options: Vec<IndexOption>,
    ) -> Result<(), CoordError> {
        let index = match self.indexes.get_mut(&id) {
            Some(index) => index,
            None => {
                if !self.catalog.is_index_enabled(&id) {
                    return Err(CoordError::InvalidAlterOnDisabledIndex(
                        self.catalog.get_by_id(&id).name().to_string(),
                    ));
                } else {
                    panic!("coord indexes out of sync")
                }
            }
        };

        for o in options {
            match o {
                IndexOption::LogicalCompactionWindow(window) => {
                    let window = window.map(duration_to_timestamp_millis);
                    index.set_compaction_window_ms(window);
                }
            }
        }
        Ok(())
    }

    /// Prepares a relation expression for execution by preparing all contained
    /// scalar expressions (see `prep_scalar_expr`), then optimizing the
    /// relation expression.
    fn prep_relation_expr(
        &mut self,
        mut expr: MirRelationExpr,
        style: ExprPrepStyle,
    ) -> Result<OptimizedMirRelationExpr, CoordError> {
        if let ExprPrepStyle::Static = &style {
            let mut opt_expr = self.view_optimizer.optimize(expr)?;
            opt_expr.0.try_visit_mut_post(&mut |e| {
                // Carefully test filter expressions, which may represent temporal filters.
                if let expr::MirRelationExpr::Filter { input, predicates } = &*e {
                    let mfp = expr::MapFilterProject::new(input.arity())
                        .filter(predicates.iter().cloned());
                    match mfp.into_plan() {
                        Err(e) => coord_bail!("{:?}", e),
                        Ok(_) => Ok(()),
                    }
                } else {
                    e.try_visit_scalars_mut1(&mut |s| Self::prep_scalar_expr(s, style))
                }
            })?;
            Ok(opt_expr)
        } else {
            expr.try_visit_scalars_mut(&mut |s| Self::prep_scalar_expr(s, style))?;

            if let (ExprPrepStyle::Write, expr::MirRelationExpr::Constant { .. }) = (&style, &expr)
            {
                // We don't perform any optimizations on an expression that is already
                // a constant for writes, as we want to maximize bulk-insert throughput.
                Ok(OptimizedMirRelationExpr(expr))
            } else {
                // TODO (wangandi): Is there anything that optimizes to a
                // constant expression that originally contains a global get? Is
                // there anything not containing a global get that cannot be
                // optimized to a constant expression?
                Ok(self.view_optimizer.optimize(expr)?)
            }
        }
    }

    /// Prepares a scalar expression for execution by replacing any placeholders
    /// with their correct values.
    ///
    /// Specifically, calls to the special function `MzLogicalTimestamp` are
    /// replaced if `style` is `OneShot { logical_timestamp }`. Calls are not
    /// replaced for the `Explain` style nor for `Static` which should not
    /// reach this point if we have correctly validated the use of placeholders.
    fn prep_scalar_expr(expr: &mut MirScalarExpr, style: ExprPrepStyle) -> Result<(), CoordError> {
        // Replace calls to `MzLogicalTimestamp` as described above.
        let mut observes_ts = false;
        expr.visit_mut_post(&mut |e| {
            if let MirScalarExpr::CallNullary(f @ NullaryFunc::MzLogicalTimestamp) = e {
                observes_ts = true;
                if let ExprPrepStyle::OneShot { logical_time } = style {
                    let ts = numeric::Numeric::from(logical_time);
                    *e = MirScalarExpr::literal_ok(Datum::from(ts), f.output_type().scalar_type);
                }
            }
        });
        if observes_ts && matches!(style, ExprPrepStyle::Static | ExprPrepStyle::Write) {
            return Err(CoordError::Unsupported(
                "calls to mz_logical_timestamp in in static or write queries",
            ));
        }
        Ok(())
    }

    /// Finalizes a dataflow and then broadcasts it to all workers.
    /// Utility method for the more general [Self::ship_dataflows]
    async fn ship_dataflow(&mut self, dataflow: DataflowDesc) {
        self.ship_dataflows(vec![dataflow]).await
    }

    /// Finalizes a list of dataflows and then broadcasts it to all workers.
    async fn ship_dataflows(&mut self, dataflows: Vec<DataflowDesc>) {
        let mut dataflow_plans = Vec::with_capacity(dataflows.len());
        for dataflow in dataflows.into_iter() {
            dataflow_plans.push(self.finalize_dataflow(dataflow));
        }
        self.dataflow_client
            .create_dataflows(DEFAULT_COMPUTE_INSTANCE_ID, dataflow_plans)
            .await;
    }

    /// Finalizes a dataflow.
    ///
    /// Finalization includes optimization, but also validation of various
    /// invariants such as ensuring that the `as_of` frontier is in advance of
    /// the various `since` frontiers of participating data inputs.
    ///
    /// In particular, there are requirement on the `as_of` field for the dataflow
    /// and the `since` frontiers of created arrangements, as a function of the `since`
    /// frontiers of dataflow inputs (sources and imported arrangements).
    ///
    /// # Panics
    ///
    /// Panics if as_of is < the `since` frontiers.
    ///
    /// Panics if the dataflow descriptions contain an invalid plan.
    fn finalize_dataflow(
        &mut self,
        mut dataflow: DataflowDesc,
    ) -> dataflow_types::DataflowDescription<dataflow_types::Plan> {
        // This function must succeed because catalog_transact has generally been run
        // before calling this function. We don't have plumbing yet to rollback catalog
        // operations if this function fails, and materialized will be in an unsafe
        // state if we do not correctly clean up the catalog.

        // The identity for `join` is the minimum element.
        let mut since = Antichain::from_elem(Timestamp::minimum());

        // Populate "valid from" information for each source.
        for (source_id, _description) in dataflow.source_imports.iter() {
            // Extract `since` information about each source and apply here.
            if let Some(source_since) = self.sources.since_of(source_id) {
                since.join_assign(&source_since);
            }
        }

        // For each imported arrangement, lower bound `since` by its own frontier.
        for (global_id, (_description, _typ)) in dataflow.index_imports.iter() {
            since.join_assign(
                &self
                    .indexes
                    .since_of(global_id)
                    .expect("global id missing at coordinator"),
            );
        }

        // For each produced arrangement, start tracking the arrangement with
        // a compaction frontier of at least `since`.
        for (global_id, _description, _typ) in dataflow.index_exports.iter() {
            let frontiers = self.new_index_frontiers(
                *global_id,
                since.elements().to_vec(),
                self.logical_compaction_window_ms,
            );
            self.indexes.insert(*global_id, frontiers);
        }

        // TODO: Produce "valid from" information for each sink.
        // For each sink, ... do nothing because we don't yield `since` for sinks.
        // for (global_id, _description) in dataflow.sink_exports.iter() {
        //     // TODO: assign `since` to a "valid from" element of the sink. E.g.
        //     self.sink_info[global_id].valid_from(&since);
        // }

        // Ensure that the dataflow's `as_of` is at least `since`.
        if let Some(as_of) = &mut dataflow.as_of {
            // It should not be possible to request an invalid time. SINK doesn't support
            // AS OF. TAIL and Peek check that their AS OF is >= since.
            assert!(
                <_ as PartialOrder>::less_equal(&since, as_of),
                "Dataflow {} requested as_of ({:?}) not >= since ({:?})",
                dataflow.debug_name,
                as_of,
                since
            );
        } else {
            // Bind the since frontier to the dataflow description.
            dataflow.set_as_of(since);
        }

        dataflow_types::Plan::finalize_dataflow(dataflow)
            .expect("Dataflow planning failed; unrecoverable error")
    }

    // Notify the timestamper thread that a source has been created or dropped.
    async fn update_timestamper(&mut self, source_id: GlobalId, create: bool) {
        if create {
            let bindings = self
                .catalog
                .load_timestamp_bindings(source_id)
                .expect("loading timestamps from coordinator cannot fail");
            if let Some(entry) = self.catalog.try_get_by_id(source_id) {
                if let CatalogItem::Source(s) = entry.item() {
                    self.dataflow_client
                        .add_source_timestamping(source_id, s.connector.clone(), bindings)
                        .await;
                }
            }
        } else {
            self.dataflow_client
                .drop_source_timestamping(source_id)
                .await;
        }
    }

    fn allocate_transient_id(&mut self) -> Result<GlobalId, CoordError> {
        let id = self.transient_id_counter;
        if id == u64::max_value() {
            coord_bail!("id counter overflows i64");
        }
        self.transient_id_counter += 1;
        Ok(GlobalId::Transient(id))
    }

    /// Return an error if the ids are from incompatible timelines. This should
    /// be used to prevent users from doing things that are either meaningless
    /// (joining data from timelines that have similar numbers with different
    /// meanings like two separate debezium topics) or will never complete (joining
    /// cdcv2 and realtime data).
    fn validate_timeline(&self, mut ids: Vec<GlobalId>) -> Result<Option<Timeline>, CoordError> {
        let mut timelines: HashMap<GlobalId, Timeline> = HashMap::new();

        // Recurse through IDs to find all sources and tables, adding new ones to
        // the set until we reach the bottom. Static views will end up with an empty
        // timelines.
        while let Some(id) = ids.pop() {
            // Protect against possible infinite recursion. Not sure if it's possible, but
            // a cheap prevention for the future.
            if timelines.contains_key(&id) {
                continue;
            }
            let entry = self.catalog.get_by_id(&id);
            match entry.item() {
                CatalogItem::Source(source) => {
                    timelines.insert(id, source.connector.timeline());
                }
                CatalogItem::Index(index) => {
                    ids.push(index.on);
                }
                CatalogItem::View(view) => {
                    ids.extend(view.optimized_expr.global_uses());
                }
                CatalogItem::Table(table) => {
                    timelines.insert(id, table.timeline());
                }
                _ => {}
            }
        }

        let timelines: HashSet<Timeline> = timelines
            .into_iter()
            .map(|(_, timeline)| timeline)
            .collect();

        // If there's more than one timeline, we will not produce meaningful
        // data to a user. Take, for example, some realtime source and a debezium
        // consistency topic source. The realtime source uses something close to now
        // for its timestamps. The debezium source starts at 1 and increments per
        // transaction. We don't want to choose some timestamp that is valid for both
        // of these because the debezium source will never get to the same value as the
        // realtime source's "milliseconds since Unix epoch" value. And even if it did,
        // it's not meaningful to join just because those two numbers happen to be the
        // same now.
        //
        // Another example: assume two separate debezium consistency topics. Both
        // start counting at 1 and thus have similarish numbers that probably overlap
        // a lot. However it's still not meaningful to join those two at a specific
        // transaction counter number because those counters are unrelated to the
        // other.
        if timelines.len() > 1 {
            return Err(CoordError::Unsupported(
                "multiple timelines within one dataflow",
            ));
        }
        Ok(timelines.into_iter().next())
    }

    /// Attempts to immediately grant `session` access to the write lock or
    /// errors if the lock is currently held.
    fn try_grant_session_write_lock(
        &self,
        session: &mut Session,
    ) -> Result<(), tokio::sync::TryLockError> {
        Arc::clone(&self.write_lock).try_lock_owned().map(|p| {
            session.grant_write_lock(p);
        })
    }

    /// Defers executing `plan` until the write lock becomes available; waiting
    /// occurs in a greenthread, so callers of this function likely want to
    /// return after calling it.
    fn defer_write(
        &mut self,
        tx: ClientTransmitter<ExecuteResponse>,
        session: Session,
        plan: Plan,
    ) {
        let conn_id = session.conn_id();
        let plan = DeferredPlan { tx, session, plan };
        self.write_lock_wait_group.push_back(plan);

        let internal_cmd_tx = self.internal_cmd_tx.clone();
        let write_lock = Arc::clone(&self.write_lock);
        // TODO(guswynn): see if there is more relevant info to add to this name
        task::spawn(|| format!("defer_write:{conn_id}"), async move {
            let guard = write_lock.lock_owned().await;
            internal_cmd_tx
                .send(Message::WriteLockGrant(guard))
                .expect("sending to internal_cmd_tx cannot fail");
        });
    }
}

/// Serves the coordinator based on the provided configuration.
///
/// For a high-level description of the coordinator, see the [crate
/// documentation](crate).
///
/// Returns a handle to the coordinator and a client to communicate with the
/// coordinator.
pub async fn serve(
    Config {
        dataflow_client,
        logging,
        storage,
        timestamp_frequency,
        logical_compaction_window,
        experimental_mode,
        disable_user_indexes,
        safe_mode,
        build_info,
        aws_external_id,
        metrics_registry,
        persister,
        now,
    }: Config,
) -> Result<(Handle, Client), CoordError> {
    let (cmd_tx, cmd_rx) = mpsc::unbounded_channel();
    let (internal_cmd_tx, internal_cmd_rx) = mpsc::unbounded_channel();

    let (catalog, builtin_table_updates) = Catalog::open(catalog::Config {
        storage,
        experimental_mode: Some(experimental_mode),
        safe_mode,
        enable_logging: logging.is_some(),
        build_info,
        aws_external_id,
        timestamp_frequency,
        now: now.clone(),
        skip_migrations: false,
        metrics_registry: &metrics_registry,
        disable_user_indexes,
        persister: &persister,
    })
    .await?;
    let cluster_id = catalog.config().cluster_id;
    let session_id = catalog.config().session_id;
    let start_instant = catalog.config().start_instant;

    let metric_scraper = Scraper::new(logging.as_ref(), metrics_registry.clone())?;

    // In order for the coordinator to support Rc and Refcell types, it cannot be
    // sent across threads. Spawn it in a thread and have this parent thread wait
    // for bootstrap completion before proceeding.
    let (bootstrap_tx, bootstrap_rx) = oneshot::channel();
    let handle = TokioHandle::current();
    let thread = thread::Builder::new()
        .name("coordinator".to_string())
        .spawn(move || {
            let mut coord = Coordinator {
                dataflow_client: dataflow_types::client::Controller::new(dataflow_client),
                view_optimizer: Optimizer::logical_optimizer(),
                catalog,
                persister,
                indexes: ArrangementFrontiers::default(),
                sources: ArrangementFrontiers::default(),
                logical_compaction_window_ms: logical_compaction_window
                    .map(duration_to_timestamp_millis),
                logging_enabled: logging.is_some(),
                internal_cmd_tx,
                metric_scraper,
                last_open_local_ts: 1,
                writes_at_open_ts: false,
                read_writes_at_open_ts: false,
                transient_id_counter: 1,
                active_conns: HashMap::new(),
                txn_reads: HashMap::new(),
                since_handles: HashMap::new(),
                index_since_updates: Rc::new(RefCell::new(HashMap::new())),
                source_since_updates: Rc::new(RefCell::new(HashMap::new())),
                sink_writes: HashMap::new(),
                pending_peeks: HashMap::new(),
                pending_tails: HashMap::new(),
                write_lock: Arc::new(tokio::sync::Mutex::new(())),
                write_lock_wait_group: VecDeque::new(),
            };
            if let Some(config) = &logging {
                handle.block_on(
                    coord.dataflow_client.enable_logging(
                        DEFAULT_COMPUTE_INSTANCE_ID,
                        DataflowLoggingConfig {
                            granularity_ns: config.granularity.as_nanos(),
                            active_logs: BUILTINS
                                .logs()
                                .map(|src| (src.variant.clone(), src.index_id))
                                .collect(),
                            log_logging: config.log_logging,
                        },
                    ),
                );
            }
            let bootstrap = handle.block_on(coord.bootstrap(builtin_table_updates));
            let ok = bootstrap.is_ok();
            bootstrap_tx.send(bootstrap).unwrap();
            if ok {
                handle.block_on(coord.serve(internal_cmd_rx, cmd_rx));
            }
        })
        .unwrap();
    match bootstrap_rx.await.unwrap() {
        Ok(()) => {
            let handle = Handle {
                cluster_id,
                session_id,
                start_instant,
                _thread: thread.join_on_drop(),
            };
            let client = Client::new(cmd_tx);
            Ok((handle, client))
        }
        Err(e) => Err(e),
    }
}

/// The styles in which an expression can be prepared.
#[derive(Clone, Copy, Debug)]
enum ExprPrepStyle {
    /// The expression is being prepared for output as part of an `EXPLAIN`
    /// query.
    Explain,
    /// The expression is being prepared for installation in a static context,
    /// like in a view.
    Static,
    /// The expression is being prepared to run once at the specified logical
    /// time.
    OneShot { logical_time: u64 },
    /// The expression is being prepared to run in an INSERT or other write.
    Write,
}

/// Constructs an [`ExecuteResponse`] that that will send some rows to the
/// client immediately, as opposed to asking the dataflow layer to send along
/// the rows after some computation.
fn send_immediate_rows(rows: Vec<Row>) -> ExecuteResponse {
    ExecuteResponse::SendingRows(Box::pin(async { PeekResponse::Rows(rows) }))
}

fn auto_generate_primary_idx(
    index_name: String,
    on_name: FullName,
    on_id: GlobalId,
    on_desc: &RelationDesc,
    conn_id: Option<u32>,
    depends_on: Vec<GlobalId>,
    enabled: bool,
) -> catalog::Index {
    let default_key = on_desc.typ().default_key();
    catalog::Index {
        create_sql: index_sql(index_name, on_name, &on_desc, &default_key),
        on: on_id,
        keys: default_key
            .iter()
            .map(|k| MirScalarExpr::Column(*k))
            .collect(),
        conn_id,
        depends_on,
        enabled,
    }
}

// TODO(benesch): constructing the canonical CREATE INDEX statement should be
// the responsibility of the SQL package.
pub fn index_sql(
    index_name: String,
    view_name: FullName,
    view_desc: &RelationDesc,
    keys: &[usize],
) -> String {
    use sql::ast::{Expr, Value};

    CreateIndexStatement::<Raw> {
        name: Some(Ident::new(index_name)),
        on_name: sql::normalize::unresolve(view_name),
        key_parts: Some(
            keys.iter()
                .map(|i| match view_desc.get_unambiguous_name(*i) {
                    Some(n) => Expr::Identifier(vec![Ident::new(n.to_string())]),
                    _ => Expr::Value(Value::Number((i + 1).to_string())),
                })
                .collect(),
        ),
        with_options: vec![],
        if_not_exists: false,
    }
    .to_ast_string_stable()
}

/// Converts a Duration to a Timestamp representing the number
/// of milliseconds contained in that Duration
fn duration_to_timestamp_millis(d: Duration) -> Timestamp {
    let millis = d.as_millis();
    if millis > Timestamp::max_value() as u128 {
        Timestamp::max_value()
    } else if millis < Timestamp::min_value() as u128 {
        Timestamp::min_value()
    } else {
        millis as Timestamp
    }
}

/// Creates a description of the statement `stmt`.
///
/// This function is identical to sql::plan::describe except this is also
/// supports describing FETCH statements which need access to bound portals
/// through the session.
pub fn describe(
    catalog: &Catalog,
    stmt: Statement<Raw>,
    param_types: &[Option<pgrepr::Type>],
    session: &Session,
) -> Result<StatementDesc, CoordError> {
    match stmt {
        // FETCH's description depends on the current session, which describe_statement
        // doesn't (and shouldn't?) have access to, so intercept it here.
        Statement::Fetch(FetchStatement { ref name, .. }) => {
            match session.get_portal(name.as_str()).map(|p| p.desc.clone()) {
                Some(desc) => Ok(desc),
                None => Err(CoordError::UnknownCursor(name.to_string())),
            }
        }
        _ => {
            let catalog = &catalog.for_session(session);
            Ok(sql::plan::describe(
                &session.pcx(),
                catalog,
                stmt,
                param_types,
            )?)
        }
    }
}

fn check_statement_safety(stmt: &Statement<Raw>) -> Result<(), CoordError> {
    let (source_or_sink, typ, with_options) = match stmt {
        Statement::CreateSource(CreateSourceStatement {
            connector,
            with_options,
            ..
        }) => ("source", ConnectorType::from(connector), with_options),
        Statement::CreateSink(CreateSinkStatement {
            connector,
            with_options,
            ..
        }) => ("sink", ConnectorType::from(connector), with_options),
        _ => return Ok(()),
    };
    match typ {
        // File sources and sinks are prohibited in safe mode because they allow
        // reading ƒrom and writing to arbitrary files on disk.
        ConnectorType::File => {
            return Err(CoordError::SafeModeViolation(format!(
                "file {}",
                source_or_sink
            )));
        }
        ConnectorType::AvroOcf => {
            return Err(CoordError::SafeModeViolation(format!(
                "Avro OCF {}",
                source_or_sink
            )));
        }
        // Kerberos-authenticated Kafka sources and sinks are prohibited in
        // safe mode because librdkafka will blindly execute the string passed
        // as `sasl_kerberos_kinit_cmd`.
        ConnectorType::Kafka => {
            // It's too bad that we have to reinvent so much of librdkafka's
            // option parsing and hardcode some of its defaults here. But there
            // isn't an obvious alternative; asking librdkafka about its =
            // defaults requires constructing a librdkafka client, and at that
            // point it's already too late.
            let mut with_options = sql::normalize::options(with_options);
            let with_options = sql::kafka_util::extract_config(&mut with_options)?;
            let security_protocol = with_options
                .get("security.protocol")
                .map(|v| v.as_str())
                .unwrap_or("plaintext");
            let sasl_mechanism = with_options
                .get("sasl.mechanisms")
                .map(|v| v.as_str())
                .unwrap_or("GSSAPI");
            if (security_protocol.eq_ignore_ascii_case("sasl_plaintext")
                || security_protocol.eq_ignore_ascii_case("sasl_ssl"))
                && sasl_mechanism.eq_ignore_ascii_case("GSSAPI")
            {
                return Err(CoordError::SafeModeViolation(format!(
                    "Kerberos-authenticated Kafka {}",
                    source_or_sink,
                )));
            }
        }
        _ => (),
    }
    Ok(())
}

/// Logic and types for fast-path determination for dataflow execution.
///
/// This module determines if a dataflow can be short-cut, by returning constant values
/// or by reading out of existing arrangements, and implements the appropriate plan.
pub mod fast_path_peek {

    use dataflow_types::client::{ComputeClient, DEFAULT_COMPUTE_INSTANCE_ID};
    use std::collections::HashMap;

    use crate::CoordError;
    use expr::{EvalError, GlobalId, Id, MirScalarExpr};
    use repr::{Diff, Row};

    #[derive(Debug)]
    pub struct PeekDataflowPlan {
        desc: dataflow_types::DataflowDescription<dataflow_types::Plan>,
        id: GlobalId,
        key: Vec<MirScalarExpr>,
        permutation: HashMap<usize, usize>,
        thinned_arity: usize,
    }

    /// Possible ways in which the coordinator could produce the result for a goal view.
    #[derive(Debug)]
    pub enum Plan {
        /// The view evaluates to a constant result that can be returned.
        Constant(Result<Vec<(Row, repr::Timestamp, Diff)>, EvalError>),
        /// The view can be read out of an existing arrangement.
        PeekExisting(GlobalId, Option<Row>, expr::SafeMfpPlan),
        /// The view must be installed as a dataflow and then read.
        PeekDataflow(PeekDataflowPlan),
    }

    /// Determine if the dataflow plan can be implemented without an actual dataflow.
    ///
    /// If the optimized plan is a `Constant` or a `Get` of a maintained arrangement,
    /// we can avoid building a dataflow (and either just return the results, or peek
    /// out of the arrangement, respectively).
    pub fn create_plan(
        dataflow_plan: dataflow_types::DataflowDescription<dataflow_types::Plan>,
        view_id: GlobalId,
        index_id: GlobalId,
        index_key: Vec<MirScalarExpr>,
        index_permutation: HashMap<usize, usize>,
        index_thinned_arity: usize,
    ) -> Result<Plan, CoordError> {
        // At this point, `dataflow_plan` contains our best optimized dataflow.
        // We will check the plan to see if there is a fast path to escape full dataflow construction.

        // We need to restrict ourselves to settings where the inserted transient view is the first thing
        // to build (no dependent views). There is likely an index to build as well, but we may not be sure.
        if dataflow_plan.objects_to_build.len() >= 1
            && dataflow_plan.objects_to_build[0].id == view_id
        {
            match &dataflow_plan.objects_to_build[0].view {
                // In the case of a constant, we can return the result now.
                dataflow_types::Plan::Constant { rows } => {
                    return Ok(Plan::Constant(rows.clone()));
                }
                // In the case of a bare `Get`, we may be able to directly index an arrangement.
                dataflow_types::Plan::Get {
                    id,
                    keys: _,
                    mfp,
                    key_val,
                } => {
                    // Convert `mfp` to an executable, non-temporal plan.
                    // It should be non-temporal, as OneShot preparation populates `mz_logical_timestamp`.
                    let map_filter_project = mfp
                        .clone()
                        .into_plan()
                        .map_err(|e| crate::error::CoordError::Unstructured(::anyhow::anyhow!(e)))?
                        .into_nontemporal()
                        .map_err(|_e| {
                            crate::error::CoordError::Unstructured(::anyhow::anyhow!(
                                "OneShot plan has temporal constraints"
                            ))
                        })?;
                    // We should only get excited if we can track down an index for `id`.
                    // If `keys` is non-empty, that means we think one exists.
                    for (index_id, (desc, _typ)) in dataflow_plan.index_imports.iter() {
                        if let Some((key, val)) = key_val {
                            if Id::Global(desc.on_id) == *id && &desc.key == key {
                                // Indicate an early exit with a specific index and key_val.
                                return Ok(Plan::PeekExisting(
                                    *index_id,
                                    val.clone(),
                                    map_filter_project,
                                ));
                            }
                        } else if Id::Global(desc.on_id) == *id {
                            // Indicate an early exit with a specific index and no key_val.
                            return Ok(Plan::PeekExisting(*index_id, None, map_filter_project));
                        }
                    }
                }
                // nothing can be done for non-trivial expressions.
                _ => {}
            }
        }
        return Ok(Plan::PeekDataflow(PeekDataflowPlan {
            desc: dataflow_plan,
            id: index_id,
            key: index_key,
            permutation: index_permutation,
            thinned_arity: index_thinned_arity,
        }));
    }

    impl crate::coord::Coordinator {
        /// Implements a peek plan produced by `create_plan` above.
        pub async fn implement_fast_path_peek(
            &mut self,
            fast_path: Plan,
            timestamp: repr::Timestamp,
            finishing: expr::RowSetFinishing,
            conn_id: u32,
            source_arity: usize,
        ) -> Result<crate::ExecuteResponse, CoordError> {
            // If the dataflow optimizes to a constant expression, we can immediately return the result.
            if let Plan::Constant(rows) = fast_path {
                let mut rows = match rows {
                    Ok(rows) => rows,
                    Err(e) => return Err(e.into()),
                };
                // retain exactly those updates less or equal to `timestamp`.
                for (_, time, diff) in rows.iter_mut() {
                    use timely::PartialOrder;
                    if time.less_equal(&timestamp) {
                        // clobber the timestamp, so consolidation occurs.
                        *time = timestamp.clone();
                    } else {
                        // zero the difference, to prevent a contribution.
                        *diff = 0;
                    }
                }
                // Consolidate down the results to get correct totals.
                differential_dataflow::consolidation::consolidate_updates(&mut rows);

                let mut results = Vec::new();
                for (ref row, _time, count) in rows {
                    if count < 0 {
                        Err(EvalError::InvalidParameterValue(format!(
                            "Negative multiplicity in constant result: {}",
                            count
                        )))?
                    };
                    for _ in 0..count {
                        // TODO: If `count` is too large, or `results` too full, we could error.
                        results.push(row.clone());
                    }
                }
                finishing.finish(&mut results);
                return Ok(crate::coord::send_immediate_rows(results));
            }

            // The remaining cases are a peek into a maintained arrangement, or building a dataflow.
            // In both cases we will want to peek, and the main difference is that we might want to
            // build a dataflow and drop it once the peek is issued. The peeks are also constructed
            // differently.

            // If we must build the view, ship the dataflow.
            let (peek_command, drop_dataflow) = match fast_path {
                Plan::PeekExisting(id, key, map_filter_project) => (
                    (
                        id,
                        key,
                        conn_id,
                        timestamp,
                        finishing.clone(),
                        map_filter_project,
                    ),
                    None,
                ),
                Plan::PeekDataflow(PeekDataflowPlan {
                    desc: dataflow,
                    id: index_id,
                    key: index_key,
                    permutation: index_permutation,
                    thinned_arity: index_thinned_arity,
                }) => {
                    // Very important: actually create the dataflow (here, so we can destructure).
                    self.dataflow_client
                        .create_dataflows(DEFAULT_COMPUTE_INSTANCE_ID, vec![dataflow])
                        .await;
                    // Create an identity MFP operator.
                    let mut map_filter_project = expr::MapFilterProject::new(source_arity);
                    map_filter_project
                        .permute(index_permutation, index_key.len() + index_thinned_arity);
                    let map_filter_project = map_filter_project
                        .into_plan()
                        .map_err(|e| crate::error::CoordError::Unstructured(::anyhow::anyhow!(e)))?
                        .into_nontemporal()
                        .map_err(|_e| {
                            crate::error::CoordError::Unstructured(::anyhow::anyhow!(
                                "OneShot plan has temporal constraints"
                            ))
                        })?;
                    (
                        (
                            index_id, // transient identifier produced by `dataflow_plan`.
                            None,
                            conn_id,
                            timestamp,
                            finishing.clone(),
                            map_filter_project,
                        ),
                        Some(index_id),
                    )
                }
                _ => {
                    unreachable!()
                }
            };

            // Endpoints for sending and receiving peek responses.
            let (rows_tx, rows_rx) = tokio::sync::mpsc::unbounded_channel();

            // The peek is ready to go for both cases, fast and non-fast.
            // Stash the response mechanism, and broadcast dataflow construction.
            self.pending_peeks.insert(conn_id, rows_tx);
            let (id, key, conn_id, timestamp, _finishing, map_filter_project) = peek_command;
            self.dataflow_client
                .peek(
                    DEFAULT_COMPUTE_INSTANCE_ID,
                    id,
                    key,
                    conn_id,
                    timestamp,
                    finishing.clone(),
                    map_filter_project,
                )
                .await;

            use dataflow_types::PeekResponse;
            use futures::FutureExt;
            use futures::StreamExt;

            // Prepare the receiver to return as a response.
            let rows_rx = tokio_stream::wrappers::UnboundedReceiverStream::new(rows_rx)
                .fold(PeekResponse::Rows(vec![]), |memo, resp| async {
                    match (memo, resp) {
                        (PeekResponse::Rows(mut memo), PeekResponse::Rows(rows)) => {
                            memo.extend(rows);
                            PeekResponse::Rows(memo)
                        }
                        (PeekResponse::Error(e), _) | (_, PeekResponse::Error(e)) => {
                            PeekResponse::Error(e)
                        }
                        (PeekResponse::Canceled, _) | (_, PeekResponse::Canceled) => {
                            PeekResponse::Canceled
                        }
                    }
                })
                .map(move |mut resp| {
                    if let PeekResponse::Rows(rows) = &mut resp {
                        finishing.finish(rows)
                    }
                    resp
                });

            // If it was created, drop the dataflow once the peek command is sent.
            if let Some(index_id) = drop_dataflow {
                self.drop_indexes(vec![index_id]).await;
            }

            Ok(crate::ExecuteResponse::SendingRows(Box::pin(rows_rx)))
        }
    }
}

#[cfg(test)]
impl Coordinator {
    #[allow(dead_code)]
    async fn verify_ship_dataflow_no_error(&mut self) {
        // ship_dataflow, ship_dataflows, and finalize_dataflow are not allowed
        // to have a `Result` return because these functions are called after
        // `catalog_transact`, after which no errors are allowed. This test exists to
        // prevent us from incorrectly teaching those functions how to return errors
        // (which has happened twice and is the motivation for this test).

        let df = DataflowDesc::new("".into());
        let _: () = self.ship_dataflow(df.clone()).await;
        let _: () = self.ship_dataflows(vec![df.clone()]).await;
        let _: DataflowDescription<dataflow_types::plan::Plan> = self.finalize_dataflow(df);
    }
}