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
// 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.

//! Translation of SQL commands into timestamped `Controller` commands.
//!
//! The various SQL commands instruct the system to take actions that are not
//! yet explicitly timestamped. On the other hand, the underlying data continually
//! change as time moves forward. On the third hand, we greatly benefit from the
//! information that some times are no longer of interest, so that we may
//! compact the representation of the continually changing collections.
//!
//! The [`Coordinator`] curates these interactions by observing the progress
//! collections make through time, choosing timestamps for its own commands,
//! and eventually communicating that certain times have irretrievably "passed".
//!
//! ## 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::borrow::Cow;
use std::collections::{BTreeMap, BTreeSet, VecDeque};
use std::fmt;
use std::net::IpAddr;
use std::num::NonZeroI64;
use std::ops::Neg;
use std::str::FromStr;
use std::sync::LazyLock;
use std::sync::{Arc, Mutex};
use std::thread;
use std::time::{Duration, Instant};

use anyhow::Context;
use chrono::{DateTime, Utc};
use derivative::Derivative;
use differential_dataflow::lattice::Lattice;
use fail::fail_point;
use futures::future::{BoxFuture, FutureExt, LocalBoxFuture};
use futures::StreamExt;
use http::Uri;
use ipnet::IpNet;
use itertools::{Either, Itertools};
use mz_adapter_types::compaction::CompactionWindow;
use mz_adapter_types::connection::ConnectionId;
use mz_adapter_types::dyncfgs::WITH_0DT_DEPLOYMENT_CAUGHT_UP_CHECK_INTERVAL;
use mz_build_info::BuildInfo;
use mz_catalog::builtin::{BUILTINS, BUILTINS_STATIC, MZ_STORAGE_USAGE_BY_SHARD};
use mz_catalog::config::{AwsPrincipalContext, BuiltinItemMigrationConfig, ClusterReplicaSizeMap};
use mz_catalog::durable::OpenableDurableCatalogState;
use mz_catalog::expr_cache::{GlobalExpressions, LocalExpressions};
use mz_catalog::memory::objects::{
    CatalogEntry, CatalogItem, ClusterReplicaProcessStatus, ClusterVariantManaged, Connection,
    DataSourceDesc, Table, TableDataSource,
};
use mz_cloud_resources::{CloudResourceController, VpcEndpointConfig, VpcEndpointEvent};
use mz_compute_client::as_of_selection;
use mz_compute_client::controller::error::InstanceMissing;
use mz_compute_types::dataflows::DataflowDescription;
use mz_compute_types::plan::Plan;
use mz_compute_types::ComputeInstanceId;
use mz_controller::clusters::{ClusterConfig, ClusterEvent, ClusterStatus, ProcessId};
use mz_controller::ControllerConfig;
use mz_controller_types::{ClusterId, ReplicaId, WatchSetId};
use mz_expr::{MapFilterProject, OptimizedMirRelationExpr, RowSetFinishing};
use mz_orchestrator::{OfflineReason, ServiceProcessMetrics};
use mz_ore::cast::{CastFrom, CastLossy};
use mz_ore::channel::trigger::Trigger;
use mz_ore::future::TimeoutError;
use mz_ore::metrics::MetricsRegistry;
use mz_ore::now::{EpochMillis, NowFn};
use mz_ore::task::{spawn, JoinHandle};
use mz_ore::thread::JoinHandleExt;
use mz_ore::tracing::{OpenTelemetryContext, TracingHandle};
use mz_ore::url::SensitiveUrl;
use mz_ore::vec::VecExt;
use mz_ore::{
    assert_none, instrument, soft_assert_eq_or_log, soft_assert_or_log, soft_panic_or_log, stack,
};
use mz_persist_client::usage::{ShardsUsageReferenced, StorageUsageClient};
use mz_repr::explain::{ExplainConfig, ExplainFormat};
use mz_repr::global_id::TransientIdGen;
use mz_repr::optimize::OptimizerFeatures;
use mz_repr::role_id::RoleId;
use mz_repr::{CatalogItemId, Diff, GlobalId, RelationDesc, Row, Timestamp};
use mz_secrets::cache::CachingSecretsReader;
use mz_secrets::{SecretsController, SecretsReader};
use mz_sql::ast::{Raw, Statement};
use mz_sql::catalog::{CatalogCluster, EnvironmentId};
use mz_sql::names::{QualifiedItemName, ResolvedIds, SchemaSpecifier};
use mz_sql::optimizer_metrics::OptimizerMetrics;
use mz_sql::plan::{
    self, AlterSinkPlan, ConnectionDetails, CreateConnectionPlan, NetworkPolicyRule,
    OnTimeoutAction, Params, QueryWhen,
};
use mz_sql::session::user::User;
use mz_sql::session::vars::{ConnectionCounter, SystemVars};
use mz_sql_parser::ast::display::AstDisplay;
use mz_sql_parser::ast::ExplainStage;
use mz_storage_client::client::TimestamplessUpdate;
use mz_storage_client::controller::{CollectionDescription, DataSource};
use mz_storage_types::connections::inline::{IntoInlineConnection, ReferencedConnection};
use mz_storage_types::connections::Connection as StorageConnection;
use mz_storage_types::connections::ConnectionContext;
use mz_storage_types::read_holds::ReadHold;
use mz_storage_types::sinks::S3SinkFormat;
use mz_storage_types::sources::Timeline;
use mz_timestamp_oracle::postgres_oracle::{
    PostgresTimestampOracle, PostgresTimestampOracleConfig,
};
use mz_timestamp_oracle::WriteTimestamp;
use mz_transform::dataflow::DataflowMetainfo;
use opentelemetry::trace::TraceContextExt;
use serde::Serialize;
use thiserror::Error;
use timely::progress::{Antichain, Timestamp as _};
use tokio::runtime::Handle as TokioHandle;
use tokio::select;
use tokio::sync::{mpsc, oneshot, watch, OwnedMutexGuard};
use tokio::time::{Interval, MissedTickBehavior};
use tracing::{debug, info, info_span, span, warn, Instrument, Level, Span};
use tracing_opentelemetry::OpenTelemetrySpanExt;
use uuid::Uuid;

use crate::active_compute_sink::ActiveComputeSink;
use crate::catalog::{BuiltinTableUpdate, Catalog, OpenCatalogResult};
use crate::client::{Client, Handle};
use crate::command::{Command, ExecuteResponse};
use crate::config::{SynchronizedParameters, SystemParameterFrontend, SystemParameterSyncConfig};
use crate::coord::appends::{
    BuiltinTableAppendNotify, DeferredWriteOp, GroupCommitPermit, PendingWriteTxn,
};
use crate::coord::caught_up::CaughtUpCheckContext;
use crate::coord::cluster_scheduling::SchedulingDecision;
use crate::coord::id_bundle::CollectionIdBundle;
use crate::coord::introspection::IntrospectionSubscribe;
use crate::coord::peek::PendingPeek;
use crate::coord::statement_logging::{StatementLogging, StatementLoggingId};
use crate::coord::timeline::{TimelineContext, TimelineState};
use crate::coord::timestamp_selection::{TimestampContext, TimestampDetermination};
use crate::coord::validity::PlanValidity;
use crate::error::AdapterError;
use crate::explain::insights::PlanInsightsContext;
use crate::explain::optimizer_trace::{DispatchGuard, OptimizerTrace};
use crate::metrics::Metrics;
use crate::optimize::dataflows::{
    dataflow_import_id_bundle, ComputeInstanceSnapshot, DataflowBuilder,
};
use crate::optimize::{self, Optimize, OptimizerConfig};
use crate::session::{EndTransactionAction, Session};
use crate::statement_logging::{StatementEndedExecutionReason, StatementLifecycleEvent};
use crate::util::{ClientTransmitter, ResultExt};
use crate::webhook::{WebhookAppenderInvalidator, WebhookConcurrencyLimiter};
use crate::{flags, AdapterNotice, ReadHolds};

pub(crate) mod id_bundle;
pub(crate) mod in_memory_oracle;
pub(crate) mod peek;
pub(crate) mod statement_logging;
pub(crate) mod timeline;
pub(crate) mod timestamp_selection;

mod appends;
mod catalog_serving;
mod caught_up;
pub mod cluster_scheduling;
mod command_handler;
pub mod consistency;
mod ddl;
mod indexes;
mod introspection;
mod message_handler;
mod privatelink_status;
pub mod read_policy;
mod sequencer;
mod sql;
mod validity;

#[derive(Debug)]
pub enum Message {
    Command(OpenTelemetryContext, Command),
    ControllerReady,
    PurifiedStatementReady(PurifiedStatementReady),
    CreateConnectionValidationReady(CreateConnectionValidationReady),
    AlterConnectionValidationReady(AlterConnectionValidationReady),
    TryDeferred {
        /// The connection that created this op.
        conn_id: ConnectionId,
        /// The write lock that notified us our deferred op might be able to run.
        ///
        /// Note: While we never want to hold a partial set of locks, it can be important to hold
        /// onto the _one_ that notified us our op might be ready. If there are multiple operations
        /// waiting on a single collection, and we don't hold this lock through retyring the op,
        /// then everything waiting on this collection will get retried causing traffic in the
        /// Coordinator's message queue.
        ///
        /// See [`DeferredWriteOp::can_be_optimistically_retried`] for more detail.
        acquired_lock: Option<(CatalogItemId, tokio::sync::OwnedMutexGuard<()>)>,
    },
    /// Initiates a group commit.
    GroupCommitInitiate(Span, Option<GroupCommitPermit>),
    DeferredStatementReady,
    AdvanceTimelines,
    ClusterEvent(ClusterEvent),
    CancelPendingPeeks {
        conn_id: ConnectionId,
    },
    LinearizeReads,
    StorageUsageSchedule,
    StorageUsageFetch,
    StorageUsageUpdate(ShardsUsageReferenced),
    StorageUsagePrune(Vec<BuiltinTableUpdate>),
    /// Performs any cleanup and logging actions necessary for
    /// finalizing a statement execution.
    RetireExecute {
        data: ExecuteContextExtra,
        otel_ctx: OpenTelemetryContext,
        reason: StatementEndedExecutionReason,
    },
    ExecuteSingleStatementTransaction {
        ctx: ExecuteContext,
        otel_ctx: OpenTelemetryContext,
        stmt: Arc<Statement<Raw>>,
        params: mz_sql::plan::Params,
    },
    PeekStageReady {
        ctx: ExecuteContext,
        span: Span,
        stage: PeekStage,
    },
    CreateIndexStageReady {
        ctx: ExecuteContext,
        span: Span,
        stage: CreateIndexStage,
    },
    CreateViewStageReady {
        ctx: ExecuteContext,
        span: Span,
        stage: CreateViewStage,
    },
    CreateMaterializedViewStageReady {
        ctx: ExecuteContext,
        span: Span,
        stage: CreateMaterializedViewStage,
    },
    SubscribeStageReady {
        ctx: ExecuteContext,
        span: Span,
        stage: SubscribeStage,
    },
    IntrospectionSubscribeStageReady {
        span: Span,
        stage: IntrospectionSubscribeStage,
    },
    SecretStageReady {
        ctx: ExecuteContext,
        span: Span,
        stage: SecretStage,
    },
    ClusterStageReady {
        ctx: ExecuteContext,
        span: Span,
        stage: ClusterStage,
    },
    ExplainTimestampStageReady {
        ctx: ExecuteContext,
        span: Span,
        stage: ExplainTimestampStage,
    },
    DrainStatementLog,
    PrivateLinkVpcEndpointEvents(Vec<VpcEndpointEvent>),
    CheckSchedulingPolicies,

    /// Scheduling policy decisions about turning clusters On/Off.
    /// `Vec<(policy name, Vec of decisions by the policy)>`
    /// A cluster will be On if and only if there is at least one On decision for it.
    /// Scheduling decisions for clusters that have `SCHEDULE = MANUAL` are ignored.
    SchedulingDecisions(Vec<(&'static str, Vec<(ClusterId, SchedulingDecision)>)>),
}

impl Message {
    /// Returns a string to identify the kind of [`Message`], useful for logging.
    pub const fn kind(&self) -> &'static str {
        match self {
            Message::Command(_, msg) => match msg {
                Command::CatalogSnapshot { .. } => "command-catalog_snapshot",
                Command::Startup { .. } => "command-startup",
                Command::Execute { .. } => "command-execute",
                Command::Commit { .. } => "command-commit",
                Command::CancelRequest { .. } => "command-cancel_request",
                Command::PrivilegedCancelRequest { .. } => "command-privileged_cancel_request",
                Command::GetWebhook { .. } => "command-get_webhook",
                Command::GetSystemVars { .. } => "command-get_system_vars",
                Command::SetSystemVars { .. } => "command-set_system_vars",
                Command::Terminate { .. } => "command-terminate",
                Command::RetireExecute { .. } => "command-retire_execute",
                Command::CheckConsistency { .. } => "command-check_consistency",
                Command::Dump { .. } => "command-dump",
            },
            Message::ControllerReady => "controller_ready",
            Message::PurifiedStatementReady(_) => "purified_statement_ready",
            Message::CreateConnectionValidationReady(_) => "create_connection_validation_ready",
            Message::TryDeferred { .. } => "try_deferred",
            Message::GroupCommitInitiate(..) => "group_commit_initiate",
            Message::AdvanceTimelines => "advance_timelines",
            Message::ClusterEvent(_) => "cluster_event",
            Message::CancelPendingPeeks { .. } => "cancel_pending_peeks",
            Message::LinearizeReads => "linearize_reads",
            Message::StorageUsageSchedule => "storage_usage_schedule",
            Message::StorageUsageFetch => "storage_usage_fetch",
            Message::StorageUsageUpdate(_) => "storage_usage_update",
            Message::StorageUsagePrune(_) => "storage_usage_prune",
            Message::RetireExecute { .. } => "retire_execute",
            Message::ExecuteSingleStatementTransaction { .. } => {
                "execute_single_statement_transaction"
            }
            Message::PeekStageReady { .. } => "peek_stage_ready",
            Message::ExplainTimestampStageReady { .. } => "explain_timestamp_stage_ready",
            Message::CreateIndexStageReady { .. } => "create_index_stage_ready",
            Message::CreateViewStageReady { .. } => "create_view_stage_ready",
            Message::CreateMaterializedViewStageReady { .. } => {
                "create_materialized_view_stage_ready"
            }
            Message::SubscribeStageReady { .. } => "subscribe_stage_ready",
            Message::IntrospectionSubscribeStageReady { .. } => {
                "introspection_subscribe_stage_ready"
            }
            Message::SecretStageReady { .. } => "secret_stage_ready",
            Message::ClusterStageReady { .. } => "cluster_stage_ready",
            Message::DrainStatementLog => "drain_statement_log",
            Message::AlterConnectionValidationReady(..) => "alter_connection_validation_ready",
            Message::PrivateLinkVpcEndpointEvents(_) => "private_link_vpc_endpoint_events",
            Message::CheckSchedulingPolicies => "check_scheduling_policies",
            Message::SchedulingDecisions { .. } => "scheduling_decision",
            Message::DeferredStatementReady => "deferred_statement_ready",
        }
    }
}

#[derive(Derivative)]
#[derivative(Debug)]
pub struct BackgroundWorkResult<T> {
    #[derivative(Debug = "ignore")]
    pub ctx: ExecuteContext,
    pub result: Result<T, AdapterError>,
    pub params: Params,
    pub plan_validity: PlanValidity,
    pub original_stmt: Arc<Statement<Raw>>,
    pub otel_ctx: OpenTelemetryContext,
}

pub type PurifiedStatementReady = BackgroundWorkResult<mz_sql::pure::PurifiedStatement>;

#[derive(Derivative)]
#[derivative(Debug)]
pub struct ValidationReady<T> {
    #[derivative(Debug = "ignore")]
    pub ctx: ExecuteContext,
    pub result: Result<T, AdapterError>,
    pub resolved_ids: ResolvedIds,
    pub connection_id: CatalogItemId,
    pub connection_gid: GlobalId,
    pub plan_validity: PlanValidity,
    pub otel_ctx: OpenTelemetryContext,
}

pub type CreateConnectionValidationReady = ValidationReady<CreateConnectionPlan>;
pub type AlterConnectionValidationReady = ValidationReady<Connection>;

#[derive(Debug)]
pub enum PeekStage {
    /// Common stages across SELECT, EXPLAIN and COPY TO queries.
    LinearizeTimestamp(PeekStageLinearizeTimestamp),
    RealTimeRecency(PeekStageRealTimeRecency),
    TimestampReadHold(PeekStageTimestampReadHold),
    Optimize(PeekStageOptimize),
    /// Final stage for a peek.
    Finish(PeekStageFinish),
    /// Final stage for an explain.
    ExplainPlan(PeekStageExplainPlan),
    ExplainPushdown(PeekStageExplainPushdown),
    /// Final stage for a copy to.
    CopyTo(PeekStageCopyTo),
}

#[derive(Debug)]
pub struct CopyToContext {
    /// The `RelationDesc` of the data to be copied.
    pub desc: RelationDesc,
    /// The destination uri of the external service where the data will be copied.
    pub uri: Uri,
    /// Connection information required to connect to the external service to copy the data.
    pub connection: StorageConnection<ReferencedConnection>,
    /// The ID of the CONNECTION object to be used for copying the data.
    pub connection_id: CatalogItemId,
    /// Format params to format the data.
    pub format: S3SinkFormat,
    /// Approximate max file size of each uploaded file.
    pub max_file_size: u64,
    /// Number of batches the output of the COPY TO will be partitioned into
    /// to distribute the load across workers deterministically.
    /// This is only an option since it's not set when CopyToContext is instantiated
    /// but immediately after in the PeekStageValidate stage.
    pub output_batch_count: Option<u64>,
}

#[derive(Debug)]
pub struct PeekStageLinearizeTimestamp {
    validity: PlanValidity,
    plan: mz_sql::plan::SelectPlan,
    max_query_result_size: Option<u64>,
    source_ids: BTreeSet<GlobalId>,
    target_replica: Option<ReplicaId>,
    timeline_context: TimelineContext,
    optimizer: Either<optimize::peek::Optimizer, optimize::copy_to::Optimizer>,
    /// An optional context set iff the state machine is initiated from
    /// sequencing an EXPLAIN for this statement.
    explain_ctx: ExplainContext,
}

#[derive(Debug)]
pub struct PeekStageRealTimeRecency {
    validity: PlanValidity,
    plan: mz_sql::plan::SelectPlan,
    max_query_result_size: Option<u64>,
    source_ids: BTreeSet<GlobalId>,
    target_replica: Option<ReplicaId>,
    timeline_context: TimelineContext,
    oracle_read_ts: Option<Timestamp>,
    optimizer: Either<optimize::peek::Optimizer, optimize::copy_to::Optimizer>,
    /// An optional context set iff the state machine is initiated from
    /// sequencing an EXPLAIN for this statement.
    explain_ctx: ExplainContext,
}

#[derive(Debug)]
pub struct PeekStageTimestampReadHold {
    validity: PlanValidity,
    plan: mz_sql::plan::SelectPlan,
    max_query_result_size: Option<u64>,
    source_ids: BTreeSet<GlobalId>,
    target_replica: Option<ReplicaId>,
    timeline_context: TimelineContext,
    oracle_read_ts: Option<Timestamp>,
    real_time_recency_ts: Option<mz_repr::Timestamp>,
    optimizer: Either<optimize::peek::Optimizer, optimize::copy_to::Optimizer>,
    /// An optional context set iff the state machine is initiated from
    /// sequencing an EXPLAIN for this statement.
    explain_ctx: ExplainContext,
}

#[derive(Debug)]
pub struct PeekStageOptimize {
    validity: PlanValidity,
    plan: mz_sql::plan::SelectPlan,
    max_query_result_size: Option<u64>,
    source_ids: BTreeSet<GlobalId>,
    id_bundle: CollectionIdBundle,
    target_replica: Option<ReplicaId>,
    determination: TimestampDetermination<mz_repr::Timestamp>,
    optimizer: Either<optimize::peek::Optimizer, optimize::copy_to::Optimizer>,
    /// An optional context set iff the state machine is initiated from
    /// sequencing an EXPLAIN for this statement.
    explain_ctx: ExplainContext,
}

#[derive(Debug)]
pub struct PeekStageFinish {
    validity: PlanValidity,
    plan: mz_sql::plan::SelectPlan,
    max_query_result_size: Option<u64>,
    id_bundle: CollectionIdBundle,
    target_replica: Option<ReplicaId>,
    source_ids: BTreeSet<GlobalId>,
    determination: TimestampDetermination<mz_repr::Timestamp>,
    cluster_id: ComputeInstanceId,
    finishing: RowSetFinishing,
    /// When present, an optimizer trace to be used for emitting a plan insights
    /// notice.
    plan_insights_optimizer_trace: Option<OptimizerTrace>,
    insights_ctx: Option<Box<PlanInsightsContext>>,
    global_lir_plan: optimize::peek::GlobalLirPlan,
    optimization_finished_at: EpochMillis,
}

#[derive(Debug)]
pub struct PeekStageCopyTo {
    validity: PlanValidity,
    optimizer: optimize::copy_to::Optimizer,
    global_lir_plan: optimize::copy_to::GlobalLirPlan,
    optimization_finished_at: EpochMillis,
    source_ids: BTreeSet<GlobalId>,
}

#[derive(Debug)]
pub struct PeekStageExplainPlan {
    validity: PlanValidity,
    optimizer: optimize::peek::Optimizer,
    df_meta: DataflowMetainfo,
    explain_ctx: ExplainPlanContext,
    insights_ctx: Option<Box<PlanInsightsContext>>,
}

#[derive(Debug)]
pub struct PeekStageExplainPushdown {
    validity: PlanValidity,
    determination: TimestampDetermination<mz_repr::Timestamp>,
    imports: BTreeMap<GlobalId, MapFilterProject>,
}

#[derive(Debug)]
pub enum CreateIndexStage {
    Optimize(CreateIndexOptimize),
    Finish(CreateIndexFinish),
    Explain(CreateIndexExplain),
}

#[derive(Debug)]
pub struct CreateIndexOptimize {
    validity: PlanValidity,
    plan: plan::CreateIndexPlan,
    resolved_ids: ResolvedIds,
    /// An optional context set iff the state machine is initiated from
    /// sequencing an EXPLAIN for this statement.
    explain_ctx: ExplainContext,
}

#[derive(Debug)]
pub struct CreateIndexFinish {
    validity: PlanValidity,
    item_id: CatalogItemId,
    global_id: GlobalId,
    plan: plan::CreateIndexPlan,
    resolved_ids: ResolvedIds,
    global_mir_plan: optimize::index::GlobalMirPlan,
    global_lir_plan: optimize::index::GlobalLirPlan,
}

#[derive(Debug)]
pub struct CreateIndexExplain {
    validity: PlanValidity,
    exported_index_id: GlobalId,
    plan: plan::CreateIndexPlan,
    df_meta: DataflowMetainfo,
    explain_ctx: ExplainPlanContext,
}

#[derive(Debug)]
pub enum CreateViewStage {
    Optimize(CreateViewOptimize),
    Finish(CreateViewFinish),
    Explain(CreateViewExplain),
}

#[derive(Debug)]
pub struct CreateViewOptimize {
    validity: PlanValidity,
    plan: plan::CreateViewPlan,
    resolved_ids: ResolvedIds,
    /// An optional context set iff the state machine is initiated from
    /// sequencing an EXPLAIN for this statement.
    explain_ctx: ExplainContext,
}

#[derive(Debug)]
pub struct CreateViewFinish {
    validity: PlanValidity,
    /// ID of this item in the Catalog.
    item_id: CatalogItemId,
    /// ID by with Compute will reference this View.
    global_id: GlobalId,
    plan: plan::CreateViewPlan,
    /// IDs of objects resolved during name resolution.
    resolved_ids: ResolvedIds,
    optimized_expr: OptimizedMirRelationExpr,
}

#[derive(Debug)]
pub struct CreateViewExplain {
    validity: PlanValidity,
    id: GlobalId,
    plan: plan::CreateViewPlan,
    explain_ctx: ExplainPlanContext,
}

#[derive(Debug)]
pub enum ExplainTimestampStage {
    Optimize(ExplainTimestampOptimize),
    RealTimeRecency(ExplainTimestampRealTimeRecency),
    Finish(ExplainTimestampFinish),
}

#[derive(Debug)]
pub struct ExplainTimestampOptimize {
    validity: PlanValidity,
    plan: plan::ExplainTimestampPlan,
    cluster_id: ClusterId,
}

#[derive(Debug)]
pub struct ExplainTimestampRealTimeRecency {
    validity: PlanValidity,
    format: ExplainFormat,
    optimized_plan: OptimizedMirRelationExpr,
    cluster_id: ClusterId,
    when: QueryWhen,
}

#[derive(Debug)]
pub struct ExplainTimestampFinish {
    validity: PlanValidity,
    format: ExplainFormat,
    optimized_plan: OptimizedMirRelationExpr,
    cluster_id: ClusterId,
    source_ids: BTreeSet<GlobalId>,
    when: QueryWhen,
    real_time_recency_ts: Option<Timestamp>,
}

#[derive(Debug)]
pub enum ClusterStage {
    Alter(AlterCluster),
    WaitForHydrated(AlterClusterWaitForHydrated),
    Finalize(AlterClusterFinalize),
}

#[derive(Debug)]
pub struct AlterCluster {
    validity: PlanValidity,
    plan: plan::AlterClusterPlan,
}

#[derive(Debug)]
pub struct AlterClusterWaitForHydrated {
    validity: PlanValidity,
    plan: plan::AlterClusterPlan,
    new_config: ClusterVariantManaged,
    timeout_time: Instant,
    on_timeout: OnTimeoutAction,
}

#[derive(Debug)]
pub struct AlterClusterFinalize {
    validity: PlanValidity,
    plan: plan::AlterClusterPlan,
    new_config: ClusterVariantManaged,
}

#[derive(Debug)]
pub enum ExplainContext {
    /// The ordinary, non-explain variant of the statement.
    None,
    /// The `EXPLAIN <level> PLAN FOR <explainee>` version of the statement.
    Plan(ExplainPlanContext),
    /// Generate a notice containing the `EXPLAIN PLAN INSIGHTS` output
    /// alongside the query's normal output.
    PlanInsightsNotice(OptimizerTrace),
    /// `EXPLAIN FILTER PUSHDOWN`
    Pushdown,
}

impl ExplainContext {
    /// If available for this context, wrap the [`OptimizerTrace`] into a
    /// [`tracing::Dispatch`] and set it as default, returning the resulting
    /// guard in a `Some(guard)` option.
    fn dispatch_guard(&self) -> Option<DispatchGuard<'_>> {
        let optimizer_trace = match self {
            ExplainContext::Plan(explain_ctx) => Some(&explain_ctx.optimizer_trace),
            ExplainContext::PlanInsightsNotice(optimizer_trace) => Some(optimizer_trace),
            _ => None,
        };
        optimizer_trace.map(|optimizer_trace| optimizer_trace.as_guard())
    }

    fn needs_cluster(&self) -> bool {
        match self {
            ExplainContext::None => true,
            ExplainContext::Plan(..) => false,
            ExplainContext::PlanInsightsNotice(..) => true,
            ExplainContext::Pushdown => false,
        }
    }

    fn needs_plan_insights(&self) -> bool {
        matches!(
            self,
            ExplainContext::Plan(ExplainPlanContext {
                stage: ExplainStage::PlanInsights,
                ..
            }) | ExplainContext::PlanInsightsNotice(_)
        )
    }
}

#[derive(Debug)]
pub struct ExplainPlanContext {
    pub broken: bool,
    pub config: ExplainConfig,
    pub format: ExplainFormat,
    pub stage: ExplainStage,
    pub replan: Option<GlobalId>,
    pub desc: Option<RelationDesc>,
    pub optimizer_trace: OptimizerTrace,
}

#[derive(Debug)]
pub enum CreateMaterializedViewStage {
    Optimize(CreateMaterializedViewOptimize),
    Finish(CreateMaterializedViewFinish),
    Explain(CreateMaterializedViewExplain),
}

#[derive(Debug)]
pub struct CreateMaterializedViewOptimize {
    validity: PlanValidity,
    plan: plan::CreateMaterializedViewPlan,
    resolved_ids: ResolvedIds,
    /// An optional context set iff the state machine is initiated from
    /// sequencing an EXPLAIN for this statement.
    explain_ctx: ExplainContext,
}

#[derive(Debug)]
pub struct CreateMaterializedViewFinish {
    /// The ID of this Materialized View in the Catalog.
    item_id: CatalogItemId,
    /// The ID of the durable pTVC backing this Materialized View.
    global_id: GlobalId,
    validity: PlanValidity,
    plan: plan::CreateMaterializedViewPlan,
    resolved_ids: ResolvedIds,
    local_mir_plan: optimize::materialized_view::LocalMirPlan,
    global_mir_plan: optimize::materialized_view::GlobalMirPlan,
    global_lir_plan: optimize::materialized_view::GlobalLirPlan,
}

#[derive(Debug)]
pub struct CreateMaterializedViewExplain {
    global_id: GlobalId,
    validity: PlanValidity,
    plan: plan::CreateMaterializedViewPlan,
    df_meta: DataflowMetainfo,
    explain_ctx: ExplainPlanContext,
}

#[derive(Debug)]
pub enum SubscribeStage {
    OptimizeMir(SubscribeOptimizeMir),
    TimestampOptimizeLir(SubscribeTimestampOptimizeLir),
    Finish(SubscribeFinish),
}

#[derive(Debug)]
pub struct SubscribeOptimizeMir {
    validity: PlanValidity,
    plan: plan::SubscribePlan,
    timeline: TimelineContext,
    dependency_ids: BTreeSet<GlobalId>,
    cluster_id: ComputeInstanceId,
    replica_id: Option<ReplicaId>,
}

#[derive(Debug)]
pub struct SubscribeTimestampOptimizeLir {
    validity: PlanValidity,
    plan: plan::SubscribePlan,
    timeline: TimelineContext,
    optimizer: optimize::subscribe::Optimizer,
    global_mir_plan: optimize::subscribe::GlobalMirPlan<optimize::subscribe::Unresolved>,
    dependency_ids: BTreeSet<GlobalId>,
    replica_id: Option<ReplicaId>,
}

#[derive(Debug)]
pub struct SubscribeFinish {
    validity: PlanValidity,
    cluster_id: ComputeInstanceId,
    replica_id: Option<ReplicaId>,
    plan: plan::SubscribePlan,
    global_lir_plan: optimize::subscribe::GlobalLirPlan,
    dependency_ids: BTreeSet<GlobalId>,
}

#[derive(Debug)]
pub enum IntrospectionSubscribeStage {
    OptimizeMir(IntrospectionSubscribeOptimizeMir),
    TimestampOptimizeLir(IntrospectionSubscribeTimestampOptimizeLir),
    Finish(IntrospectionSubscribeFinish),
}

#[derive(Debug)]
pub struct IntrospectionSubscribeOptimizeMir {
    validity: PlanValidity,
    plan: plan::SubscribePlan,
    subscribe_id: GlobalId,
    cluster_id: ComputeInstanceId,
    replica_id: ReplicaId,
}

#[derive(Debug)]
pub struct IntrospectionSubscribeTimestampOptimizeLir {
    validity: PlanValidity,
    optimizer: optimize::subscribe::Optimizer,
    global_mir_plan: optimize::subscribe::GlobalMirPlan<optimize::subscribe::Unresolved>,
    cluster_id: ComputeInstanceId,
    replica_id: ReplicaId,
}

#[derive(Debug)]
pub struct IntrospectionSubscribeFinish {
    validity: PlanValidity,
    global_lir_plan: optimize::subscribe::GlobalLirPlan,
    read_holds: ReadHolds<Timestamp>,
    cluster_id: ComputeInstanceId,
    replica_id: ReplicaId,
}

#[derive(Debug)]
pub enum SecretStage {
    CreateEnsure(CreateSecretEnsure),
    CreateFinish(CreateSecretFinish),
    RotateKeysEnsure(RotateKeysSecretEnsure),
    RotateKeysFinish(RotateKeysSecretFinish),
    Alter(AlterSecret),
}

#[derive(Debug)]
pub struct CreateSecretEnsure {
    validity: PlanValidity,
    plan: plan::CreateSecretPlan,
}

#[derive(Debug)]
pub struct CreateSecretFinish {
    validity: PlanValidity,
    item_id: CatalogItemId,
    global_id: GlobalId,
    plan: plan::CreateSecretPlan,
}

#[derive(Debug)]
pub struct RotateKeysSecretEnsure {
    validity: PlanValidity,
    id: CatalogItemId,
}

#[derive(Debug)]
pub struct RotateKeysSecretFinish {
    validity: PlanValidity,
    ops: Vec<crate::catalog::Op>,
}

#[derive(Debug)]
pub struct AlterSecret {
    validity: PlanValidity,
    plan: plan::AlterSecretPlan,
}

/// An enum describing which cluster to run a statement on.
///
/// One example usage would be that if a query depends only on system tables, we might
/// automatically run it on the catalog server cluster to benefit from indexes that exist there.
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum TargetCluster {
    /// The catalog server cluster.
    CatalogServer,
    /// The current user's active cluster.
    Active,
    /// The cluster selected at the start of a transaction.
    Transaction(ClusterId),
}

/// Result types for each stage of a sequence.
pub(crate) enum StageResult<T> {
    /// A task was spawned that will return the next stage.
    Handle(JoinHandle<Result<T, AdapterError>>),
    /// A task was spawned that will return a response for the client.
    HandleRetire(JoinHandle<Result<ExecuteResponse, AdapterError>>),
    /// The next stage is immediately ready and will execute.
    Immediate(T),
    /// The final stage was executed and is ready to respond to the client.
    Response(ExecuteResponse),
}

/// Common functionality for [Coordinator::sequence_staged].
pub(crate) trait Staged: Send {
    type Ctx: StagedContext;

    fn validity(&mut self) -> &mut PlanValidity;

    /// Returns the next stage or final result.
    async fn stage(
        self,
        coord: &mut Coordinator,
        ctx: &mut Self::Ctx,
    ) -> Result<StageResult<Box<Self>>, AdapterError>;

    /// Prepares a message for the Coordinator.
    fn message(self, ctx: Self::Ctx, span: Span) -> Message;

    /// Whether it is safe to SQL cancel this stage.
    fn cancel_enabled(&self) -> bool;
}

pub trait StagedContext {
    fn retire(self, result: Result<ExecuteResponse, AdapterError>);
    fn session(&self) -> Option<&Session>;
}

impl StagedContext for ExecuteContext {
    fn retire(self, result: Result<ExecuteResponse, AdapterError>) {
        self.retire(result);
    }

    fn session(&self) -> Option<&Session> {
        Some(self.session())
    }
}

impl StagedContext for () {
    fn retire(self, _result: Result<ExecuteResponse, AdapterError>) {}

    fn session(&self) -> Option<&Session> {
        None
    }
}

/// Configures a coordinator.
pub struct Config {
    pub controller_config: ControllerConfig,
    pub controller_envd_epoch: NonZeroI64,
    pub storage: Box<dyn mz_catalog::durable::DurableCatalogState>,
    pub timestamp_oracle_url: Option<SensitiveUrl>,
    pub unsafe_mode: bool,
    pub all_features: bool,
    pub build_info: &'static BuildInfo,
    pub environment_id: EnvironmentId,
    pub metrics_registry: MetricsRegistry,
    pub now: NowFn,
    pub secrets_controller: Arc<dyn SecretsController>,
    pub cloud_resource_controller: Option<Arc<dyn CloudResourceController>>,
    pub availability_zones: Vec<String>,
    pub cluster_replica_sizes: ClusterReplicaSizeMap,
    pub builtin_system_cluster_replica_size: String,
    pub builtin_catalog_server_cluster_replica_size: String,
    pub builtin_probe_cluster_replica_size: String,
    pub builtin_support_cluster_replica_size: String,
    pub builtin_analytics_cluster_replica_size: String,
    pub system_parameter_defaults: BTreeMap<String, String>,
    pub storage_usage_client: StorageUsageClient,
    pub storage_usage_collection_interval: Duration,
    pub storage_usage_retention_period: Option<Duration>,
    pub segment_client: Option<mz_segment::Client>,
    pub egress_addresses: Vec<IpNet>,
    pub remote_system_parameters: Option<BTreeMap<String, String>>,
    pub aws_account_id: Option<String>,
    pub aws_privatelink_availability_zones: Option<Vec<String>>,
    pub connection_context: ConnectionContext,
    pub active_connection_count: Arc<Mutex<ConnectionCounter>>,
    pub webhook_concurrency_limit: WebhookConcurrencyLimiter,
    pub http_host_name: Option<String>,
    pub tracing_handle: TracingHandle,
    /// Whether or not to start controllers in read-only mode. This is only
    /// meant for use during development of read-only clusters and 0dt upgrades
    /// and should go away once we have proper orchestration during upgrades.
    pub read_only_controllers: bool,
    /// Whether to enable zero-downtime deployments.
    pub enable_0dt_deployment: bool,

    /// A trigger that signals that the current deployment has caught up with a
    /// previous deployment. Only used during 0dt deployment, while in read-only
    /// mode.
    pub caught_up_trigger: Option<Trigger>,

    pub helm_chart_version: Option<String>,
}

/// Soft-state metadata about a compute replica
#[derive(Clone, Default, Debug, Eq, PartialEq)]
pub struct ReplicaMetadata {
    /// The last known CPU and memory metrics
    pub metrics: Option<Vec<ServiceProcessMetrics>>,
}

/// Metadata about an active connection.
#[derive(Debug, Serialize)]
pub struct ConnMeta {
    /// 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,
    /// The time when the session's connection was initiated.
    connected_at: EpochMillis,
    user: User,
    application_name: String,
    uuid: Uuid,
    conn_id: ConnectionId,
    client_ip: Option<IpAddr>,

    /// Sinks that will need to be dropped when the current transaction, if
    /// any, is cleared.
    drop_sinks: BTreeSet<GlobalId>,

    /// Lock for the Coordinator's deferred statements that is dropped on transaction clear.
    #[serde(skip)]
    deferred_lock: Option<OwnedMutexGuard<()>>,

    /// Cluster reconfigurations that will need to be
    /// cleaned up when the current transaction is cleared
    pending_cluster_alters: BTreeSet<ClusterId>,

    /// Channel on which to send notices to a session.
    #[serde(skip)]
    notice_tx: mpsc::UnboundedSender<AdapterNotice>,

    /// The role that initiated the database context. Fixed for the duration of the connection.
    /// WARNING: This role reference is not updated when the role is dropped.
    /// Consumers should not assume that this role exist.
    authenticated_role: RoleId,
}

impl ConnMeta {
    pub fn conn_id(&self) -> &ConnectionId {
        &self.conn_id
    }

    pub fn user(&self) -> &User {
        &self.user
    }

    pub fn application_name(&self) -> &str {
        &self.application_name
    }

    pub fn authenticated_role_id(&self) -> &RoleId {
        &self.authenticated_role
    }

    pub fn uuid(&self) -> Uuid {
        self.uuid
    }

    pub fn client_ip(&self) -> Option<IpAddr> {
        self.client_ip
    }

    pub fn connected_at(&self) -> EpochMillis {
        self.connected_at
    }
}

#[derive(Debug)]
/// A pending transaction waiting to be committed.
pub struct PendingTxn {
    /// Context used to send a response back to the client.
    ctx: ExecuteContext,
    /// Client response for transaction.
    response: Result<PendingTxnResponse, AdapterError>,
    /// The action to take at the end of the transaction.
    action: EndTransactionAction,
}

#[derive(Debug)]
/// The response we'll send for a [`PendingTxn`].
pub enum PendingTxnResponse {
    /// The transaction will be committed.
    Committed {
        /// Parameters that will change, and their values, once this transaction is complete.
        params: BTreeMap<&'static str, String>,
    },
    /// The transaction will be rolled back.
    Rolledback {
        /// Parameters that will change, and their values, once this transaction is complete.
        params: BTreeMap<&'static str, String>,
    },
}

impl PendingTxnResponse {
    pub fn extend_params(&mut self, p: impl IntoIterator<Item = (&'static str, String)>) {
        match self {
            PendingTxnResponse::Committed { params }
            | PendingTxnResponse::Rolledback { params } => params.extend(p),
        }
    }
}

impl From<PendingTxnResponse> for ExecuteResponse {
    fn from(value: PendingTxnResponse) -> Self {
        match value {
            PendingTxnResponse::Committed { params } => {
                ExecuteResponse::TransactionCommitted { params }
            }
            PendingTxnResponse::Rolledback { params } => {
                ExecuteResponse::TransactionRolledBack { params }
            }
        }
    }
}

#[derive(Debug)]
/// A pending read transaction waiting to be linearized along with metadata about it's state
pub struct PendingReadTxn {
    /// The transaction type
    txn: PendingRead,
    /// The timestamp context of the transaction.
    timestamp_context: TimestampContext<mz_repr::Timestamp>,
    /// When we created this pending txn, when the transaction ends. Only used for metrics.
    created: Instant,
    /// Number of times we requeued the processing of this pending read txn.
    /// Requeueing is necessary if the time we executed the query is after the current oracle time;
    /// see [`Coordinator::message_linearize_reads`] for more details.
    num_requeues: u64,
    /// Telemetry context.
    otel_ctx: OpenTelemetryContext,
}

impl PendingReadTxn {
    /// Return the timestamp context of the pending read transaction.
    pub fn timestamp_context(&self) -> &TimestampContext<mz_repr::Timestamp> {
        &self.timestamp_context
    }

    pub(crate) fn take_context(self) -> ExecuteContext {
        self.txn.take_context()
    }
}

#[derive(Debug)]
/// A pending read transaction waiting to be linearized.
enum PendingRead {
    Read {
        /// The inner transaction.
        txn: PendingTxn,
    },
    ReadThenWrite {
        /// Context used to send a response back to the client.
        ctx: ExecuteContext,
        /// Channel used to alert the transaction that the read has been linearized and send back
        /// `ctx`.
        tx: oneshot::Sender<Option<ExecuteContext>>,
    },
}

impl PendingRead {
    /// Alert the client that the read has been linearized.
    ///
    /// If it is necessary to finalize an execute, return the state necessary to do so
    /// (execution context and result)
    #[instrument(level = "debug")]
    pub fn finish(self) -> Option<(ExecuteContext, Result<ExecuteResponse, AdapterError>)> {
        match self {
            PendingRead::Read {
                txn:
                    PendingTxn {
                        mut ctx,
                        response,
                        action,
                    },
                ..
            } => {
                let changed = ctx.session_mut().vars_mut().end_transaction(action);
                // Append any parameters that changed to the response.
                let response = response.map(|mut r| {
                    r.extend_params(changed);
                    ExecuteResponse::from(r)
                });

                Some((ctx, response))
            }
            PendingRead::ReadThenWrite { ctx, tx, .. } => {
                // Ignore errors if the caller has hung up.
                let _ = tx.send(Some(ctx));
                None
            }
        }
    }

    fn label(&self) -> &'static str {
        match self {
            PendingRead::Read { .. } => "read",
            PendingRead::ReadThenWrite { .. } => "read_then_write",
        }
    }

    pub(crate) fn take_context(self) -> ExecuteContext {
        match self {
            PendingRead::Read { txn, .. } => txn.ctx,
            PendingRead::ReadThenWrite { ctx, tx, .. } => {
                // Inform the transaction that we've taken their context.
                // Ignore errors if the caller has hung up.
                let _ = tx.send(None);
                ctx
            }
        }
    }
}

/// State that the coordinator must process as part of retiring
/// command execution.  `ExecuteContextExtra::Default` is guaranteed
/// to produce a value that will cause the coordinator to do nothing, and
/// is intended for use by code that invokes the execution processing flow
/// (i.e., `sequence_plan`) without actually being a statement execution.
///
/// This struct must not be dropped if it contains non-trivial
/// state. The only valid way to get rid of it is to pass it to the
/// coordinator for retirement. To enforce this, we assert in the
/// `Drop` implementation.
#[derive(Debug, Default)]
#[must_use]
pub struct ExecuteContextExtra {
    statement_uuid: Option<StatementLoggingId>,
}

impl ExecuteContextExtra {
    pub(crate) fn new(statement_uuid: Option<StatementLoggingId>) -> Self {
        Self { statement_uuid }
    }
    pub fn is_trivial(&self) -> bool {
        let Self { statement_uuid } = self;
        statement_uuid.is_none()
    }
    pub fn contents(&self) -> Option<StatementLoggingId> {
        let Self { statement_uuid } = self;
        *statement_uuid
    }
    /// Take responsibility for the contents.  This should only be
    /// called from code that knows what to do to finish up logging
    /// based on the inner value.
    #[must_use]
    fn retire(mut self) -> Option<StatementLoggingId> {
        let Self { statement_uuid } = &mut self;
        statement_uuid.take()
    }
}

impl Drop for ExecuteContextExtra {
    fn drop(&mut self) {
        let Self { statement_uuid } = &*self;
        if let Some(statement_uuid) = statement_uuid {
            // Note: the impact when this error hits
            // is that the statement will never be marked
            // as finished in the statement log.
            soft_panic_or_log!("execute context for statement {statement_uuid:?} dropped without being properly retired.");
        }
    }
}

/// Bundle of state related to statement execution.
///
/// This struct collects a bundle of state that needs to be threaded
/// through various functions as part of statement execution.
/// Currently, it is only used to finalize execution, by calling one
/// of the methods `retire` or `retire_aysnc`. Finalizing execution
/// involves sending the session back to the pgwire layer so that it
/// may be used to process further commands. In the future, it will
/// also involve performing some work on the main coordinator thread
/// (e.g., recording the time at which the statement finished
/// executing) the state necessary to perform this work is bundled in
/// the `ExecuteContextExtra` object (today, it is simply empty).
#[derive(Debug)]
pub struct ExecuteContext {
    inner: Box<ExecuteContextInner>,
}

impl std::ops::Deref for ExecuteContext {
    type Target = ExecuteContextInner;
    fn deref(&self) -> &Self::Target {
        &*self.inner
    }
}

impl std::ops::DerefMut for ExecuteContext {
    fn deref_mut(&mut self) -> &mut Self::Target {
        &mut *self.inner
    }
}

#[derive(Debug)]
pub struct ExecuteContextInner {
    tx: ClientTransmitter<ExecuteResponse>,
    internal_cmd_tx: mpsc::UnboundedSender<Message>,
    session: Session,
    extra: ExecuteContextExtra,
}

impl ExecuteContext {
    pub fn session(&self) -> &Session {
        &self.session
    }

    pub fn session_mut(&mut self) -> &mut Session {
        &mut self.session
    }

    pub fn tx(&self) -> &ClientTransmitter<ExecuteResponse> {
        &self.tx
    }

    pub fn tx_mut(&mut self) -> &mut ClientTransmitter<ExecuteResponse> {
        &mut self.tx
    }

    pub fn from_parts(
        tx: ClientTransmitter<ExecuteResponse>,
        internal_cmd_tx: mpsc::UnboundedSender<Message>,
        session: Session,
        extra: ExecuteContextExtra,
    ) -> Self {
        Self {
            inner: ExecuteContextInner {
                tx,
                session,
                extra,
                internal_cmd_tx,
            }
            .into(),
        }
    }

    /// By calling this function, the caller takes responsibility for
    /// dealing with the instance of `ExecuteContextExtra`. This is
    /// intended to support protocols (like `COPY FROM`) that involve
    /// multiple passes of sending the session back and forth between
    /// the coordinator and the pgwire layer. As part of any such
    /// protocol, we must ensure that the `ExecuteContextExtra`
    /// (possibly wrapped in a new `ExecuteContext`) is passed back to the coordinator for
    /// eventual retirement.
    pub fn into_parts(
        self,
    ) -> (
        ClientTransmitter<ExecuteResponse>,
        mpsc::UnboundedSender<Message>,
        Session,
        ExecuteContextExtra,
    ) {
        let ExecuteContextInner {
            tx,
            internal_cmd_tx,
            session,
            extra,
        } = *self.inner;
        (tx, internal_cmd_tx, session, extra)
    }

    /// Retire the execution, by sending a message to the coordinator.
    #[instrument(level = "debug")]
    pub fn retire(self, result: Result<ExecuteResponse, AdapterError>) {
        let ExecuteContextInner {
            tx,
            internal_cmd_tx,
            session,
            extra,
        } = *self.inner;
        let reason = if extra.is_trivial() {
            None
        } else {
            Some((&result).into())
        };
        tx.send(result, session);
        if let Some(reason) = reason {
            if let Err(e) = internal_cmd_tx.send(Message::RetireExecute {
                otel_ctx: OpenTelemetryContext::obtain(),
                data: extra,
                reason,
            }) {
                warn!("internal_cmd_rx dropped before we could send: {:?}", e);
            }
        }
    }

    pub fn extra(&self) -> &ExecuteContextExtra {
        &self.extra
    }

    pub fn extra_mut(&mut self) -> &mut ExecuteContextExtra {
        &mut self.extra
    }
}

#[derive(Debug)]
struct ClusterReplicaStatuses(
    BTreeMap<ClusterId, BTreeMap<ReplicaId, BTreeMap<ProcessId, ClusterReplicaProcessStatus>>>,
);

impl ClusterReplicaStatuses {
    pub(crate) fn new() -> ClusterReplicaStatuses {
        ClusterReplicaStatuses(BTreeMap::new())
    }

    /// Initializes the statuses of the specified cluster.
    ///
    /// Panics if the cluster statuses are already initialized.
    pub(crate) fn initialize_cluster_statuses(&mut self, cluster_id: ClusterId) {
        let prev = self.0.insert(cluster_id, BTreeMap::new());
        assert_eq!(
            prev, None,
            "cluster {cluster_id} statuses already initialized"
        );
    }

    /// Initializes the statuses of the specified cluster replica.
    ///
    /// Panics if the cluster replica statuses are already initialized.
    pub(crate) fn initialize_cluster_replica_statuses(
        &mut self,
        cluster_id: ClusterId,
        replica_id: ReplicaId,
        num_processes: usize,
        time: DateTime<Utc>,
    ) {
        tracing::info!(
            ?cluster_id,
            ?replica_id,
            ?time,
            "initializing cluster replica status"
        );
        let replica_statuses = self.0.entry(cluster_id).or_default();
        let process_statuses = (0..num_processes)
            .map(|process_id| {
                let status = ClusterReplicaProcessStatus {
                    status: ClusterStatus::Offline(Some(OfflineReason::Initializing)),
                    time: time.clone(),
                };
                (u64::cast_from(process_id), status)
            })
            .collect();
        let prev = replica_statuses.insert(replica_id, process_statuses);
        assert_none!(
            prev,
            "cluster replica {cluster_id}.{replica_id} statuses already initialized"
        );
    }

    /// Removes the statuses of the specified cluster.
    ///
    /// Panics if the cluster does not exist.
    pub(crate) fn remove_cluster_statuses(
        &mut self,
        cluster_id: &ClusterId,
    ) -> BTreeMap<ReplicaId, BTreeMap<ProcessId, ClusterReplicaProcessStatus>> {
        let prev = self.0.remove(cluster_id);
        prev.unwrap_or_else(|| panic!("unknown cluster: {cluster_id}"))
    }

    /// Removes the statuses of the specified cluster replica.
    ///
    /// Panics if the cluster or replica does not exist.
    pub(crate) fn remove_cluster_replica_statuses(
        &mut self,
        cluster_id: &ClusterId,
        replica_id: &ReplicaId,
    ) -> BTreeMap<ProcessId, ClusterReplicaProcessStatus> {
        let replica_statuses = self
            .0
            .get_mut(cluster_id)
            .unwrap_or_else(|| panic!("unknown cluster: {cluster_id}"));
        let prev = replica_statuses.remove(replica_id);
        prev.unwrap_or_else(|| panic!("unknown cluster replica: {cluster_id}.{replica_id}"))
    }

    /// Inserts or updates the status of the specified cluster replica process.
    ///
    /// Panics if the cluster or replica does not exist.
    pub(crate) fn ensure_cluster_status(
        &mut self,
        cluster_id: ClusterId,
        replica_id: ReplicaId,
        process_id: ProcessId,
        status: ClusterReplicaProcessStatus,
    ) {
        let replica_statuses = self
            .0
            .get_mut(&cluster_id)
            .unwrap_or_else(|| panic!("unknown cluster: {cluster_id}"))
            .get_mut(&replica_id)
            .unwrap_or_else(|| panic!("unknown cluster replica: {cluster_id}.{replica_id}"));
        replica_statuses.insert(process_id, status);
    }

    /// Computes the status of the cluster replica as a whole.
    ///
    /// Panics if `cluster_id` or `replica_id` don't exist.
    pub fn get_cluster_replica_status(
        &self,
        cluster_id: ClusterId,
        replica_id: ReplicaId,
    ) -> ClusterStatus {
        let process_status = self.get_cluster_replica_statuses(cluster_id, replica_id);
        Self::cluster_replica_status(process_status)
    }

    /// Computes the status of the cluster replica as a whole.
    pub fn cluster_replica_status(
        process_status: &BTreeMap<ProcessId, ClusterReplicaProcessStatus>,
    ) -> ClusterStatus {
        process_status
            .values()
            .fold(ClusterStatus::Online, |s, p| match (s, p.status) {
                (ClusterStatus::Online, ClusterStatus::Online) => ClusterStatus::Online,
                (x, y) => {
                    let reason_x = match x {
                        ClusterStatus::Offline(reason) => reason,
                        ClusterStatus::Online => None,
                    };
                    let reason_y = match y {
                        ClusterStatus::Offline(reason) => reason,
                        ClusterStatus::Online => None,
                    };
                    // Arbitrarily pick the first known not-ready reason.
                    ClusterStatus::Offline(reason_x.or(reason_y))
                }
            })
    }

    /// Gets the statuses of the given cluster replica.
    ///
    /// Panics if the cluster or replica does not exist
    pub(crate) fn get_cluster_replica_statuses(
        &self,
        cluster_id: ClusterId,
        replica_id: ReplicaId,
    ) -> &BTreeMap<ProcessId, ClusterReplicaProcessStatus> {
        self.try_get_cluster_replica_statuses(cluster_id, replica_id)
            .unwrap_or_else(|| panic!("unknown cluster replica: {cluster_id}.{replica_id}"))
    }

    /// Gets the statuses of the given cluster replica.
    pub(crate) fn try_get_cluster_replica_statuses(
        &self,
        cluster_id: ClusterId,
        replica_id: ReplicaId,
    ) -> Option<&BTreeMap<ProcessId, ClusterReplicaProcessStatus>> {
        self.try_get_cluster_statuses(cluster_id)
            .and_then(|statuses| statuses.get(&replica_id))
    }

    /// Gets the statuses of the given cluster.
    ///
    /// Panics if the cluster does not exist
    pub(crate) fn get_cluster_statuses(
        &self,
        cluster_id: ClusterId,
    ) -> &BTreeMap<ReplicaId, BTreeMap<ProcessId, ClusterReplicaProcessStatus>> {
        self.try_get_cluster_statuses(cluster_id)
            .unwrap_or_else(|| panic!("unknown cluster: {cluster_id}"))
    }

    /// Gets the statuses of the given cluster.
    pub(crate) fn try_get_cluster_statuses(
        &self,
        cluster_id: ClusterId,
    ) -> Option<&BTreeMap<ReplicaId, BTreeMap<ProcessId, ClusterReplicaProcessStatus>>> {
        self.0.get(&cluster_id)
    }
}

/// Glues the external world to the Timely workers.
#[derive(Derivative)]
#[derivative(Debug)]
pub struct Coordinator {
    /// The controller for the storage and compute layers.
    #[derivative(Debug = "ignore")]
    controller: mz_controller::Controller,
    /// The catalog in an Arc suitable for readonly references. The Arc allows
    /// us to hand out cheap copies of the catalog to functions that can use it
    /// off of the main coordinator thread. If the coordinator needs to mutate
    /// the catalog, call [`Self::catalog_mut`], which will clone this struct member,
    /// allowing it to be mutated here while the other off-thread references can
    /// read their catalog as long as needed. In the future we would like this
    /// to be a pTVC, but for now this is sufficient.
    catalog: Arc<Catalog>,

    /// Channel to manage internal commands from the coordinator to itself.
    internal_cmd_tx: mpsc::UnboundedSender<Message>,
    /// Notification that triggers a group commit.
    group_commit_tx: appends::GroupCommitNotifier,

    /// Channel for strict serializable reads ready to commit.
    strict_serializable_reads_tx: mpsc::UnboundedSender<(ConnectionId, PendingReadTxn)>,

    /// Mechanism for totally ordering write and read timestamps, so that all reads
    /// reflect exactly the set of writes that precede them, and no writes that follow.
    global_timelines: BTreeMap<Timeline, TimelineState<Timestamp>>,

    /// A generator for transient [`GlobalId`]s, shareable with other threads.
    transient_id_gen: Arc<TransientIdGen>,
    /// A map from connection ID to metadata about that connection for all
    /// active connections.
    active_conns: BTreeMap<ConnectionId, ConnMeta>,

    /// For each transaction, the read holds taken to support any performed reads.
    ///
    /// Upon completing a transaction, these read holds should be dropped.
    txn_read_holds: BTreeMap<ConnectionId, read_policy::ReadHolds<Timestamp>>,

    /// Access to the peek fields should be restricted to methods in the [`peek`] API.
    /// A map from pending peek ids to the queue into which responses are sent, and
    /// the connection id of the client that initiated the peek.
    pending_peeks: BTreeMap<Uuid, PendingPeek>,
    /// A map from client connection ids to a set of all pending peeks for that client.
    client_pending_peeks: BTreeMap<ConnectionId, BTreeMap<Uuid, ClusterId>>,

    /// A map from client connection ids to pending linearize read transaction.
    pending_linearize_read_txns: BTreeMap<ConnectionId, PendingReadTxn>,

    /// A map from the compute sink ID to it's state description.
    active_compute_sinks: BTreeMap<GlobalId, ActiveComputeSink>,
    /// A map from active webhooks to their invalidation handle.
    active_webhooks: BTreeMap<CatalogItemId, WebhookAppenderInvalidator>,
    /// A map from connection ids to a watch channel that is set to `true` if the connection
    /// received a cancel request.
    staged_cancellation: BTreeMap<ConnectionId, (watch::Sender<bool>, watch::Receiver<bool>)>,
    /// Active introspection subscribes.
    introspection_subscribes: BTreeMap<GlobalId, IntrospectionSubscribe>,

    /// Locks that grant access to a specific object, populated lazily as objects are written to.
    write_locks: BTreeMap<CatalogItemId, Arc<tokio::sync::Mutex<()>>>,
    /// Plans that are currently deferred and waiting on a write lock.
    deferred_write_ops: BTreeMap<ConnectionId, DeferredWriteOp>,

    /// Pending writes waiting for a group commit.
    pending_writes: Vec<PendingWriteTxn>,

    /// For the realtime timeline, 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 X`, SUBSCRIBing 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 behavior
    /// of realtime sources that close off timestamps on an interval.
    ///
    /// For non-realtime timelines, nothing pushes the timestamps forward, so we must do
    /// it manually.
    advance_timelines_interval: Interval,

    /// Serialized DDL. DDL must be serialized because:
    /// - Many of them do off-thread work and need to verify the catalog is in a valid state, but
    ///   [`PlanValidity`] does not currently support tracking all changes. Doing that correctly
    ///   seems to be more difficult than it's worth, so we would instead re-plan and re-sequence
    ///   the statements.
    /// - Re-planning a statement is hard because Coordinator and Session state is mutated at
    ///   various points, and we would need to correctly reset those changes before re-planning and
    ///   re-sequencing.
    serialized_ddl: LockedVecDeque<DeferredPlanStatement>,

    /// Handle to secret manager that can create and delete secrets from
    /// an arbitrary secret storage engine.
    secrets_controller: Arc<dyn SecretsController>,
    /// A secrets reader than maintains an in-memory cache, where values have a set TTL.
    caching_secrets_reader: CachingSecretsReader,

    /// Handle to a manager that can create and delete kubernetes resources
    /// (ie: VpcEndpoint objects)
    cloud_resource_controller: Option<Arc<dyn CloudResourceController>>,

    /// Metadata about replicas that doesn't need to be persisted.
    /// Intended for inclusion in system tables.
    ///
    /// `None` is used as a tombstone value for replicas that have been
    /// dropped and for which no further updates should be recorded.
    transient_replica_metadata: BTreeMap<ReplicaId, Option<ReplicaMetadata>>,

    /// Persist client for fetching storage metadata such as size metrics.
    storage_usage_client: StorageUsageClient,
    /// The interval at which to collect storage usage information.
    storage_usage_collection_interval: Duration,

    /// Segment analytics client.
    #[derivative(Debug = "ignore")]
    segment_client: Option<mz_segment::Client>,

    /// Coordinator metrics.
    metrics: Metrics,
    /// Optimizer metrics.
    optimizer_metrics: OptimizerMetrics,

    /// Tracing handle.
    tracing_handle: TracingHandle,

    /// Data used by the statement logging feature.
    statement_logging: StatementLogging,

    /// Limit for how many concurrent webhook requests we allow.
    webhook_concurrency_limit: WebhookConcurrencyLimiter,

    /// Optional config for the Postgres-backed timestamp oracle. This is
    /// _required_ when `postgres` is configured using the `timestamp_oracle`
    /// system variable.
    pg_timestamp_oracle_config: Option<PostgresTimestampOracleConfig>,

    /// Periodically asks cluster scheduling policies to make their decisions.
    check_cluster_scheduling_policies_interval: Interval,

    /// This keeps the last On/Off decision for each cluster and each scheduling policy.
    /// (Clusters that have been dropped or are otherwise out of scope for automatic scheduling are
    /// periodically cleaned up from this Map.)
    cluster_scheduling_decisions: BTreeMap<ClusterId, BTreeMap<&'static str, SchedulingDecision>>,

    /// When doing 0dt upgrades/in read-only mode, periodically ask all known
    /// clusters/collections whether they are caught up.
    caught_up_check_interval: Interval,

    /// Context needed to check whether all clusters/collections have caught up.
    /// Only used during 0dt deployment, while in read-only mode.
    caught_up_check: Option<CaughtUpCheckContext>,

    /// Tracks the state associated with the currently installed watchsets.
    installed_watch_sets: BTreeMap<WatchSetId, (ConnectionId, WatchSetResponse)>,

    /// Tracks the currently installed watchsets for each connection.
    connection_watch_sets: BTreeMap<ConnectionId, BTreeSet<WatchSetId>>,

    /// Tracks the statuses of all cluster replicas.
    cluster_replica_statuses: ClusterReplicaStatuses,

    /// Whether or not to start controllers in read-only mode. This is only
    /// meant for use during development of read-only clusters and 0dt upgrades
    /// and should go away once we have proper orchestration during upgrades.
    read_only_controllers: bool,

    /// Updates to builtin tables that are being buffered while we are in
    /// read-only mode. We apply these all at once when coming out of read-only
    /// mode.
    ///
    /// This is a `Some` while in read-only mode and will be replaced by a
    /// `None` when we transition out of read-only mode and write out any
    /// buffered updates.
    buffered_builtin_table_updates: Option<Vec<BuiltinTableUpdate>>,
}

impl Coordinator {
    /// Initializes coordinator state based on the contained catalog. Must be
    /// called after creating the coordinator and before calling the
    /// `Coordinator::serve` method.
    #[instrument(name = "coord::bootstrap")]
    pub(crate) async fn bootstrap(
        &mut self,
        boot_ts: Timestamp,
        migrated_storage_collections_0dt: BTreeSet<CatalogItemId>,
        mut builtin_table_updates: Vec<BuiltinTableUpdate>,
        cached_global_exprs: BTreeMap<GlobalId, GlobalExpressions>,
        uncached_local_exprs: BTreeMap<GlobalId, LocalExpressions>,
    ) -> Result<(), AdapterError> {
        let bootstrap_start = Instant::now();
        info!("startup: coordinator init: bootstrap beginning");
        info!("startup: coordinator init: bootstrap: preamble beginning");

        // Initialize cluster replica statuses.
        // Gross iterator is to avoid partial borrow issues.
        let cluster_statuses: Vec<(_, Vec<_>)> = self
            .catalog()
            .clusters()
            .map(|cluster| {
                (
                    cluster.id(),
                    cluster
                        .replicas()
                        .map(|replica| {
                            (replica.replica_id, replica.config.location.num_processes())
                        })
                        .collect(),
                )
            })
            .collect();
        let now = self.now_datetime();
        for (cluster_id, replica_statuses) in cluster_statuses {
            self.cluster_replica_statuses
                .initialize_cluster_statuses(cluster_id);
            for (replica_id, num_processes) in replica_statuses {
                self.cluster_replica_statuses
                    .initialize_cluster_replica_statuses(
                        cluster_id,
                        replica_id,
                        num_processes,
                        now,
                    );
            }
        }
        for replica_statuses in self.cluster_replica_statuses.0.values() {
            for (replica_id, processes_statuses) in replica_statuses {
                for (process_id, status) in processes_statuses {
                    let builtin_table_update = self
                        .catalog()
                        .state()
                        .pack_cluster_replica_status_update(*replica_id, *process_id, status, 1);
                    let builtin_table_update = self
                        .catalog()
                        .state()
                        .resolve_builtin_table_update(builtin_table_update);
                    builtin_table_updates.push(builtin_table_update);
                }
            }
        }

        // Inform the controllers about their initial configuration.
        let system_config = self.catalog().system_config();
        let compute_config = flags::compute_config(system_config);
        let storage_config = flags::storage_config(system_config);
        let scheduling_config = flags::orchestrator_scheduling_config(system_config);
        let exert_prop = system_config.arrangement_exert_proportionality();
        self.controller.compute.update_configuration(compute_config);
        self.controller.storage.update_parameters(storage_config);
        self.controller
            .update_orchestrator_scheduling_config(scheduling_config);
        self.controller
            .set_arrangement_exert_proportionality(exert_prop);

        let mut policies_to_set: BTreeMap<CompactionWindow, CollectionIdBundle> =
            Default::default();

        let enable_worker_core_affinity =
            self.catalog().system_config().enable_worker_core_affinity();
        for instance in self.catalog.clusters() {
            self.controller.create_cluster(
                instance.id,
                ClusterConfig {
                    arranged_logs: instance.log_indexes.clone(),
                    workload_class: instance.config.workload_class.clone(),
                },
            )?;
            for replica in instance.replicas() {
                let role = instance.role();
                self.controller.create_replica(
                    instance.id,
                    replica.replica_id,
                    role,
                    replica.config.clone(),
                    enable_worker_core_affinity,
                )?;
            }
        }

        info!(
            "startup: coordinator init: bootstrap: preamble complete in {:?}",
            bootstrap_start.elapsed()
        );

        let init_storage_collections_start = Instant::now();
        info!("startup: coordinator init: bootstrap: storage collections init beginning");
        self.bootstrap_storage_collections(&migrated_storage_collections_0dt)
            .await;
        info!(
            "startup: coordinator init: bootstrap: storage collections init complete in {:?}",
            init_storage_collections_start.elapsed()
        );

        let optimize_dataflows_start = Instant::now();
        info!("startup: coordinator init: bootstrap: optimize dataflow plans beginning");
        let entries: Vec<_> = self.catalog().entries().cloned().collect();
        let uncached_global_exps = self.bootstrap_dataflow_plans(&entries, cached_global_exprs)?;
        info!(
            "startup: coordinator init: bootstrap: optimize dataflow plans complete in {:?}",
            optimize_dataflows_start.elapsed()
        );

        // We don't need to wait for the cache to update.
        let _fut = self.catalog().update_expression_cache(
            uncached_local_exprs.into_iter().collect(),
            uncached_global_exps.into_iter().collect(),
        );

        // Select dataflow as-ofs. This step relies on the storage collections created by
        // `bootstrap_storage_collections` and the dataflow plans created by
        // `bootstrap_dataflow_plans`.
        let bootstrap_as_ofs_start = Instant::now();
        info!("startup: coordinator init: bootstrap: dataflow as-of bootstrapping beginning");
        let dataflow_read_holds = self.bootstrap_dataflow_as_ofs().await;
        info!(
            "startup: coordinator init: bootstrap: dataflow as-of bootstrapping complete in {:?}",
            bootstrap_as_ofs_start.elapsed()
        );

        let postamble_start = Instant::now();
        info!("startup: coordinator init: bootstrap: postamble beginning");

        let logs: BTreeSet<_> = BUILTINS::logs()
            .map(|log| self.catalog().resolve_builtin_log(log))
            .flat_map(|item_id| self.catalog().get_global_ids(&item_id))
            .collect();

        let mut privatelink_connections = BTreeMap::new();

        for entry in &entries {
            // TODO(database-issues#7922): we should move this invariant into `CatalogEntry`.
            mz_ore::soft_assert_or_log!(
                // We only expect user objects to objects obey this invariant.
                // System objects, for instance, can depend on other system
                // objects that belong to a schema that is simply loaded first.
                // To meaningfully resolve this, we could need more careful
                // loading order or more complex IDs, neither of which seem very
                // beneficial.
                //
                // HACK: sinks are permitted to depend on items with larger IDs,
                // due to `ALTER SINK`.
                !entry.id().is_user()
                    || entry.is_sink()
                    || entry
                        .uses()
                        .iter()
                        .all(|dependency_id| *dependency_id <= entry.id),
                "entries should only use to items with lesser `GlobalId`s, but \
                {:?} uses {:?}",
                entry.id,
                entry.uses()
            );

            debug!(
                "coordinator init: installing {} {}",
                entry.item().typ(),
                entry.id()
            );
            let mut policy = entry.item().initial_logical_compaction_window();
            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) => {
                    // Propagate source compaction windows to subsources if needed.
                    if source.custom_logical_compaction_window.is_none() {
                        if let DataSourceDesc::IngestionExport { ingestion_id, .. } =
                            source.data_source
                        {
                            policy = Some(
                                self.catalog()
                                    .get_entry(&ingestion_id)
                                    .source()
                                    .expect("must be source")
                                    .custom_logical_compaction_window
                                    .unwrap_or_default(),
                            );
                        }
                    }
                    policies_to_set
                        .entry(policy.expect("sources have a compaction window"))
                        .or_insert_with(Default::default)
                        .storage_ids
                        .insert(source.global_id());
                }
                CatalogItem::Table(table) => {
                    policies_to_set
                        .entry(policy.expect("tables have a compaction window"))
                        .or_insert_with(Default::default)
                        .storage_ids
                        .extend(table.global_ids());
                }
                CatalogItem::Index(idx) => {
                    let policy_entry = policies_to_set
                        .entry(policy.expect("indexes have a compaction window"))
                        .or_insert_with(Default::default);

                    if logs.contains(&idx.on) {
                        policy_entry
                            .compute_ids
                            .entry(idx.cluster_id)
                            .or_insert_with(BTreeSet::new)
                            .insert(idx.global_id());
                    } else {
                        let df_desc = self
                            .catalog()
                            .try_get_physical_plan(&idx.global_id())
                            .expect("added in `bootstrap_dataflow_plans`")
                            .clone();

                        let df_meta = self
                            .catalog()
                            .try_get_dataflow_metainfo(&idx.global_id())
                            .expect("added in `bootstrap_dataflow_plans`");

                        if self.catalog().state().system_config().enable_mz_notices() {
                            // Collect optimization hint updates.
                            self.catalog().state().pack_optimizer_notices(
                                &mut builtin_table_updates,
                                df_meta.optimizer_notices.iter(),
                                1,
                            );
                        }

                        // What follows is morally equivalent to `self.ship_dataflow(df, idx.cluster_id)`,
                        // but we cannot call that as it will also downgrade the read hold on the index.
                        policy_entry
                            .compute_ids
                            .entry(idx.cluster_id)
                            .or_insert_with(Default::default)
                            .extend(df_desc.export_ids());

                        self.controller
                            .compute
                            .create_dataflow(idx.cluster_id, df_desc, None)
                            .unwrap_or_terminate("cannot fail to create dataflows");
                    }
                }
                CatalogItem::View(_) => (),
                CatalogItem::MaterializedView(mview) => {
                    policies_to_set
                        .entry(policy.expect("materialized views have a compaction window"))
                        .or_insert_with(Default::default)
                        .storage_ids
                        .insert(mview.global_id());

                    let mut df_desc = self
                        .catalog()
                        .try_get_physical_plan(&mview.global_id())
                        .expect("added in `bootstrap_dataflow_plans`")
                        .clone();

                    if let Some(initial_as_of) = mview.initial_as_of.clone() {
                        df_desc.set_initial_as_of(initial_as_of);
                    }

                    // If we have a refresh schedule that has a last refresh, then set the `until` to the last refresh.
                    let until = mview
                        .refresh_schedule
                        .as_ref()
                        .and_then(|s| s.last_refresh())
                        .and_then(|r| r.try_step_forward());
                    if let Some(until) = until {
                        df_desc.until.meet_assign(&Antichain::from_elem(until));
                    }

                    let df_meta = self
                        .catalog()
                        .try_get_dataflow_metainfo(&mview.global_id())
                        .expect("added in `bootstrap_dataflow_plans`");

                    if self.catalog().state().system_config().enable_mz_notices() {
                        // Collect optimization hint updates.
                        self.catalog().state().pack_optimizer_notices(
                            &mut builtin_table_updates,
                            df_meta.optimizer_notices.iter(),
                            1,
                        );
                    }

                    self.ship_dataflow(df_desc, mview.cluster_id, None).await;
                }
                CatalogItem::Sink(sink) => {
                    self.create_storage_export(sink.global_id(), sink)
                        .await
                        .unwrap_or_terminate("cannot fail to create exports");
                }
                CatalogItem::Connection(catalog_connection) => {
                    if let ConnectionDetails::AwsPrivatelink(conn) = &catalog_connection.details {
                        privatelink_connections.insert(
                            entry.id(),
                            VpcEndpointConfig {
                                aws_service_name: conn.service_name.clone(),
                                availability_zone_ids: conn.availability_zones.clone(),
                            },
                        );
                    }
                }
                CatalogItem::ContinualTask(ct) => {
                    policies_to_set
                        .entry(policy.expect("continual tasks have a compaction window"))
                        .or_insert_with(Default::default)
                        .storage_ids
                        .insert(ct.global_id());

                    let mut df_desc = self
                        .catalog()
                        .try_get_physical_plan(&ct.global_id())
                        .expect("added in `bootstrap_dataflow_plans`")
                        .clone();

                    if let Some(initial_as_of) = ct.initial_as_of.clone() {
                        df_desc.set_initial_as_of(initial_as_of);
                    }

                    let df_meta = self
                        .catalog()
                        .try_get_dataflow_metainfo(&ct.global_id())
                        .expect("added in `bootstrap_dataflow_plans`");

                    if self.catalog().state().system_config().enable_mz_notices() {
                        // Collect optimization hint updates.
                        self.catalog().state().pack_optimizer_notices(
                            &mut builtin_table_updates,
                            df_meta.optimizer_notices.iter(),
                            1,
                        );
                    }

                    self.ship_dataflow(df_desc, ct.cluster_id, None).await;
                }
                // Nothing to do for these cases
                CatalogItem::Log(_)
                | CatalogItem::Type(_)
                | CatalogItem::Func(_)
                | CatalogItem::Secret(_) => {}
            }
        }

        if let Some(cloud_resource_controller) = &self.cloud_resource_controller {
            // Clean up any extraneous VpcEndpoints that shouldn't exist.
            let existing_vpc_endpoints = cloud_resource_controller.list_vpc_endpoints().await?;
            let existing_vpc_endpoints = BTreeSet::from_iter(existing_vpc_endpoints.into_keys());
            let desired_vpc_endpoints = privatelink_connections.keys().cloned().collect();
            let vpc_endpoints_to_remove = existing_vpc_endpoints.difference(&desired_vpc_endpoints);
            for id in vpc_endpoints_to_remove {
                cloud_resource_controller.delete_vpc_endpoint(*id).await?;
            }

            // Ensure desired VpcEndpoints are up to date.
            for (id, spec) in privatelink_connections {
                cloud_resource_controller
                    .ensure_vpc_endpoint(id, spec)
                    .await?;
            }
        }

        // Having installed all entries, creating all constraints, we can now drop read holds and
        // relax read policies.
        drop(dataflow_read_holds);
        // TODO -- Improve `initialize_read_policies` API so we can avoid calling this in a loop.
        for (cw, policies) in policies_to_set {
            self.initialize_read_policies(&policies, cw).await;
        }

        // Expose mapping from T-shirt sizes to actual sizes
        builtin_table_updates.extend(
            self.catalog().state().resolve_builtin_table_updates(
                self.catalog().state().pack_all_replica_size_updates(),
            ),
        );

        debug!("startup: coordinator init: bootstrap: initializing migrated builtin tables");
        // When 0dt is enabled, we create new shards for any migrated builtin storage collections.
        // In read-only mode, the migrated builtin tables (which are a subset of migrated builtin
        // storage collections) need to be back-filled so that any dependent dataflow can be
        // hydrated. Additionally, these shards are not registered with the txn-shard, and cannot
        // be registered while in read-only, so they are written to directly.
        let migrated_updates_fut = if self.controller.read_only() {
            let min_timestamp = Timestamp::minimum();
            let migrated_builtin_table_updates: Vec<_> = builtin_table_updates
                .drain_filter_swapping(|update| {
                    let gid = self.catalog().get_entry(&update.id).latest_global_id();
                    migrated_storage_collections_0dt.contains(&update.id)
                        && self
                            .controller
                            .storage_collections
                            .collection_frontiers(gid)
                            .expect("all tables are registered")
                            .write_frontier
                            .elements()
                            == &[min_timestamp]
                })
                .collect();
            if migrated_builtin_table_updates.is_empty() {
                futures::future::ready(()).boxed()
            } else {
                let mut appends: BTreeMap<GlobalId, Vec<(Row, Diff)>> = BTreeMap::new();
                for update in migrated_builtin_table_updates {
                    let gid = self.catalog().get_entry(&update.id).latest_global_id();
                    appends
                        .entry(gid)
                        .or_default()
                        .push((update.row, update.diff));
                }
                for (_, updates) in &mut appends {
                    differential_dataflow::consolidation::consolidate(updates);
                }
                info!(
                    "coordinator init: rehydrating migrated builtin tables in read-only mode: {:?}",
                    appends.keys().collect::<Vec<_>>()
                );
                let appends = appends
                    .into_iter()
                    .map(|(id, updates)| {
                        let updates = updates
                            .into_iter()
                            .map(|(row, diff)| TimestamplessUpdate { row, diff })
                            .collect();
                        (id, updates)
                    })
                    .collect();
                let fut = self
                    .controller
                    .storage
                    .append_table(min_timestamp, boot_ts.step_forward(), appends)
                    .expect("cannot fail to append");
                async {
                    fut.await
                        .expect("One-shot shouldn't be dropped during bootstrap")
                        .unwrap_or_terminate("cannot fail to append")
                }
                .boxed()
            }
        } else {
            futures::future::ready(()).boxed()
        };

        info!(
            "startup: coordinator init: bootstrap: postamble complete in {:?}",
            postamble_start.elapsed()
        );

        let builtin_update_start = Instant::now();
        info!("startup: coordinator init: bootstrap: generate builtin updates beginning");
        if self.controller.read_only() {
            info!("coordinator init: bootstrap: stashing builtin table updates while in read-only mode");

            self.buffered_builtin_table_updates
                .as_mut()
                .expect("in read-only mode")
                .append(&mut builtin_table_updates);
        } else {
            self.bootstrap_tables(&entries, builtin_table_updates).await;
        };
        info!(
            "startup: coordinator init: bootstrap: generate builtin updates complete in {:?}",
            builtin_update_start.elapsed()
        );

        let cleanup_secrets_start = Instant::now();
        info!("startup: coordinator init: bootstrap: generate secret cleanup beginning");
        // Cleanup orphaned secrets. Errors during list() or delete() do not
        // need to prevent bootstrap from succeeding; we will retry next
        // startup.
        {
            // Destructure Self so we can selectively move fields into the async
            // task.
            let Self {
                secrets_controller,
                catalog,
                ..
            } = self;

            let next_user_item_id = catalog.get_next_user_item_id().await?;
            let next_system_item_id = catalog.get_next_system_item_id().await?;
            let read_only = self.controller.read_only();
            // Fetch all IDs from the catalog to future-proof against other
            // things using secrets. Today, SECRET and CONNECTION objects use
            // secrets_controller.ensure, but more things could in the future
            // that would be easy to miss adding here.
            let catalog_ids: BTreeSet<CatalogItemId> =
                catalog.entries().map(|entry| entry.id()).collect();
            let secrets_controller = Arc::clone(secrets_controller);

            spawn(|| "cleanup-orphaned-secrets", async move {
                if read_only {
                    info!("coordinator init: not cleaning up orphaned secrets while in read-only mode");
                    return;
                }
                info!("coordinator init: cleaning up orphaned secrets");

                match secrets_controller.list().await {
                    Ok(controller_secrets) => {
                        let controller_secrets: BTreeSet<CatalogItemId> =
                            controller_secrets.into_iter().collect();
                        let orphaned = controller_secrets.difference(&catalog_ids);
                        for id in orphaned {
                            let id_too_large = match id {
                                CatalogItemId::System(id) => *id >= next_system_item_id,
                                CatalogItemId::User(id) => *id >= next_user_item_id,
                                CatalogItemId::Transient(_) => false,
                            };
                            if id_too_large {
                                info!(
                                    %next_user_item_id, %next_system_item_id,
                                    "coordinator init: not deleting orphaned secret {id} that was likely created by a newer deploy generation"
                                );
                            } else {
                                info!("coordinator init: deleting orphaned secret {id}");
                                fail_point!("orphan_secrets");
                                if let Err(e) = secrets_controller.delete(*id).await {
                                    warn!(
                                        "Dropping orphaned secret has encountered an error: {}",
                                        e
                                    );
                                }
                            }
                        }
                    }
                    Err(e) => warn!("Failed to list secrets during orphan cleanup: {:?}", e),
                }
            });
        }
        info!(
            "startup: coordinator init: bootstrap: generate secret cleanup complete in {:?}",
            cleanup_secrets_start.elapsed()
        );

        // Run all of our final steps concurrently.
        let final_steps_start = Instant::now();
        info!(
            "startup: coordinator init: bootstrap: migrate builtin tables in read-only mode beginning"
        );
        migrated_updates_fut
            .instrument(info_span!("coord::bootstrap::final"))
            .await;

        debug!("startup: coordinator init: bootstrap: announcing completion of initialization to controller");
        // Announce the completion of initialization.
        self.controller.initialization_complete();

        // Initialize unified introspection.
        self.bootstrap_introspection_subscribes().await;

        info!(
            "startup: coordinator init: bootstrap: migrate builtin tables in read-only mode complete in {:?}", final_steps_start.elapsed()
        );

        info!(
            "startup: coordinator init: bootstrap complete in {:?}",
            bootstrap_start.elapsed()
        );
        Ok(())
    }

    /// Prepares tables for writing by resetting them to a known state and
    /// appending the given builtin table updates. The timestamp oracle
    /// will be advanced to the write timestamp of the append when this
    /// method returns.
    #[allow(clippy::async_yields_async)]
    #[instrument]
    async fn bootstrap_tables(
        &mut self,
        entries: &[CatalogEntry],
        mut builtin_table_updates: Vec<BuiltinTableUpdate>,
    ) {
        /// Smaller helper struct of metadata for bootstrapping tables.
        struct TableMetadata<'a> {
            id: CatalogItemId,
            name: &'a QualifiedItemName,
            table: &'a Table,
        }

        // Filter our entries down to just tables.
        let table_metas: Vec<_> = entries
            .into_iter()
            .filter_map(|entry| {
                entry.table().map(|table| TableMetadata {
                    id: entry.id(),
                    name: entry.name(),
                    table,
                })
            })
            .collect();

        // Append empty batches to advance the timestamp of all tables.
        debug!("coordinator init: advancing all tables to current timestamp");
        let WriteTimestamp {
            timestamp: write_ts,
            advance_to,
        } = self.get_local_write_ts().await;
        let appends = table_metas
            .iter()
            .map(|meta| (meta.table.global_id_writes(), Vec::new()))
            .collect();
        // Append the tables in the background. We apply the write timestamp before getting a read
        // timestamp and reading a snapshot of each table, so the snapshots will block on their own
        // until the appends are complete.
        let table_fence_rx = self
            .controller
            .storage
            .append_table(write_ts.clone(), advance_to, appends)
            .expect("invalid updates");

        self.apply_local_write(write_ts).await;

        // Add builtin table updates the clear the contents of all system tables
        debug!("coordinator init: resetting system tables");
        let read_ts = self.get_local_read_ts().await;

        // Filter out the 'mz_storage_usage_by_shard' table since we need to retain that info for
        // billing purposes.
        let mz_storage_usage_by_shard_schema: SchemaSpecifier = self
            .catalog()
            .resolve_system_schema(MZ_STORAGE_USAGE_BY_SHARD.schema)
            .into();
        let is_storage_usage_by_shard = |meta: &TableMetadata| -> bool {
            meta.name.item == MZ_STORAGE_USAGE_BY_SHARD.name
                && meta.name.qualifiers.schema_spec == mz_storage_usage_by_shard_schema
        };

        let mut retraction_tasks = Vec::new();
        let system_tables = table_metas
            .iter()
            .filter(|meta| meta.id.is_system() && !is_storage_usage_by_shard(meta));

        for system_table in system_tables {
            let table_id = system_table.id;
            let full_name = self.catalog().resolve_full_name(system_table.name, None);
            debug!("coordinator init: resetting system table {full_name} ({table_id})");

            // Fetch the current contents of the table for retraction.
            let current_contents_fut = self
                .controller
                .storage
                .snapshot(system_table.table.global_id_writes(), read_ts);
            // Fetch a snapshot of the current tables concurrently.
            let task = spawn(|| format!("snapshot-{table_id}"), async move {
                let current_contents = current_contents_fut
                    .await
                    .unwrap_or_terminate("cannot fail to fetch snapshot");
                let contents_len = current_contents.len();
                debug!("coordinator init: table ({table_id}) size {contents_len}",);

                // Retract the current contents.
                current_contents
                    .into_iter()
                    .map(|(row, diff)| BuiltinTableUpdate {
                        id: table_id,
                        row,
                        diff: diff.neg(),
                    })
                    .collect::<Vec<_>>()
            });
            retraction_tasks.push(task);
        }

        let retractions_res = futures::future::join_all(retraction_tasks).await;
        for retractions in retractions_res {
            let retractions = retractions.expect("cannot fail to fetch snapshot");
            builtin_table_updates.extend(retractions);
        }

        // Now that the snapshots are complete, the appends must also be complete.
        table_fence_rx
            .await
            .expect("One-shot shouldn't be dropped during bootstrap")
            .unwrap_or_terminate("cannot fail to append");

        debug!("coordinator init: sending builtin table updates");
        let (_builtin_updates_fut, write_ts) = self
            .builtin_table_update()
            .execute(builtin_table_updates)
            .await;
        if let Some(write_ts) = write_ts {
            self.apply_local_write(write_ts).await;
        }
    }

    /// Initializes all storage collections required by catalog objects in the storage controller.
    ///
    /// This method takes care of collection creation, as well as migration of existing
    /// collections.
    ///
    /// Creating all storage collections in a single `create_collections` call, rather than on
    /// demand, is more efficient as it reduces the number of writes to durable storage. It also
    /// allows subsequent bootstrap logic to fetch metadata (such as frontiers) of arbitrary
    /// storage collections, without needing to worry about dependency order.
    ///
    /// `migrated_storage_collections` is a set of builtin storage collections that have been
    /// migrated and should be handled specially.
    #[instrument]
    async fn bootstrap_storage_collections(
        &mut self,
        migrated_storage_collections: &BTreeSet<CatalogItemId>,
    ) {
        let catalog = self.catalog();
        let source_status_collection_id = catalog
            .resolve_builtin_storage_collection(&mz_catalog::builtin::MZ_SOURCE_STATUS_HISTORY);
        let source_status_collection_id = catalog
            .get_entry(&source_status_collection_id)
            .latest_global_id();

        let source_desc =
            |data_source: &DataSourceDesc, desc: &RelationDesc, timeline: &Timeline| {
                let (data_source, status_collection_id) = match data_source.clone() {
                    // Re-announce the source description.
                    DataSourceDesc::Ingestion {
                        ingestion_desc:
                            mz_sql::plan::Ingestion {
                                desc,
                                progress_subsource,
                            },
                        cluster_id,
                    } => {
                        let desc = desc.into_inline_connection(catalog.state());
                        // TODO(parkmycar): We should probably check the type here, but I'm not sure if
                        // this will always be a Source or a Table.
                        let progress_subsource =
                            catalog.get_entry(&progress_subsource).latest_global_id();
                        let ingestion = mz_storage_types::sources::IngestionDescription::new(
                            desc,
                            cluster_id,
                            progress_subsource,
                        );

                        (
                            DataSource::Ingestion(ingestion.clone()),
                            Some(source_status_collection_id),
                        )
                    }
                    DataSourceDesc::IngestionExport {
                        ingestion_id,
                        external_reference: _,
                        details,
                        data_config,
                    } => {
                        // TODO(parkmycar): We should probably check the type here, but I'm not sure if
                        // this will always be a Source or a Table.
                        let ingestion_id = catalog.get_entry(&ingestion_id).latest_global_id();
                        (
                            DataSource::IngestionExport {
                                ingestion_id,
                                details,
                                data_config: data_config.into_inline_connection(catalog.state()),
                            },
                            Some(source_status_collection_id),
                        )
                    }
                    DataSourceDesc::Webhook { .. } => {
                        (DataSource::Webhook, Some(source_status_collection_id))
                    }
                    DataSourceDesc::Progress => (DataSource::Progress, None),
                    DataSourceDesc::Introspection(introspection) => {
                        (DataSource::Introspection(introspection), None)
                    }
                };
                CollectionDescription {
                    desc: desc.clone(),
                    data_source,
                    since: None,
                    status_collection_id,
                    timeline: Some(timeline.clone()),
                }
            };

        let mut collections = vec![];
        let mut new_builtin_continual_tasks = vec![];
        for entry in catalog.entries() {
            match entry.item() {
                CatalogItem::Source(source) => {
                    collections.push((
                        source.global_id(),
                        source_desc(&source.data_source, &source.desc, &source.timeline),
                    ));
                }
                CatalogItem::Table(table) => {
                    match &table.data_source {
                        TableDataSource::TableWrites { defaults: _ } => {
                            let collections_descs = table.collection_descs().map(|(gid, desc)| {
                                (gid, CollectionDescription::for_table(desc.clone()))
                            });
                            collections.extend(collections_descs);
                        }
                        TableDataSource::DataSource {
                            desc: data_source_desc,
                            timeline,
                        } => {
                            // TODO(alter_table): Support versioning tables that read from sources.
                            soft_assert_eq_or_log!(table.collections.len(), 1);
                            let collection_descs = table.collection_descs().map(|(gid, desc)| {
                                (gid, source_desc(data_source_desc, &desc, timeline))
                            });
                            collections.extend(collection_descs);
                        }
                    };
                }
                CatalogItem::MaterializedView(mv) => {
                    let collection_desc = CollectionDescription {
                        desc: mv.desc.clone(),
                        data_source: DataSource::Other,
                        since: mv.initial_as_of.clone(),
                        status_collection_id: None,
                        timeline: None,
                    };
                    collections.push((mv.global_id(), collection_desc));
                }
                CatalogItem::ContinualTask(ct) => {
                    let collection_desc = CollectionDescription {
                        desc: ct.desc.clone(),
                        data_source: DataSource::Other,
                        since: ct.initial_as_of.clone(),
                        status_collection_id: None,
                        timeline: None,
                    };
                    if ct.global_id().is_system() && collection_desc.since.is_none() {
                        // We need a non-0 since to make as_of selection work. Fill it in below with
                        // the `bootstrap_builtin_continual_tasks` call, which can only be run after
                        // `create_collections_for_bootstrap`.
                        new_builtin_continual_tasks.push((ct.global_id(), collection_desc));
                    } else {
                        collections.push((ct.global_id(), collection_desc));
                    }
                }
                _ => (),
            }
        }

        let register_ts = if self.controller.read_only() {
            self.get_local_read_ts().await
        } else {
            // Getting a write timestamp bumps the write timestamp in the
            // oracle, which we're not allowed in read-only mode.
            self.get_local_write_ts().await.timestamp
        };

        let storage_metadata = self.catalog.state().storage_metadata();
        let migrated_storage_collections = migrated_storage_collections
            .into_iter()
            .flat_map(|item_id| self.catalog.get_entry(item_id).global_ids())
            .collect();

        self.controller
            .storage
            .create_collections_for_bootstrap(
                storage_metadata,
                Some(register_ts),
                collections,
                &migrated_storage_collections,
            )
            .await
            .unwrap_or_terminate("cannot fail to create collections");

        self.bootstrap_builtin_continual_tasks(new_builtin_continual_tasks)
            .await;

        if !self.controller.read_only() {
            self.apply_local_write(register_ts).await;
        }
    }

    /// Make as_of selection happy for builtin CTs. Ideally we'd write the
    /// initial as_of down in the durable catalog, but that's hard because of
    /// boot ordering. Instead, we set the since of the storage collection to
    /// something that's a reasonable lower bound for the as_of. Then, if the
    /// upper is 0, the as_of selection code will allow us to jump it forward to
    /// this since.
    async fn bootstrap_builtin_continual_tasks(
        &mut self,
        // TODO(alter_table): Switch to CatalogItemId.
        mut collections: Vec<(GlobalId, CollectionDescription<Timestamp>)>,
    ) {
        for (id, collection) in &mut collections {
            let entry = self.catalog.get_entry_by_global_id(id);
            let ct = match &entry.item {
                CatalogItem::ContinualTask(ct) => ct.clone(),
                _ => unreachable!("only called with continual task builtins"),
            };
            let debug_name = self
                .catalog()
                .resolve_full_name(entry.name(), None)
                .to_string();
            let (_optimized_plan, physical_plan, _metainfo) = self
                .optimize_create_continual_task(&ct, *id, self.owned_catalog(), debug_name)
                .expect("builtin CT should optimize successfully");

            // Determine an as of for the new continual task.
            let mut id_bundle = dataflow_import_id_bundle(&physical_plan, ct.cluster_id);
            // Can't acquire a read hold on ourselves because we don't exist yet.
            id_bundle.storage_ids.remove(id);
            let read_holds = self.acquire_read_holds(&id_bundle);
            let as_of = read_holds.least_valid_read();

            collection.since = Some(as_of.clone());
        }
        self.controller
            .storage
            .create_collections(self.catalog.state().storage_metadata(), None, collections)
            .await
            .unwrap_or_terminate("cannot fail to create collections");
    }

    /// Invokes the optimizer on all indexes and materialized views in the catalog and inserts the
    /// resulting dataflow plans into the catalog state.
    ///
    /// `ordered_catalog_entries` must be sorted in dependency order, with dependencies ordered
    /// before their dependants.
    ///
    /// This method does not perform timestamp selection for the dataflows, nor does it create them
    /// in the compute controller. Both of these steps happen later during bootstrapping.
    ///
    /// Returns a map of expressions that were not cached.
    #[instrument]
    fn bootstrap_dataflow_plans(
        &mut self,
        ordered_catalog_entries: &[CatalogEntry],
        mut cached_global_exprs: BTreeMap<GlobalId, GlobalExpressions>,
    ) -> Result<BTreeMap<GlobalId, GlobalExpressions>, AdapterError> {
        // The optimizer expects to be able to query its `ComputeInstanceSnapshot` for
        // collections the current dataflow can depend on. But since we don't yet install anything
        // on compute instances, the snapshot information is incomplete. We fix that by manually
        // updating `ComputeInstanceSnapshot` objects to ensure they contain collections previously
        // optimized.
        let mut instance_snapshots = BTreeMap::new();
        let mut uncached_expressions = BTreeMap::new();

        let optimizer_config = OptimizerConfig::from(self.catalog().system_config());

        for entry in ordered_catalog_entries {
            match entry.item() {
                CatalogItem::Index(idx) => {
                    // Collect optimizer parameters.
                    let compute_instance =
                        instance_snapshots.entry(idx.cluster_id).or_insert_with(|| {
                            self.instance_snapshot(idx.cluster_id)
                                .expect("compute instance exists")
                        });
                    let global_id = idx.global_id();

                    // The index may already be installed on the compute instance. For example,
                    // this is the case for introspection indexes.
                    if compute_instance.contains_collection(&global_id) {
                        continue;
                    }

                    let (optimized_plan, physical_plan, metainfo) =
                        match cached_global_exprs.remove(&global_id) {
                            Some(global_expressions)
                                if global_expressions.optimizer_features
                                    == optimizer_config.features =>
                            {
                                info!("global expression cache hit for {global_id:?}");
                                (
                                    global_expressions.global_mir,
                                    global_expressions.physical_plan,
                                    global_expressions.dataflow_metainfos,
                                )
                            }
                            Some(_) | None => {
                                let (optimized_plan, global_lir_plan) = {
                                    // Build an optimizer for this INDEX.
                                    let mut optimizer = optimize::index::Optimizer::new(
                                        self.owned_catalog(),
                                        compute_instance.clone(),
                                        global_id,
                                        optimizer_config.clone(),
                                        self.optimizer_metrics(),
                                    );

                                    // MIR ⇒ MIR optimization (global)
                                    let index_plan = optimize::index::Index::new(
                                        entry.name().clone(),
                                        idx.on,
                                        idx.keys.to_vec(),
                                    );
                                    let global_mir_plan = optimizer.optimize(index_plan)?;
                                    let optimized_plan = global_mir_plan.df_desc().clone();

                                    // MIR ⇒ LIR lowering and LIR ⇒ LIR optimization (global)
                                    let global_lir_plan = optimizer.optimize(global_mir_plan)?;

                                    (optimized_plan, global_lir_plan)
                                };

                                let (physical_plan, metainfo) = global_lir_plan.unapply();
                                let metainfo = {
                                    // Pre-allocate a vector of transient GlobalIds for each notice.
                                    let notice_ids =
                                        std::iter::repeat_with(|| self.allocate_transient_id())
                                            .map(|(_item_id, gid)| gid)
                                            .take(metainfo.optimizer_notices.len())
                                            .collect::<Vec<_>>();
                                    // Return a metainfo with rendered notices.
                                    self.catalog().render_notices(
                                        metainfo,
                                        notice_ids,
                                        Some(idx.global_id()),
                                    )
                                };
                                uncached_expressions.insert(
                                    global_id,
                                    GlobalExpressions {
                                        global_mir: optimized_plan.clone(),
                                        physical_plan: physical_plan.clone(),
                                        dataflow_metainfos: metainfo.clone(),
                                        optimizer_features: OptimizerFeatures::from(
                                            self.catalog().system_config(),
                                        ),
                                    },
                                );
                                (optimized_plan, physical_plan, metainfo)
                            }
                        };

                    let catalog = self.catalog_mut();
                    catalog.set_optimized_plan(idx.global_id(), optimized_plan);
                    catalog.set_physical_plan(idx.global_id(), physical_plan);
                    catalog.set_dataflow_metainfo(idx.global_id(), metainfo);

                    compute_instance.insert_collection(idx.global_id());
                }
                CatalogItem::MaterializedView(mv) => {
                    // Collect optimizer parameters.
                    let compute_instance =
                        instance_snapshots.entry(mv.cluster_id).or_insert_with(|| {
                            self.instance_snapshot(mv.cluster_id)
                                .expect("compute instance exists")
                        });
                    let global_id = mv.global_id();

                    let (optimized_plan, physical_plan, metainfo) =
                        match cached_global_exprs.remove(&global_id) {
                            Some(global_expressions)
                                if global_expressions.optimizer_features
                                    == optimizer_config.features =>
                            {
                                info!("global expression cache hit for {global_id:?}");
                                (
                                    global_expressions.global_mir,
                                    global_expressions.physical_plan,
                                    global_expressions.dataflow_metainfos,
                                )
                            }
                            Some(_) | None => {
                                let (_, internal_view_id) = self.allocate_transient_id();
                                let debug_name = self
                                    .catalog()
                                    .resolve_full_name(entry.name(), None)
                                    .to_string();
                                let force_non_monotonic = Default::default();

                                let (optimized_plan, global_lir_plan) = {
                                    // Build an optimizer for this MATERIALIZED VIEW.
                                    let mut optimizer = optimize::materialized_view::Optimizer::new(
                                        self.owned_catalog().as_optimizer_catalog(),
                                        compute_instance.clone(),
                                        global_id,
                                        internal_view_id,
                                        mv.desc.iter_names().cloned().collect(),
                                        mv.non_null_assertions.clone(),
                                        mv.refresh_schedule.clone(),
                                        debug_name,
                                        optimizer_config.clone(),
                                        self.optimizer_metrics(),
                                        force_non_monotonic,
                                    );

                                    // MIR ⇒ MIR optimization (global)
                                    let global_mir_plan =
                                        optimizer.optimize(mv.optimized_expr.as_ref().clone())?;
                                    let optimized_plan = global_mir_plan.df_desc().clone();

                                    // MIR ⇒ LIR lowering and LIR ⇒ LIR optimization (global)
                                    let global_lir_plan = optimizer.optimize(global_mir_plan)?;

                                    (optimized_plan, global_lir_plan)
                                };

                                let (physical_plan, metainfo) = global_lir_plan.unapply();
                                let metainfo = {
                                    // Pre-allocate a vector of transient GlobalIds for each notice.
                                    let notice_ids =
                                        std::iter::repeat_with(|| self.allocate_transient_id())
                                            .map(|(_item_id, global_id)| global_id)
                                            .take(metainfo.optimizer_notices.len())
                                            .collect::<Vec<_>>();
                                    // Return a metainfo with rendered notices.
                                    self.catalog().render_notices(
                                        metainfo,
                                        notice_ids,
                                        Some(mv.global_id()),
                                    )
                                };
                                uncached_expressions.insert(
                                    global_id,
                                    GlobalExpressions {
                                        global_mir: optimized_plan.clone(),
                                        physical_plan: physical_plan.clone(),
                                        dataflow_metainfos: metainfo.clone(),
                                        optimizer_features: OptimizerFeatures::from(
                                            self.catalog().system_config(),
                                        ),
                                    },
                                );
                                (optimized_plan, physical_plan, metainfo)
                            }
                        };

                    let catalog = self.catalog_mut();
                    catalog.set_optimized_plan(mv.global_id(), optimized_plan);
                    catalog.set_physical_plan(mv.global_id(), physical_plan);
                    catalog.set_dataflow_metainfo(mv.global_id(), metainfo);

                    compute_instance.insert_collection(mv.global_id());
                }
                CatalogItem::ContinualTask(ct) => {
                    let compute_instance =
                        instance_snapshots.entry(ct.cluster_id).or_insert_with(|| {
                            self.instance_snapshot(ct.cluster_id)
                                .expect("compute instance exists")
                        });
                    let global_id = ct.global_id();

                    let (optimized_plan, physical_plan, metainfo) =
                        match cached_global_exprs.remove(&global_id) {
                            Some(global_expressions)
                                if global_expressions.optimizer_features
                                    == optimizer_config.features =>
                            {
                                info!("global expression cache hit for {global_id:?}");
                                (
                                    global_expressions.global_mir,
                                    global_expressions.physical_plan,
                                    global_expressions.dataflow_metainfos,
                                )
                            }
                            Some(_) | None => {
                                let debug_name = self
                                    .catalog()
                                    .resolve_full_name(entry.name(), None)
                                    .to_string();
                                let (optimized_plan, physical_plan, metainfo) = self
                                    .optimize_create_continual_task(
                                        ct,
                                        global_id,
                                        self.owned_catalog(),
                                        debug_name,
                                    )?;
                                uncached_expressions.insert(
                                    global_id,
                                    GlobalExpressions {
                                        global_mir: optimized_plan.clone(),
                                        physical_plan: physical_plan.clone(),
                                        dataflow_metainfos: metainfo.clone(),
                                        optimizer_features: OptimizerFeatures::from(
                                            self.catalog().system_config(),
                                        ),
                                    },
                                );
                                (optimized_plan, physical_plan, metainfo)
                            }
                        };

                    let catalog = self.catalog_mut();
                    catalog.set_optimized_plan(ct.global_id(), optimized_plan);
                    catalog.set_physical_plan(ct.global_id(), physical_plan);
                    catalog.set_dataflow_metainfo(ct.global_id(), metainfo);

                    compute_instance.insert_collection(ct.global_id());
                }
                _ => (),
            }
        }

        Ok(uncached_expressions)
    }

    /// Selects for each compute dataflow an as-of suitable for bootstrapping it.
    ///
    /// Returns a set of [`ReadHold`]s that ensures the read frontiers of involved collections stay
    /// in place and that must not be dropped before all compute dataflows have been created with
    /// the compute controller.
    ///
    /// This method expects all storage collections and dataflow plans to be available, so it must
    /// run after [`Coordinator::bootstrap_storage_collections`] and
    /// [`Coordinator::bootstrap_dataflow_plans`].
    async fn bootstrap_dataflow_as_ofs(&mut self) -> BTreeMap<GlobalId, ReadHold<Timestamp>> {
        let mut catalog_ids = Vec::new();
        let mut dataflows = Vec::new();
        let mut read_policies = BTreeMap::new();
        for entry in self.catalog.entries() {
            let gid = match entry.item() {
                CatalogItem::Index(idx) => idx.global_id(),
                CatalogItem::MaterializedView(mv) => mv.global_id(),
                CatalogItem::ContinualTask(ct) => ct.global_id(),
                CatalogItem::Table(_)
                | CatalogItem::Source(_)
                | CatalogItem::Log(_)
                | CatalogItem::View(_)
                | CatalogItem::Sink(_)
                | CatalogItem::Type(_)
                | CatalogItem::Func(_)
                | CatalogItem::Secret(_)
                | CatalogItem::Connection(_) => continue,
            };
            if let Some(plan) = self.catalog.try_get_physical_plan(&gid) {
                catalog_ids.push(gid);
                dataflows.push(plan.clone());

                if let Some(compaction_window) = entry.item().initial_logical_compaction_window() {
                    read_policies.insert(gid, compaction_window.into());
                }
            }
        }

        let read_ts = self.get_local_read_ts().await;
        let read_holds = as_of_selection::run(
            &mut dataflows,
            &read_policies,
            &*self.controller.storage_collections,
            read_ts,
        );

        let catalog = self.catalog_mut();
        for (id, plan) in catalog_ids.into_iter().zip(dataflows) {
            catalog.set_physical_plan(id, plan);
        }

        read_holds
    }

    /// 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.
    ///
    /// BOXED FUTURE: As of Nov 2023 the returned Future from this function was 92KB. This would
    /// get stored on the stack which is bad for runtime performance, and blow up our stack usage.
    /// Because of that we purposefully move this Future onto the heap (i.e. Box it).
    fn serve(
        mut self,
        mut internal_cmd_rx: mpsc::UnboundedReceiver<Message>,
        mut strict_serializable_reads_rx: mpsc::UnboundedReceiver<(ConnectionId, PendingReadTxn)>,
        mut cmd_rx: mpsc::UnboundedReceiver<(OpenTelemetryContext, Command)>,
        group_commit_rx: appends::GroupCommitWaiter,
    ) -> LocalBoxFuture<'static, ()> {
        async move {
            // Watcher that listens for and reports cluster service status changes.
            let mut cluster_events = self.controller.events_stream();
            let last_message = Arc::new(Mutex::new(LastMessage {
                kind: "none",
                stmt: None,
            }));

            let (idle_tx, mut idle_rx) = tokio::sync::mpsc::channel(1);
            let idle_metric = self.metrics.queue_busy_seconds.with_label_values(&[]);
            let last_message_watchdog = Arc::clone(&last_message);

            spawn(|| "coord watchdog", async move {
                // Every 5 seconds, attempt to measure how long it takes for the
                // coord select loop to be empty, because this message is the last
                // processed. If it is idle, this will result in some microseconds
                // of measurement.
                let mut interval = tokio::time::interval(Duration::from_secs(5));
                // If we end up having to wait more than 5 seconds for the coord to respond, then the
                // behavior of Delay results in the interval "restarting" from whenever we yield
                // instead of trying to catch up.
                interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay);

                // Track if we become stuck to de-dupe error reporting.
                let mut coord_stuck = false;

                loop {
                    interval.tick().await;

                    // Wait for space in the channel, if we timeout then the coordinator is stuck!
                    let duration = tokio::time::Duration::from_secs(30);
                    let timeout = tokio::time::timeout(duration, idle_tx.reserve()).await;
                    let Ok(maybe_permit) = timeout else {
                        // Only log if we're newly stuck, to prevent logging repeatedly.
                        if !coord_stuck {
                            let last_message = last_message_watchdog.lock().expect("poisoned");
                            tracing::warn!(
                                last_message_kind = %last_message.kind,
                                last_message_sql = %last_message.stmt_to_string(),
                                "coordinator stuck for {duration:?}",
                            );
                        }
                        coord_stuck = true;

                        continue;
                    };

                    // We got a permit, we're not stuck!
                    if coord_stuck {
                        tracing::info!("Coordinator became unstuck");
                    }
                    coord_stuck = false;

                    // If we failed to acquire a permit it's because we're shutting down.
                    let Ok(permit) = maybe_permit else {
                        break;
                    };

                    permit.send(idle_metric.start_timer());
                }
            });

            self.schedule_storage_usage_collection().await;
            self.spawn_privatelink_vpc_endpoints_watch_task();
            self.spawn_statement_logging_task();
            flags::tracing_config(self.catalog.system_config()).apply(&self.tracing_handle);

            // Report if the handling of a single message takes longer than this threshold.
            let warn_threshold = self
                .catalog()
                .system_config()
                .coord_slow_message_warn_threshold();

            // How many messages we'd like to batch up before processing them. Must be > 0.
            const MESSAGE_BATCH: usize = 64;
            let mut messages = Vec::with_capacity(MESSAGE_BATCH);
            let mut cmd_messages = Vec::with_capacity(MESSAGE_BATCH);

            let message_batch = self.metrics
                .message_batch
                .with_label_values(&[]);

            loop {
                // Before adding a branch to this select loop, please ensure that the branch is
                // cancellation safe and add a comment explaining why. You can refer here for more
                // info: https://docs.rs/tokio/latest/tokio/macro.select.html#cancellation-safety
                select! {
                    // We prioritize internal commands over other commands. However, we work through
                    // batches of commands in some branches of this select, which means that even if
                    // a command generates internal commands, we will work through the current batch
                    // before receiving a new batch of commands.
                    biased;

                    // `recv_many()` on `UnboundedReceiver` is cancellation safe:
                    // https://docs.rs/tokio/1.38.0/tokio/sync/mpsc/struct.UnboundedReceiver.html#cancel-safety-1
                    // Receive a batch of commands.
                    _ = internal_cmd_rx.recv_many(&mut messages, MESSAGE_BATCH) => {},
                    // `next()` on any stream is cancel-safe:
                    // https://docs.rs/tokio-stream/0.1.9/tokio_stream/trait.StreamExt.html#cancel-safety
                    // Receive a single command.
                    Some(event) = cluster_events.next() => messages.push(Message::ClusterEvent(event)),
                    // See [`mz_controller::Controller::Controller::ready`] for notes
                    // on why this is cancel-safe.
                    // Receive a single command.
                    () = self.controller.ready() => {
                        messages.push(Message::ControllerReady);
                    }
                    // See [`appends::GroupCommitWaiter`] for notes on why this is cancel safe.
                    // Receive a single command.
                    permit = group_commit_rx.ready() => {
                        // If we happen to have batched exactly one user write, use
                        // that span so the `emit_trace_id_notice` hooks up.
                        // Otherwise, the best we can do is invent a new root span
                        // and make it follow from all the Spans in the pending
                        // writes.
                        let user_write_spans = self.pending_writes.iter().flat_map(|x| match x {
                            PendingWriteTxn::User{span, ..} => Some(span),
                            PendingWriteTxn::System{..} => None,
                        });
                        let span = match user_write_spans.exactly_one() {
                            Ok(span) => span.clone(),
                            Err(user_write_spans) => {
                                let span = info_span!(parent: None, "group_commit_notify");
                                for s in user_write_spans {
                                    span.follows_from(s);
                                }
                                span
                            }
                        };
                        messages.push(Message::GroupCommitInitiate(span, Some(permit)));
                    },
                    // `recv_many()` on `UnboundedReceiver` is cancellation safe:
                    // https://docs.rs/tokio/1.38.0/tokio/sync/mpsc/struct.UnboundedReceiver.html#cancel-safety-1
                    // Receive a batch of commands.
                    count = cmd_rx.recv_many(&mut cmd_messages, MESSAGE_BATCH) => {
                        if count == 0 {
                            break;
                        } else {
                            messages.extend(cmd_messages.drain(..).map(|(otel_ctx, cmd)| Message::Command(otel_ctx, cmd)));
                        }
                    },
                    // `recv()` on `UnboundedReceiver` is cancellation safe:
                    // https://docs.rs/tokio/1.38.0/tokio/sync/mpsc/struct.UnboundedReceiver.html#cancel-safety
                    // Receive a single command.
                    Some(pending_read_txn) = strict_serializable_reads_rx.recv() => {
                        let mut pending_read_txns = vec![pending_read_txn];
                        while let Ok(pending_read_txn) = strict_serializable_reads_rx.try_recv() {
                            pending_read_txns.push(pending_read_txn);
                        }
                        for (conn_id, pending_read_txn) in pending_read_txns {
                            let prev = self.pending_linearize_read_txns.insert(conn_id, pending_read_txn);
                            soft_assert_or_log!(
                                prev.is_none(),
                                "connections can not have multiple concurrent reads, prev: {prev:?}"
                            )
                        }
                        messages.push(Message::LinearizeReads);
                    }
                    // `tick()` on `Interval` is cancel-safe:
                    // https://docs.rs/tokio/1.19.2/tokio/time/struct.Interval.html#cancel-safety
                    // Receive a single command.
                    _ = self.advance_timelines_interval.tick() => {
                        if self.controller.read_only() {
                            tracing::info!("not advancing timelines in read-only mode");
                            continue;
                        }
                        let span = info_span!(parent: None, "coord::advance_timelines_interval");
                        span.follows_from(Span::current());
                        messages.push(Message::GroupCommitInitiate(span, None));
                    },
                    // `tick()` on `Interval` is cancel-safe:
                    // https://docs.rs/tokio/1.19.2/tokio/time/struct.Interval.html#cancel-safety
                    // Receive a single command.
                    _ = self.check_cluster_scheduling_policies_interval.tick() => {
                        messages.push(Message::CheckSchedulingPolicies);
                    },

                    // `tick()` on `Interval` is cancel-safe:
                    // https://docs.rs/tokio/1.19.2/tokio/time/struct.Interval.html#cancel-safety
                    // Receive a single command.
                    _ = self.caught_up_check_interval.tick() => {
                        // We do this directly on the main loop instead of
                        // firing off a message. We are still in read-only mode,
                        // so optimizing for latency, not blocking the main loop
                        // is not that important.
                        self.maybe_check_caught_up().await;

                        continue;
                    },

                    // Process the idle metric at the lowest priority to sample queue non-idle time.
                    // `recv()` on `Receiver` is cancellation safe:
                    // https://docs.rs/tokio/1.8.0/tokio/sync/mpsc/struct.Receiver.html#cancel-safety
                    // Receive a single command.
                    timer = idle_rx.recv() => {
                        timer.expect("does not drop").observe_duration();
                        self.metrics
                            .message_handling
                            .with_label_values(&["watchdog"])
                            .observe(0.0);
                        continue;
                    }
                };

                // Observe the number of messages we're processing at once.
                message_batch.observe(f64::cast_lossy(messages.len()));

                for msg in messages.drain(..) {
                    // All message processing functions trace. Start a parent span
                    // for them to make it easy to find slow messages.
                    let msg_kind = msg.kind();
                    let span = span!(
                        target: "mz_adapter::coord::handle_message_loop",
                        Level::INFO,
                        "coord::handle_message",
                        kind = msg_kind
                    );
                    let otel_context = span.context().span().span_context().clone();

                    // Record the last kind of message in case we get stuck. For
                    // execute commands, we additionally stash the user's SQL,
                    // statement, so we can log it in case we get stuck.
                    *last_message.lock().expect("poisoned") = LastMessage {
                        kind: msg_kind,
                        stmt: match &msg {
                            Message::Command(
                                _,
                                Command::Execute {
                                    portal_name,
                                    session,
                                    ..
                                },
                            ) => session
                                .get_portal_unverified(portal_name)
                                .and_then(|p| p.stmt.as_ref().map(Arc::clone)),
                            _ => None,
                        },
                    };

                    let start = Instant::now();
                    self.handle_message(msg).instrument(span).await;
                    let duration = start.elapsed();

                    self.metrics
                        .message_handling
                        .with_label_values(&[msg_kind])
                        .observe(duration.as_secs_f64());

                    // If something is _really_ slow, print a trace id for debugging, if OTEL is enabled.
                    if duration > warn_threshold {
                        let trace_id = otel_context.is_valid().then(|| otel_context.trace_id());
                        tracing::error!(
                            ?msg_kind,
                            ?trace_id,
                            ?duration,
                            "very slow coordinator message"
                        );
                    }
                }
            }
            // Try and cleanup as a best effort. There may be some async tasks out there holding a
            // reference that prevents us from cleaning up.
            if let Some(catalog) = Arc::into_inner(self.catalog) {
                catalog.expire().await;
            }
        }
        .boxed_local()
    }

    /// Obtain a read-only Catalog reference.
    fn catalog(&self) -> &Catalog {
        &self.catalog
    }

    /// Obtain a read-only Catalog snapshot, suitable for giving out to
    /// non-Coordinator thread tasks.
    fn owned_catalog(&self) -> Arc<Catalog> {
        Arc::clone(&self.catalog)
    }

    /// Obtain a handle to the optimizer metrics, suitable for giving
    /// out to non-Coordinator thread tasks.
    fn optimizer_metrics(&self) -> OptimizerMetrics {
        self.optimizer_metrics.clone()
    }

    /// Obtain a writeable Catalog reference.
    fn catalog_mut(&mut self) -> &mut Catalog {
        // make_mut will cause any other Arc references (from owned_catalog) to
        // continue to be valid by cloning the catalog, putting it in a new Arc,
        // which lives at self._catalog. If there are no other Arc references,
        // then no clone is made, and it returns a reference to the existing
        // object. This makes this method and owned_catalog both very cheap: at
        // most one clone per catalog mutation, but only if there's a read-only
        // reference to it.
        Arc::make_mut(&mut self.catalog)
    }

    /// Obtain a reference to the coordinator's connection context.
    fn connection_context(&self) -> &ConnectionContext {
        self.controller.connection_context()
    }

    /// Obtain a reference to the coordinator's secret reader, in an `Arc`.
    fn secrets_reader(&self) -> &Arc<dyn SecretsReader> {
        &self.connection_context().secrets_reader
    }

    /// Publishes a notice message to all sessions.
    pub(crate) fn broadcast_notice(&self, notice: AdapterNotice) {
        for meta in self.active_conns.values() {
            let _ = meta.notice_tx.send(notice.clone());
        }
    }

    pub(crate) fn active_conns(&self) -> &BTreeMap<ConnectionId, ConnMeta> {
        &self.active_conns
    }

    #[instrument(level = "debug")]
    pub(crate) fn retire_execution(
        &mut self,
        reason: StatementEndedExecutionReason,
        ctx_extra: ExecuteContextExtra,
    ) {
        if let Some(uuid) = ctx_extra.retire() {
            self.end_statement_execution(uuid, reason);
        }
    }

    /// Creates a new dataflow builder from the catalog and indexes in `self`.
    #[instrument(level = "debug")]
    pub fn dataflow_builder(&self, instance: ComputeInstanceId) -> DataflowBuilder {
        let compute = self
            .instance_snapshot(instance)
            .expect("compute instance does not exist");
        DataflowBuilder::new(self.catalog().state(), compute)
    }

    /// Return a reference-less snapshot to the indicated compute instance.
    pub fn instance_snapshot(
        &self,
        id: ComputeInstanceId,
    ) -> Result<ComputeInstanceSnapshot, InstanceMissing> {
        ComputeInstanceSnapshot::new(&self.controller, id)
    }

    /// Call into the compute controller to install a finalized dataflow, and
    /// initialize the read policies for its exported readable objects.
    pub(crate) async fn ship_dataflow(
        &mut self,
        dataflow: DataflowDescription<Plan>,
        instance: ComputeInstanceId,
        subscribe_target_replica: Option<ReplicaId>,
    ) {
        // We must only install read policies for indexes, not for sinks.
        // Sinks are write-only compute collections that don't have read policies.
        let export_ids = dataflow.exported_index_ids().collect();

        self.controller
            .compute
            .create_dataflow(instance, dataflow, subscribe_target_replica)
            .unwrap_or_terminate("dataflow creation cannot fail");

        self.initialize_compute_read_policies(export_ids, instance, CompactionWindow::Default)
            .await;
    }

    /// Like `ship_dataflow`, but also await on builtin table updates.
    pub(crate) async fn ship_dataflow_and_notice_builtin_table_updates(
        &mut self,
        dataflow: DataflowDescription<Plan>,
        instance: ComputeInstanceId,
        notice_builtin_updates_fut: Option<BuiltinTableAppendNotify>,
    ) {
        if let Some(notice_builtin_updates_fut) = notice_builtin_updates_fut {
            let ship_dataflow_fut = self.ship_dataflow(dataflow, instance, None);
            let ((), ()) =
                futures::future::join(notice_builtin_updates_fut, ship_dataflow_fut).await;
        } else {
            self.ship_dataflow(dataflow, instance, None).await;
        }
    }

    /// Install a _watch set_ in the controller that is automatically associated with the given
    /// connection id. The watchset will be automatically cleared if the connection terminates
    /// before the watchset completes.
    pub fn install_compute_watch_set(
        &mut self,
        conn_id: ConnectionId,
        objects: BTreeSet<GlobalId>,
        t: Timestamp,
        state: WatchSetResponse,
    ) {
        let ws_id = self.controller.install_compute_watch_set(objects, t);
        self.connection_watch_sets
            .entry(conn_id.clone())
            .or_default()
            .insert(ws_id);
        self.installed_watch_sets.insert(ws_id, (conn_id, state));
    }

    /// Install a _watch set_ in the controller that is automatically associated with the given
    /// connection id. The watchset will be automatically cleared if the connection terminates
    /// before the watchset completes.
    pub fn install_storage_watch_set(
        &mut self,
        conn_id: ConnectionId,
        objects: BTreeSet<GlobalId>,
        t: Timestamp,
        state: WatchSetResponse,
    ) {
        let ws_id = self.controller.install_storage_watch_set(objects, t);
        self.connection_watch_sets
            .entry(conn_id.clone())
            .or_default()
            .insert(ws_id);
        self.installed_watch_sets.insert(ws_id, (conn_id, state));
    }

    /// Cancels pending watchsets associated with the provided connection id.
    pub fn cancel_pending_watchsets(&mut self, conn_id: &ConnectionId) {
        if let Some(ws_ids) = self.connection_watch_sets.remove(conn_id) {
            for ws_id in ws_ids {
                self.installed_watch_sets.remove(&ws_id);
            }
        }
    }

    /// Returns the state of the [`Coordinator`] formatted as JSON.
    ///
    /// The returned value is not guaranteed to be stable and may change at any point in time.
    pub async fn dump(&self) -> Result<serde_json::Value, anyhow::Error> {
        // Note: We purposefully use the `Debug` formatting for the value of all fields in the
        // returned object as a tradeoff between usability and stability. `serde_json` will fail
        // to serialize an object if the keys aren't strings, so `Debug` formatting the values
        // prevents a future unrelated change from silently breaking this method.

        let global_timelines: BTreeMap<_, _> = self
            .global_timelines
            .iter()
            .map(|(timeline, state)| (timeline.to_string(), format!("{state:?}")))
            .collect();
        let active_conns: BTreeMap<_, _> = self
            .active_conns
            .iter()
            .map(|(id, meta)| (id.unhandled().to_string(), format!("{meta:?}")))
            .collect();
        let txn_read_holds: BTreeMap<_, _> = self
            .txn_read_holds
            .iter()
            .map(|(id, capability)| (id.unhandled().to_string(), format!("{capability:?}")))
            .collect();
        let pending_peeks: BTreeMap<_, _> = self
            .pending_peeks
            .iter()
            .map(|(id, peek)| (id.to_string(), format!("{peek:?}")))
            .collect();
        let client_pending_peeks: BTreeMap<_, _> = self
            .client_pending_peeks
            .iter()
            .map(|(id, peek)| {
                let peek: BTreeMap<_, _> = peek
                    .iter()
                    .map(|(uuid, storage_id)| (uuid.to_string(), storage_id))
                    .collect();
                (id.to_string(), peek)
            })
            .collect();
        let pending_linearize_read_txns: BTreeMap<_, _> = self
            .pending_linearize_read_txns
            .iter()
            .map(|(id, read_txn)| (id.unhandled().to_string(), format!("{read_txn:?}")))
            .collect();

        let map = serde_json::Map::from_iter([
            (
                "global_timelines".to_string(),
                serde_json::to_value(global_timelines)?,
            ),
            (
                "active_conns".to_string(),
                serde_json::to_value(active_conns)?,
            ),
            (
                "txn_read_holds".to_string(),
                serde_json::to_value(txn_read_holds)?,
            ),
            (
                "pending_peeks".to_string(),
                serde_json::to_value(pending_peeks)?,
            ),
            (
                "client_pending_peeks".to_string(),
                serde_json::to_value(client_pending_peeks)?,
            ),
            (
                "pending_linearize_read_txns".to_string(),
                serde_json::to_value(pending_linearize_read_txns)?,
            ),
            ("controller".to_string(), self.controller.dump().await?),
        ]);
        Ok(serde_json::Value::Object(map))
    }

    /// Prune all storage usage events from the [`MZ_STORAGE_USAGE_BY_SHARD`] table that are older
    /// than `retention_period`.
    ///
    /// This method will read the entire contents of [`MZ_STORAGE_USAGE_BY_SHARD`] into memory
    /// which can be expensive.
    ///
    /// DO NOT call this method outside of startup. The safety of reading at the current oracle read
    /// timestamp and then writing at whatever the current write timestamp is (instead of
    /// `read_ts + 1`) relies on the fact that there are no outstanding writes during startup.
    ///
    /// Group commit, which this method uses to write the retractions, has builtin fencing, and we
    /// never commit retractions to [`MZ_STORAGE_USAGE_BY_SHARD`] outside of this method, which is
    /// only called once during startup. So we don't have to worry about double/invalid retractions.
    async fn prune_storage_usage_events_on_startup(&self, retention_period: Duration) {
        let item_id = self
            .catalog()
            .resolve_builtin_table(&MZ_STORAGE_USAGE_BY_SHARD);
        let global_id = self.catalog.get_entry(&item_id).latest_global_id();
        let read_ts = self.get_local_read_ts().await;
        let current_contents_fut = self.controller.storage.snapshot(global_id, read_ts);
        let internal_cmd_tx = self.internal_cmd_tx.clone();
        spawn(|| "storage_usage_prune", async move {
            let mut current_contents = current_contents_fut
                .await
                .unwrap_or_terminate("cannot fail to fetch snapshot");
            differential_dataflow::consolidation::consolidate(&mut current_contents);

            let cutoff_ts = u128::from(read_ts).saturating_sub(retention_period.as_millis());
            let mut expired = Vec::new();
            for (row, diff) in current_contents {
                assert_eq!(
                    diff, 1,
                    "consolidated contents should not contain retractions: ({row:#?}, {diff:#?})"
                );
                // This logic relies on the definition of `mz_storage_usage_by_shard` not changing.
                let collection_timestamp = row
                    .unpack()
                    .get(3)
                    .expect("definition of mz_storage_by_shard changed")
                    .unwrap_timestamptz();
                let collection_timestamp = collection_timestamp.timestamp_millis();
                let collection_timestamp: u128 = collection_timestamp
                    .try_into()
                    .expect("all collections happen after Jan 1 1970");
                if collection_timestamp < cutoff_ts {
                    debug!("pruning storage event {row:?}");
                    let builtin_update = BuiltinTableUpdate {
                        id: item_id,
                        row,
                        diff: -1,
                    };
                    expired.push(builtin_update);
                }
            }

            // main thread has shut down.
            let _ = internal_cmd_tx.send(Message::StorageUsagePrune(expired));
        });
    }
}

#[cfg(test)]
impl Coordinator {
    #[allow(dead_code)]
    async fn verify_ship_dataflow_no_error(&mut self, dataflow: DataflowDescription<Plan>) {
        // `ship_dataflow_new` is not allowed to have a `Result` return because this function is
        // 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).

        // An arbitrary compute instance ID to satisfy the function calls below. Note that
        // this only works because this function will never run.
        let compute_instance = ComputeInstanceId::User(1);

        let _: () = self.ship_dataflow(dataflow, compute_instance, None).await;
    }
}

/// Contains information about the last message the [`Coordinator`] processed.
struct LastMessage {
    kind: &'static str,
    stmt: Option<Arc<Statement<Raw>>>,
}

impl LastMessage {
    /// Returns a redacted version of the statement that is safe for logs.
    fn stmt_to_string(&self) -> Cow<'static, str> {
        self.stmt
            .as_ref()
            .map(|stmt| stmt.to_ast_string_redacted().into())
            .unwrap_or("<none>".into())
    }
}

impl fmt::Debug for LastMessage {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        f.debug_struct("LastMessage")
            .field("kind", &self.kind)
            .field("stmt", &self.stmt_to_string())
            .finish()
    }
}

impl Drop for LastMessage {
    fn drop(&mut self) {
        // Only print the last message if we're currently panicking, otherwise we'd spam our logs.
        if std::thread::panicking() {
            // If we're panicking theres no guarantee `tracing` still works, so print to stderr.
            eprintln!("Coordinator panicking, dumping last message\n{self:?}",);
        }
    }
}

/// 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.
///
/// BOXED FUTURE: As of Nov 2023 the returned Future from this function was 42KB. This would
/// get stored on the stack which is bad for runtime performance, and blow up our stack usage.
/// Because of that we purposefully move this Future onto the heap (i.e. Box it).
pub fn serve(
    Config {
        controller_config,
        controller_envd_epoch,
        storage,
        timestamp_oracle_url,
        unsafe_mode,
        all_features,
        build_info,
        environment_id,
        metrics_registry,
        now,
        secrets_controller,
        cloud_resource_controller,
        cluster_replica_sizes,
        builtin_system_cluster_replica_size,
        builtin_catalog_server_cluster_replica_size,
        builtin_probe_cluster_replica_size,
        builtin_support_cluster_replica_size,
        builtin_analytics_cluster_replica_size,
        system_parameter_defaults,
        availability_zones,
        storage_usage_client,
        storage_usage_collection_interval,
        storage_usage_retention_period,
        segment_client,
        egress_addresses,
        aws_account_id,
        aws_privatelink_availability_zones,
        connection_context,
        remote_system_parameters,
        active_connection_count,
        webhook_concurrency_limit,
        http_host_name,
        tracing_handle,
        read_only_controllers,
        enable_0dt_deployment,
        caught_up_trigger: clusters_caught_up_trigger,
        helm_chart_version,
    }: Config,
) -> BoxFuture<'static, Result<(Handle, Client), AdapterError>> {
    async move {
        let coord_start = Instant::now();
        info!("startup: coordinator init: beginning");
        info!("startup: coordinator init: preamble beginning");

        // Initializing the builtins can be an expensive process and consume a lot of memory. We
        // forcibly initialize it early while the stack is relatively empty to avoid stack
        // overflows later.
        let _builtins = LazyLock::force(&BUILTINS_STATIC);

        let (cmd_tx, cmd_rx) = mpsc::unbounded_channel();
        let (internal_cmd_tx, internal_cmd_rx) = mpsc::unbounded_channel();
        let (group_commit_tx, group_commit_rx) = appends::notifier();
        let (strict_serializable_reads_tx, strict_serializable_reads_rx) =
            mpsc::unbounded_channel();

        // Validate and process availability zones.
        if !availability_zones.iter().all_unique() {
            coord_bail!("availability zones must be unique");
        }

        let aws_principal_context = match (
            aws_account_id,
            connection_context.aws_external_id_prefix.clone(),
        ) {
            (Some(aws_account_id), Some(aws_external_id_prefix)) => Some(AwsPrincipalContext {
                aws_account_id,
                aws_external_id_prefix,
            }),
            _ => None,
        };

        let aws_privatelink_availability_zones = aws_privatelink_availability_zones
            .map(|azs_vec| BTreeSet::from_iter(azs_vec.iter().cloned()));

        info!(
            "startup: coordinator init: preamble complete in {:?}",
            coord_start.elapsed()
        );
        let oracle_init_start = Instant::now();
        info!("startup: coordinator init: timestamp oracle init beginning");

        let pg_timestamp_oracle_config = timestamp_oracle_url
            .map(|pg_url| PostgresTimestampOracleConfig::new(&pg_url, &metrics_registry));
        let mut initial_timestamps =
            get_initial_oracle_timestamps(&pg_timestamp_oracle_config).await?;

        // Insert an entry for the `EpochMilliseconds` timeline if one doesn't exist,
        // which will ensure that the timeline is initialized since it's required
        // by the system.
        initial_timestamps
            .entry(Timeline::EpochMilliseconds)
            .or_insert_with(mz_repr::Timestamp::minimum);
        let mut timestamp_oracles = BTreeMap::new();
        for (timeline, initial_timestamp) in initial_timestamps {
            Coordinator::ensure_timeline_state_with_initial_time(
                &timeline,
                initial_timestamp,
                now.clone(),
                pg_timestamp_oracle_config.clone(),
                &mut timestamp_oracles,
                read_only_controllers,
            )
            .await;
        }
        // Choose a time at which to boot. This is used, for example, to prune
        // old storage usage data or migrate audit log entries.
        //
        // This time is usually the current system time, but with protection
        // against backwards time jumps, even across restarts.
        let epoch_millis_oracle = &timestamp_oracles
            .get(&Timeline::EpochMilliseconds)
            .expect("inserted above")
            .oracle;

        let boot_ts = if read_only_controllers {
            epoch_millis_oracle.read_ts().await
        } else {
            // Getting a write timestamp bumps the write timestamp in the
            // oracle, which we're not allowed in read-only mode.
            epoch_millis_oracle.write_ts().await.timestamp
        };

        info!(
            "startup: coordinator init: timestamp oracle init complete in {:?}",
            oracle_init_start.elapsed()
        );

        let catalog_open_start = Instant::now();
        info!("startup: coordinator init: catalog open beginning");
        let persist_client = controller_config
            .persist_clients
            .open(controller_config.persist_location.clone())
            .await
            .context("opening persist client")?;
        let builtin_item_migration_config = if enable_0dt_deployment {
            BuiltinItemMigrationConfig::ZeroDownTime {
                persist_client: persist_client.clone(),
                deploy_generation: controller_config.deploy_generation,
                read_only: read_only_controllers,
            }
        } else {
            BuiltinItemMigrationConfig::Legacy
        };
        let OpenCatalogResult {
            mut catalog,
            storage_collections_to_drop,
            migrated_storage_collections_0dt,
            new_builtin_collections,
            builtin_table_updates,
            cached_global_exprs,
            uncached_local_exprs,
        } = Catalog::open(mz_catalog::config::Config {
            storage,
            metrics_registry: &metrics_registry,
            state: mz_catalog::config::StateConfig {
                unsafe_mode,
                all_features,
                build_info,
                environment_id: environment_id.clone(),
                read_only: read_only_controllers,
                now: now.clone(),
                boot_ts: boot_ts.clone(),
                skip_migrations: false,
                cluster_replica_sizes,
                builtin_system_cluster_replica_size,
                builtin_catalog_server_cluster_replica_size,
                builtin_probe_cluster_replica_size,
                builtin_support_cluster_replica_size,
                builtin_analytics_cluster_replica_size,
                system_parameter_defaults,
                remote_system_parameters,
                availability_zones,
                egress_addresses,
                aws_principal_context,
                aws_privatelink_availability_zones,
                connection_context,
                active_connection_count,
                http_host_name,
                builtin_item_migration_config,
                persist_client: persist_client.clone(),
                enable_expression_cache_override: None,
                enable_0dt_deployment,
                helm_chart_version,
            },
        })
        .await?;

        if !read_only_controllers {
            epoch_millis_oracle.apply_write(boot_ts).await;
        }

        info!(
            "startup: coordinator init: catalog open complete in {:?}",
            catalog_open_start.elapsed()
        );

        let coord_thread_start = Instant::now();
        info!("startup: coordinator init: coordinator thread start beginning");

        let session_id = catalog.config().session_id;
        let start_instant = catalog.config().start_instant;

        // 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 metrics = Metrics::register_into(&metrics_registry);
        let metrics_clone = metrics.clone();
        let optimizer_metrics = OptimizerMetrics::register_into(&metrics_registry);
        let segment_client_clone = segment_client.clone();
        let coord_now = now.clone();
        let advance_timelines_interval = tokio::time::interval(catalog.config().timestamp_interval);
        let mut check_scheduling_policies_interval = tokio::time::interval(
            catalog
                .system_config()
                .cluster_check_scheduling_policies_interval(),
        );
        check_scheduling_policies_interval.set_missed_tick_behavior(MissedTickBehavior::Delay);

        let clusters_caught_up_check_interval = if read_only_controllers {
            let dyncfgs = catalog.system_config().dyncfgs();
            let interval = WITH_0DT_DEPLOYMENT_CAUGHT_UP_CHECK_INTERVAL.get(dyncfgs);

            let mut interval = tokio::time::interval(interval);
            interval.set_missed_tick_behavior(MissedTickBehavior::Skip);
            interval
        } else {
            // When not in read-only mode, we don't do hydration checks. But we
            // still have to provide _some_ interval. This is large enough that
            // it doesn't matter.
            //
            // TODO(aljoscha): We cannot use Duration::MAX right now because of
            // https://github.com/tokio-rs/tokio/issues/6634. Use that once it's
            // fixed for good.
            let mut interval = tokio::time::interval(Duration::from_secs(60 * 60));
            interval.set_missed_tick_behavior(MissedTickBehavior::Skip);
            interval
        };

        let clusters_caught_up_check =
            clusters_caught_up_trigger.map(|trigger| CaughtUpCheckContext {
                trigger,
                exclude_collections: new_builtin_collections.into_iter().collect(),
            });

        if let Some(config) = pg_timestamp_oracle_config.as_ref() {
            // Apply settings from system vars as early as possible because some
            // of them are locked in right when an oracle is first opened!
            let pg_timestamp_oracle_params =
                flags::pg_timstamp_oracle_config(catalog.system_config());
            pg_timestamp_oracle_params.apply(config);
        }

        let parent_span = tracing::Span::current();
        let thread = thread::Builder::new()
            // The Coordinator thread tends to keep a lot of data on its stack. To
            // prevent a stack overflow we allocate a stack three times as big as the default
            // stack.
            .stack_size(3 * stack::STACK_SIZE)
            .name("coordinator".to_string())
            .spawn(move || {
                let span = info_span!(parent: parent_span, "coord::coordinator").entered();

                let controller = handle
                    .block_on({
                        catalog.initialize_controller(
                            controller_config,
                            controller_envd_epoch,
                            read_only_controllers,
                            storage_collections_to_drop,
                        )
                    })
                    .expect("failed to initialize storage_controller");

                let catalog = Arc::new(catalog);

                let caching_secrets_reader = CachingSecretsReader::new(secrets_controller.reader());
                let mut coord = Coordinator {
                    controller,
                    catalog,
                    internal_cmd_tx,
                    group_commit_tx,
                    strict_serializable_reads_tx,
                    global_timelines: timestamp_oracles,
                    transient_id_gen: Arc::new(TransientIdGen::new()),
                    active_conns: BTreeMap::new(),
                    txn_read_holds: Default::default(),
                    pending_peeks: BTreeMap::new(),
                    client_pending_peeks: BTreeMap::new(),
                    pending_linearize_read_txns: BTreeMap::new(),
                    serialized_ddl: LockedVecDeque::new(),
                    active_compute_sinks: BTreeMap::new(),
                    active_webhooks: BTreeMap::new(),
                    staged_cancellation: BTreeMap::new(),
                    introspection_subscribes: BTreeMap::new(),
                    write_locks: BTreeMap::new(),
                    deferred_write_ops: BTreeMap::new(),
                    pending_writes: Vec::new(),
                    advance_timelines_interval,
                    secrets_controller,
                    caching_secrets_reader,
                    cloud_resource_controller,
                    transient_replica_metadata: BTreeMap::new(),
                    storage_usage_client,
                    storage_usage_collection_interval,
                    segment_client,
                    metrics,
                    optimizer_metrics,
                    tracing_handle,
                    statement_logging: StatementLogging::new(coord_now.clone()),
                    webhook_concurrency_limit,
                    pg_timestamp_oracle_config,
                    check_cluster_scheduling_policies_interval: check_scheduling_policies_interval,
                    cluster_scheduling_decisions: BTreeMap::new(),
                    caught_up_check_interval: clusters_caught_up_check_interval,
                    caught_up_check: clusters_caught_up_check,
                    installed_watch_sets: BTreeMap::new(),
                    connection_watch_sets: BTreeMap::new(),
                    cluster_replica_statuses: ClusterReplicaStatuses::new(),
                    read_only_controllers,
                    buffered_builtin_table_updates: Some(Vec::new()),
                };
                let bootstrap = handle.block_on(async {
                    coord
                        .bootstrap(
                            boot_ts,
                            migrated_storage_collections_0dt,
                            builtin_table_updates,
                            cached_global_exprs,
                            uncached_local_exprs,
                        )
                        .await?;
                    coord
                        .controller
                        .remove_orphaned_replicas(
                            coord.catalog().get_next_user_replica_id().await?,
                            coord.catalog().get_next_system_replica_id().await?,
                        )
                        .await
                        .map_err(AdapterError::Orchestrator)?;

                    if let Some(retention_period) = storage_usage_retention_period {
                        coord
                            .prune_storage_usage_events_on_startup(retention_period)
                            .await;
                    }

                    Ok(())
                });
                let ok = bootstrap.is_ok();
                drop(span);
                bootstrap_tx
                    .send(bootstrap)
                    .expect("bootstrap_rx is not dropped until it receives this message");
                if ok {
                    handle.block_on(coord.serve(
                        internal_cmd_rx,
                        strict_serializable_reads_rx,
                        cmd_rx,
                        group_commit_rx,
                    ));
                }
            })
            .expect("failed to create coordinator thread");
        match bootstrap_rx
            .await
            .expect("bootstrap_tx always sends a message or panics/halts")
        {
            Ok(()) => {
                info!(
                    "startup: coordinator init: coordinator thread start complete in {:?}",
                    coord_thread_start.elapsed()
                );
                info!(
                    "startup: coordinator init: complete in {:?}",
                    coord_start.elapsed()
                );
                let handle = Handle {
                    session_id,
                    start_instant,
                    _thread: thread.join_on_drop(),
                };
                let client = Client::new(
                    build_info,
                    cmd_tx.clone(),
                    metrics_clone,
                    now,
                    environment_id,
                    segment_client_clone,
                );
                Ok((handle, client))
            }
            Err(e) => Err(e),
        }
    }
    .boxed()
}

// Determines and returns the highest timestamp for each timeline, for all known
// timestamp oracle implementations.
//
// Initially, we did this so that we can switch between implementations of
// timestamp oracle, but now we also do this to determine a monotonic boot
// timestamp, a timestamp that does not regress across reboots.
//
// This mostly works, but there can be linearizability violations, because there
// is no central moment where we do distributed coordination for all oracle
// types. Working around this seems prohibitively hard, maybe even impossible so
// we have to live with this window of potential violations during the upgrade
// window (which is the only point where we should switch oracle
// implementations).
async fn get_initial_oracle_timestamps(
    pg_timestamp_oracle_config: &Option<PostgresTimestampOracleConfig>,
) -> Result<BTreeMap<Timeline, Timestamp>, AdapterError> {
    let mut initial_timestamps = BTreeMap::new();

    if let Some(pg_timestamp_oracle_config) = pg_timestamp_oracle_config {
        let postgres_oracle_timestamps =
            PostgresTimestampOracle::<NowFn>::get_all_timelines(pg_timestamp_oracle_config.clone())
                .await?;

        let debug_msg = || {
            postgres_oracle_timestamps
                .iter()
                .map(|(timeline, ts)| format!("{:?} -> {}", timeline, ts))
                .join(", ")
        };
        info!(
            "current timestamps from the postgres-backed timestamp oracle: {}",
            debug_msg()
        );

        for (timeline, ts) in postgres_oracle_timestamps {
            let entry = initial_timestamps
                .entry(Timeline::from_str(&timeline).expect("could not parse timeline"));

            entry
                .and_modify(|current_ts| *current_ts = std::cmp::max(*current_ts, ts))
                .or_insert(ts);
        }
    } else {
        info!("no postgres url for postgres-backed timestamp oracle configured!");
    };

    let debug_msg = || {
        initial_timestamps
            .iter()
            .map(|(timeline, ts)| format!("{:?}: {}", timeline, ts))
            .join(", ")
    };
    info!("initial oracle timestamps: {}", debug_msg());

    Ok(initial_timestamps)
}

#[instrument]
pub async fn load_remote_system_parameters(
    storage: &mut Box<dyn OpenableDurableCatalogState>,
    system_parameter_sync_config: Option<SystemParameterSyncConfig>,
    system_parameter_sync_timeout: Duration,
) -> Result<Option<BTreeMap<String, String>>, AdapterError> {
    if let Some(system_parameter_sync_config) = system_parameter_sync_config {
        tracing::info!("parameter sync on boot: start sync");

        // We intentionally block initial startup, potentially forever,
        // on initializing LaunchDarkly. This may seem scary, but the
        // alternative is even scarier. Over time, we expect that the
        // compiled-in default values for the system parameters will
        // drift substantially from the defaults configured in
        // LaunchDarkly, to the point that starting an environment
        // without loading the latest values from LaunchDarkly will
        // result in running an untested configuration.
        //
        // Note this only applies during initial startup. Restarting
        // after we've synced once only blocks for a maximum of
        // `FRONTEND_SYNC_TIMEOUT` on LaunchDarkly, as it seems
        // reasonable to assume that the last-synced configuration was
        // valid enough.
        //
        // This philosophy appears to provide a good balance between not
        // running untested configurations in production while also not
        // making LaunchDarkly a "tier 1" dependency for existing
        // environments.
        //
        // If this proves to be an issue, we could seek to address the
        // configuration drift in a different way--for example, by
        // writing a script that runs in CI nightly and checks for
        // deviation between the compiled Rust code and LaunchDarkly.
        //
        // If it is absolutely necessary to bring up a new environment
        // while LaunchDarkly is down, the following manual mitigation
        // can be performed:
        //
        //    1. Edit the environmentd startup parameters to omit the
        //       LaunchDarkly configuration.
        //    2. Boot environmentd.
        //    3. Use the catalog-debug tool to run `edit config "{\"key\":\"system_config_synced\"}" "{\"value\": 1}"`.
        //    4. Adjust any other parameters as necessary to avoid
        //       running a nonstandard configuration in production.
        //    5. Edit the environmentd startup parameters to restore the
        //       LaunchDarkly configuration, for when LaunchDarkly comes
        //       back online.
        //    6. Reboot environmentd.
        let mut params = SynchronizedParameters::new(SystemVars::default());
        let frontend_sync = async {
            let frontend = SystemParameterFrontend::from(&system_parameter_sync_config).await?;
            frontend.pull(&mut params);
            let ops = params
                .modified()
                .into_iter()
                .map(|param| {
                    let name = param.name;
                    let value = param.value;
                    tracing::info!(name, value, initial = true, "sync parameter");
                    (name, value)
                })
                .collect();
            tracing::info!("parameter sync on boot: end sync");
            Ok(Some(ops))
        };
        if !storage.has_system_config_synced_once().await? {
            frontend_sync.await
        } else {
            match mz_ore::future::timeout(system_parameter_sync_timeout, frontend_sync).await {
                Ok(ops) => Ok(ops),
                Err(TimeoutError::Inner(e)) => Err(e),
                Err(TimeoutError::DeadlineElapsed) => {
                    tracing::info!("parameter sync on boot: sync has timed out");
                    Ok(None)
                }
            }
        }
    } else {
        Ok(None)
    }
}

#[derive(Debug)]
pub(crate) enum WatchSetResponse {
    StatementDependenciesReady(StatementLoggingId, StatementLifecycleEvent),
    AlterSinkReady(AlterSinkReadyContext),
}

#[derive(Debug)]
pub(crate) struct AlterSinkReadyContext {
    ctx: Option<ExecuteContext>,
    otel_ctx: OpenTelemetryContext,
    plan: AlterSinkPlan,
    plan_validity: PlanValidity,
    resolved_ids: ResolvedIds,
    read_hold: ReadHolds<Timestamp>,
}

impl AlterSinkReadyContext {
    fn ctx(&mut self) -> &mut ExecuteContext {
        self.ctx.as_mut().expect("only cleared on drop")
    }

    fn retire(mut self, result: Result<ExecuteResponse, AdapterError>) {
        self.ctx
            .take()
            .expect("only cleared on drop")
            .retire(result);
    }
}

impl Drop for AlterSinkReadyContext {
    fn drop(&mut self) {
        if let Some(ctx) = self.ctx.take() {
            ctx.retire(Err(AdapterError::Canceled));
        }
    }
}

/// A struct for tracking the ownership of a lock and a VecDeque to store to-be-done work after the
/// lock is freed.
#[derive(Debug)]
struct LockedVecDeque<T> {
    items: VecDeque<T>,
    lock: Arc<tokio::sync::Mutex<()>>,
}

impl<T> LockedVecDeque<T> {
    pub fn new() -> Self {
        Self {
            items: VecDeque::new(),
            lock: Arc::new(tokio::sync::Mutex::new(())),
        }
    }

    pub fn try_lock_owned(&self) -> Result<OwnedMutexGuard<()>, tokio::sync::TryLockError> {
        Arc::clone(&self.lock).try_lock_owned()
    }

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

    pub fn push_back(&mut self, value: T) {
        self.items.push_back(value)
    }

    pub fn pop_front(&mut self) -> Option<T> {
        self.items.pop_front()
    }

    pub fn remove(&mut self, index: usize) -> Option<T> {
        self.items.remove(index)
    }

    pub fn iter(&self) -> std::collections::vec_deque::Iter<'_, T> {
        self.items.iter()
    }
}

#[derive(Debug)]
struct DeferredPlanStatement {
    ctx: ExecuteContext,
    ps: PlanStatement,
}

#[derive(Debug)]
enum PlanStatement {
    Statement {
        stmt: Arc<Statement<Raw>>,
        params: Params,
    },
    Plan {
        plan: mz_sql::plan::Plan,
        resolved_ids: ResolvedIds,
    },
}

#[derive(Debug, Error)]
pub enum NetworkPolicyError {
    #[error("Access denied for address {0}")]
    AddressDenied(IpAddr),
    #[error("Access denied missing IP address")]
    MissingIp,
}

pub(crate) fn validate_ip_with_policy_rules(
    ip: &IpAddr,
    rules: &Vec<NetworkPolicyRule>,
) -> Result<(), NetworkPolicyError> {
    // At the moment we're not handling action or direction
    // as those are only able to be "allow" and "ingress" respectively
    if rules.iter().any(|r| r.address.0.contains(ip)) {
        Ok(())
    } else {
        Err(NetworkPolicyError::AddressDenied(ip.clone()))
    }
}