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

//! A controller that provides an interface to the storage layer.
//!
//! The storage controller curates the creation of sources, the progress of readers through these collections,
//! and their eventual dropping and resource reclamation.
//!
//! The storage controller can be viewed as a partial map from `GlobalId` to collection. It is an error to
//! use an identifier before it has been "created" with `create_source()`. Once created, the controller holds
//! a read capability for each source, which is manipulated with `update_read_capabilities()`.
//! Eventually, the source is dropped with either `drop_sources()` or by allowing compaction to the
//! empty frontier.

use std::any::Any;
use std::collections::{BTreeMap, BTreeSet};
use std::error::Error;
use std::fmt::{self, Debug};
use std::num::NonZeroI64;
use std::str::FromStr;
use std::sync::Arc;

use anyhow::Context;
use async_trait::async_trait;
use bytes::BufMut;
use derivative::Derivative;
use differential_dataflow::lattice::Lattice;
use itertools::Itertools;
use proptest::prelude::{any, Arbitrary, BoxedStrategy, Strategy};
use proptest_derive::Arbitrary;
use prost::Message;
use serde::{Deserialize, Serialize};
use timely::order::{PartialOrder, TotalOrder};
use timely::progress::frontier::{AntichainRef, MutableAntichain};
use timely::progress::{Antichain, ChangeBatch, Timestamp};
use tokio_stream::StreamMap;
use tracing::{debug, info};

use mz_build_info::BuildInfo;
use mz_cluster_client::client::ClusterReplicaLocation;
use mz_ore::metrics::MetricsRegistry;
use mz_ore::now::{EpochMillis, NowFn};
use mz_ore::soft_assert;
use mz_persist_client::cache::PersistClientCache;
use mz_persist_client::critical::SinceHandle;
use mz_persist_client::write::WriteHandle;
use mz_persist_client::{PersistClient, PersistLocation, ShardId};
use mz_persist_types::codec_impls::UnitSchema;
use mz_persist_types::{Codec64, Opaque};
use mz_proto::{IntoRustIfSome, ProtoType, RustType, TryFromProtoError};
use mz_repr::{Datum, Diff, GlobalId, RelationDesc, Row, TimestampManipulation};
use mz_stash::{self, AppendBatch, StashError, StashFactory, TypedCollection};

use crate::client::{
    CreateSinkCommand, CreateSourceCommand, ProtoStorageCommand, ProtoStorageResponse,
    SinkStatisticsUpdate, SourceStatisticsUpdate, StorageCommand, StorageResponse, Update,
};
use crate::controller::rehydration::RehydratingStorageClient;
use crate::healthcheck;
use crate::metrics::StorageControllerMetrics;
use crate::types::errors::DataflowError;
use crate::types::instances::StorageInstanceId;
use crate::types::parameters::StorageParameters;
use crate::types::sinks::{
    MetadataUnfilled, ProtoDurableExportMetadata, SinkAsOf, StorageSinkDesc,
};
use crate::types::sources::{IngestionDescription, SourceData, SourceEnvelope, SourceExport};

mod collection_mgmt;
mod command_wals;
mod persist_handles;
mod rehydration;
mod remap_migration;
mod statistics;

include!(concat!(env!("OUT_DIR"), "/mz_storage_client.controller.rs"));

pub static METADATA_COLLECTION: TypedCollection<GlobalId, DurableCollectionMetadata> =
    TypedCollection::new("storage-collection-metadata");

pub static METADATA_EXPORT: TypedCollection<GlobalId, DurableExportMetadata<mz_repr::Timestamp>> =
    TypedCollection::new("storage-export-metadata-u64");

pub static ALL_COLLECTIONS: &[&str] = &[
    METADATA_COLLECTION.name(),
    METADATA_EXPORT.name(),
    command_wals::SHARD_FINALIZATION.name(),
];

// Do this dance so that we keep the storage controller expressed in terms of a generic timestamp `T`.
struct MetadataExportFetcher;
trait MetadataExport<T>
where
    // Associated type would be better but you can't express this relationship without unstable
    DurableExportMetadata<T>: mz_stash::Data,
{
    fn get_stash_collection() -> &'static TypedCollection<GlobalId, DurableExportMetadata<T>>;
}

impl MetadataExport<mz_repr::Timestamp> for MetadataExportFetcher {
    fn get_stash_collection(
    ) -> &'static TypedCollection<GlobalId, DurableExportMetadata<mz_repr::Timestamp>> {
        &METADATA_EXPORT
    }
}

#[derive(Clone, Copy, Debug, Serialize, Deserialize, Eq, PartialEq, Hash, PartialOrd, Ord)]
pub enum IntrospectionType {
    /// We're not responsible for appending to this collection automatically, but we should
    /// automatically bump the write frontier from time to time.
    SinkStatusHistory,
    SourceStatusHistory,
    ShardMapping,

    // Note that this single-shard introspection source will be changed to per-replica,
    // once we allow multiplexing multiple sources/sinks on a single cluster.
    StorageSourceStatistics,
    StorageSinkStatistics,
}

/// Describes how data is written to the collection.
#[derive(Clone, Debug, Eq, PartialEq)]
pub enum DataSource {
    /// Ingest data from some external source.
    Ingestion(IngestionDescription),
    /// Data comes from introspection sources, which the controller itself is
    /// responsible for generating.
    Introspection(IntrospectionType),
    /// Data comes from the source's remapping/reclock operator.
    Progress,
    /// This source's data is does not need to be managed by the storage
    /// controller, e.g. it's a materialized view, table, or subsource.
    // TODO? Add a means to track some data sources' GlobalIds.
    Other,
}

/// Describes a request to create a source.
#[derive(Clone, Debug, Eq, PartialEq)]
pub struct CollectionDescription<T> {
    /// The schema of this collection
    pub desc: RelationDesc,
    /// The source of this collection's data.
    pub data_source: DataSource,
    /// An optional frontier to which the collection's `since` should be advanced.
    pub since: Option<Antichain<T>>,
    /// A GlobalId to use for this collection to use for the status collection.
    /// Used to keep track of source status/error information.
    pub status_collection_id: Option<GlobalId>,
}

impl<T> CollectionDescription<T> {
    /// Returns IDs for all storage objects that this `CollectionDescription`
    /// depends on.
    ///
    /// TODO: @sean: This is where the remap shard would slot in.
    fn get_storage_dependencies(&self) -> Vec<GlobalId> {
        let mut result = Vec::new();

        // NOTE: Exhaustive match for future proofing.
        match &self.data_source {
            DataSource::Ingestion(ingestion) => {
                match &ingestion.desc.envelope {
                    SourceEnvelope::Debezium(envelope_debezium) => {
                        let tx_metadata_topic = &envelope_debezium.dedup.tx_metadata;
                        if let Some(tx_input) = tx_metadata_topic {
                            result.push(tx_input.tx_metadata_global_id);
                        }
                    }
                    // NOTE: We explicitly list envelopes instead of using a catch all to
                    // make sure that we change this when adding/removing and envelope.
                    SourceEnvelope::None(_) | SourceEnvelope::Upsert(_) | SourceEnvelope::CdcV2 => {
                        // No storage dependencies.
                    }
                }
                result.push(ingestion.remap_collection_id);
            }
            DataSource::Introspection(_) | DataSource::Progress => {
                // Introspection, Progress sources have no dependencies, for
                // now.
            }
            DataSource::Other => {
                // We don't know anything about it's dependencies.
            }
        }

        result
    }
}

impl<T> From<RelationDesc> for CollectionDescription<T> {
    fn from(desc: RelationDesc) -> Self {
        Self {
            desc,
            data_source: DataSource::Other,
            since: None,
            status_collection_id: None,
        }
    }
}

#[derive(Clone, Debug, Eq, PartialEq)]
pub struct ExportDescription<T = mz_repr::Timestamp> {
    pub sink: StorageSinkDesc<MetadataUnfilled, T>,
    /// The ID of the instance in which to install the export.
    pub instance_id: StorageInstanceId,
}

/// Opaque token to ensure `prepare_export` is called before `create_exports`.  This token proves
/// that compaction is being held back on `from_id` at least until `id` is created.  It should be
/// held while the AS OF is determined.
#[derive(Clone, Debug, Eq, PartialEq)]
pub struct CreateExportToken<T = mz_repr::Timestamp> {
    id: GlobalId,
    from_id: GlobalId,
    acquired_since: Antichain<T>,
}

impl CreateExportToken {
    /// Returns the ID of the export with which the token is associated.
    pub fn id(&self) -> GlobalId {
        self.id
    }
}

#[async_trait(?Send)]
pub trait StorageController: Debug + Send {
    type Timestamp;

    /// Marks the end of any initialization commands.
    ///
    /// The implementor may wait for this method to be called before implementing prior commands,
    /// and so it is important for a user to invoke this method as soon as it is comfortable.
    /// This method can be invoked immediately, at the potential expense of performance.
    fn initialization_complete(&mut self);

    /// Update storage configuration.
    fn update_configuration(&mut self, config_params: StorageParameters);

    /// Acquire an immutable reference to the collection state, should it exist.
    fn collection(&self, id: GlobalId) -> Result<&CollectionState<Self::Timestamp>, StorageError>;

    /// Creates a storage instance with the specified ID.
    ///
    /// A storage instance can have zero or one replicas. The instance is
    /// created with zero replicas.
    ///
    /// Panics if a storage instance with the given ID already exists.
    fn create_instance(&mut self, id: StorageInstanceId);

    /// Drops the storage instance with the given ID.
    ///
    /// If you call this method while the storage instance has a replica
    /// attached, that replica will be leaked. Call `drop_replica` first.
    ///
    /// Panics if a storage instance with the given ID does not exist.
    fn drop_instance(&mut self, id: StorageInstanceId);

    /// Connects the storage instance to the specified replica.
    ///
    /// If the storage instance is already attached to a replica, communication
    /// with that replica is severed in favor of the new replica.
    ///
    /// In the future, this API will be adjusted to support active replication
    /// of storage instances (i.e., multiple replicas attached to a given
    /// storage instance).
    fn connect_replica(&mut self, id: StorageInstanceId, location: ClusterReplicaLocation);

    /// Acquire a mutable reference to the collection state, should it exist.
    fn collection_mut(
        &mut self,
        id: GlobalId,
    ) -> Result<&mut CollectionState<Self::Timestamp>, StorageError>;

    /// Acquire an iterator over all collection states.
    fn collections(
        &self,
    ) -> Box<dyn Iterator<Item = (&GlobalId, &CollectionState<Self::Timestamp>)> + '_>;

    /// Migrate any storage controller state from previous versions to this
    /// version's expectations.
    ///
    /// This function must "see" the GlobalId of every collection you plan to
    /// create, but can be called with all of the catalog's collections at once.
    async fn migrate_collections(
        &mut self,
        collections: Vec<(GlobalId, CollectionDescription<Self::Timestamp>)>,
    ) -> Result<(), StorageError>;

    /// Create the sources described in the individual CreateSourceCommand commands.
    ///
    /// Each command carries the source id, the source description, and any associated metadata
    /// needed to ingest the particular source.
    ///
    /// This command installs collection state for the indicated sources, and the are
    /// now valid to use in queries at times beyond the initial `since` frontiers. Each
    /// collection also acquires a read capability at this frontier, which will need to
    /// be repeatedly downgraded with `allow_compaction()` to permit compaction.
    ///
    /// This method is NOT idempotent; It can fail between processing of different
    /// collections and leave the controller in an inconsistent state. It is almost
    /// always wrong to do anything but abort the process on `Err`.
    async fn create_collections(
        &mut self,
        collections: Vec<(GlobalId, CollectionDescription<Self::Timestamp>)>,
    ) -> Result<(), StorageError>;

    /// Acquire an immutable reference to the export state, should it exist.
    fn export(&self, id: GlobalId) -> Result<&ExportState<Self::Timestamp>, StorageError>;

    /// Acquire a mutable reference to the export state, should it exist.
    fn export_mut(
        &mut self,
        id: GlobalId,
    ) -> Result<&mut ExportState<Self::Timestamp>, StorageError>;

    /// Create the sinks described by the `ExportDescription`.
    async fn create_exports(
        &mut self,
        exports: Vec<(
            CreateExportToken<Self::Timestamp>,
            ExportDescription<Self::Timestamp>,
        )>,
    ) -> Result<(), StorageError>;

    /// Notify the storage controller to prepare for an export to be created
    fn prepare_export(
        &mut self,
        id: GlobalId,
        from_id: GlobalId,
    ) -> Result<CreateExportToken<Self::Timestamp>, StorageError>;

    /// Cancel the pending export
    fn cancel_prepare_export(&mut self, token: CreateExportToken<Self::Timestamp>);

    /// Drops the read capability for the sources and allows their resources to be reclaimed.
    fn drop_sources(&mut self, identifiers: Vec<GlobalId>) -> Result<(), StorageError>;

    /// Drops the read capability for the sinks and allows their resources to be reclaimed.
    fn drop_sinks(&mut self, identifiers: Vec<GlobalId>) -> Result<(), StorageError>;

    /// Drops the read capability for the sinks and allows their resources to be reclaimed.
    ///
    /// TODO(jkosh44): This method does not validate the provided identifiers. Currently when the
    ///     controller starts/restarts it has no durable state. That means that it has no way of
    ///     remembering any past commands sent. In the future we plan on persisting state for the
    ///     controller so that it is aware of past commands.
    ///     Therefore this method is for dropping sinks that we know to have been previously
    ///     created, but have been forgotten by the controller due to a restart.
    ///     Once command history becomes durable we can remove this method and use the normal
    ///     `drop_sinks`.
    fn drop_sinks_unvalidated(&mut self, identifiers: Vec<GlobalId>);

    /// Drops the read capability for the sources and allows their resources to be reclaimed.
    ///
    /// TODO(jkosh44): This method does not validate the provided identifiers. Currently when the
    ///     controller starts/restarts it has no durable state. That means that it has no way of
    ///     remembering any past commands sent. In the future we plan on persisting state for the
    ///     controller so that it is aware of past commands.
    ///     Therefore this method is for dropping sources that we know to have been previously
    ///     created, but have been forgotten by the controller due to a restart.
    ///     Once command history becomes durable we can remove this method and use the normal
    ///     `drop_sources`.
    fn drop_sources_unvalidated(&mut self, identifiers: Vec<GlobalId>);

    /// Append `updates` into the local input named `id` and advance its upper to `upper`.
    ///
    /// The method returns a oneshot that can be awaited to indicate completion of the write.
    /// The method may return an error, indicating an immediately visible error, and also the
    /// oneshot may return an error if one is encountered during the write.
    // TODO(petrosagg): switch upper to `Antichain<Timestamp>`
    fn append(
        &mut self,
        commands: Vec<(GlobalId, Vec<Update<Self::Timestamp>>, Self::Timestamp)>,
    ) -> Result<tokio::sync::oneshot::Receiver<Result<(), StorageError>>, StorageError>;

    /// Returns the snapshot of the contents of the local input named `id` at `as_of`.
    async fn snapshot(
        &self,
        id: GlobalId,
        as_of: Self::Timestamp,
    ) -> Result<Vec<(Row, Diff)>, StorageError>;

    /// Assigns a read policy to specific identifiers.
    ///
    /// The policies are assigned in the order presented, and repeated identifiers should
    /// conclude with the last policy. Changing a policy will immediately downgrade the read
    /// capability if appropriate, but it will not "recover" the read capability if the prior
    /// capability is already ahead of it.
    ///
    /// The `StorageController` may include its own overrides on these policies.
    ///
    /// Identifiers not present in `policies` retain their existing read policies.
    fn set_read_policy(&mut self, policies: Vec<(GlobalId, ReadPolicy<Self::Timestamp>)>);

    /// Ingests write frontier updates for collections that this controller
    /// maintains and potentially generates updates to read capabilities, which
    /// are passed on to [`StorageController::update_read_capabilities`].
    ///
    /// These updates come from the entity that is responsible for writing to
    /// the collection, and in turn advancing its `upper` (aka
    /// `write_frontier`). The most common such "writers" are:
    ///
    /// * `clusterd` instances, for source ingestions
    ///
    /// * introspection collections (which this controller writes to)
    ///
    /// * Tables (which are written to by this controller)
    ///
    /// * Materialized Views, which are running inside COMPUTE, and for which
    /// COMPUTE sends updates to this storage controller
    ///
    /// The so-called "implied capability" is a read capability for a collection
    /// that is updated based on the write frontier and the collections
    /// [`ReadPolicy`]. Advancing the write frontier might change this implied
    /// capability, which in turn might change the overall `since` (a
    /// combination of all read capabilities) of a collection.
    fn update_write_frontiers(&mut self, updates: &[(GlobalId, Antichain<Self::Timestamp>)]);

    /// Applies `updates` and sends any appropriate compaction command.
    fn update_read_capabilities(
        &mut self,
        updates: &mut BTreeMap<GlobalId, ChangeBatch<Self::Timestamp>>,
    );

    /// Waits until the controller is ready to process a response.
    ///
    /// This method may block for an arbitrarily long time.
    ///
    /// When the method returns, the owner should call
    /// [`StorageController::process`] to process the ready message.
    ///
    /// This method is cancellation safe.
    async fn ready(&mut self);

    /// Processes the work queued by [`StorageController::ready`].
    ///
    /// This method is guaranteed to return "quickly" unless doing so would
    /// compromise the correctness of the system.
    ///
    /// This method is **not** guaranteed to be cancellation safe. It **must**
    /// be awaited to completion.
    async fn process(&mut self) -> Result<(), anyhow::Error>;

    /// Signal to the controller that the adapter has populated all of its
    /// initial state and the controller can reconcile (i.e. drop) any unclaimed
    /// resources.
    async fn reconcile_state(&mut self);
}

/// Compaction policies for collections maintained by `Controller`.
///
/// NOTE(benesch): this might want to live somewhere besides the storage crate,
/// because it is fundamental to both storage and compute.
#[derive(Clone, Derivative)]
#[derivative(Debug)]
pub enum ReadPolicy<T> {
    /// No-one has yet requested a `ReadPolicy` from us, which means that we can
    /// still change the implied_capability/the collection since if we need
    /// to.
    NoPolicy { initial_since: Antichain<T> },
    /// Maintain the collection as valid from this frontier onward.
    ValidFrom(Antichain<T>),
    /// Maintain the collection as valid from a function of the write frontier.
    ///
    /// This function will only be re-evaluated when the write frontier changes.
    /// If the intended behavior is to change in response to external signals,
    /// consider using the `ValidFrom` variant to manually pilot compaction.
    ///
    /// The `Arc` makes the function cloneable.
    LagWriteFrontier(
        #[derivative(Debug = "ignore")] Arc<dyn Fn(AntichainRef<T>) -> Antichain<T> + Send + Sync>,
    ),
    /// Allows one to express multiple read policies, taking the least of
    /// the resulting frontiers.
    Multiple(Vec<ReadPolicy<T>>),
}

impl<T> ReadPolicy<T>
where
    T: Timestamp + TimestampManipulation,
{
    /// Creates a read policy that lags the write frontier "by one".
    pub fn step_back() -> Self {
        Self::LagWriteFrontier(Arc::new(move |upper| {
            if upper.is_empty() {
                Antichain::from_elem(Timestamp::minimum())
            } else {
                let stepped_back = upper
                    .to_owned()
                    .into_iter()
                    .map(|time| {
                        if time == T::minimum() {
                            time
                        } else {
                            time.step_back().unwrap()
                        }
                    })
                    .collect_vec();
                stepped_back.into()
            }
        }))
    }
}

impl ReadPolicy<mz_repr::Timestamp> {
    /// Creates a read policy that lags the write frontier by the indicated amount, rounded down to (at most) the specified value.
    /// The rounding down is done to reduce the number of changes the capability undergoes.
    pub fn lag_writes_by(lag: mz_repr::Timestamp, max_granularity: mz_repr::Timestamp) -> Self {
        Self::LagWriteFrontier(Arc::new(move |upper| {
            if upper.is_empty() {
                Antichain::from_elem(Timestamp::minimum())
            } else {
                // Subtract the lag from the time, and then round down to a multiple of `granularity` to cut chatter.
                let mut time = upper[0];
                if lag != mz_repr::Timestamp::default() {
                    time = time.saturating_sub(lag);
                    // It makes little sense to refuse to compact if the user genuinely
                    // sets a smaller compaction window than the default, so honor it here.
                    let granularity = std::cmp::min(lag, max_granularity);
                    time = time.saturating_sub(time % granularity);
                }
                Antichain::from_elem(time)
            }
        }))
    }
}

impl<T: Timestamp> ReadPolicy<T> {
    pub fn frontier(&self, write_frontier: AntichainRef<T>) -> Antichain<T> {
        match self {
            ReadPolicy::NoPolicy { initial_since } => initial_since.clone(),
            ReadPolicy::ValidFrom(frontier) => frontier.clone(),
            ReadPolicy::LagWriteFrontier(logic) => logic(write_frontier),
            ReadPolicy::Multiple(policies) => {
                let mut frontier = Antichain::new();
                for policy in policies.iter() {
                    for time in policy.frontier(write_frontier).iter() {
                        frontier.insert(time.clone());
                    }
                }
                frontier
            }
        }
    }
}

/// Metadata required by a storage instance to read a storage collection
#[derive(Arbitrary, Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct CollectionMetadata {
    /// The persist location where the shards are located.
    pub persist_location: PersistLocation,
    /// The persist shard id of the remap collection used to reclock this collection.
    pub remap_shard: Option<ShardId>,
    /// The persist shard containing the contents of this storage collection.
    pub data_shard: ShardId,
    /// The persist shard containing the status updates for this storage collection.
    pub status_shard: Option<ShardId>,
    /// The `RelationDesc` that describes the contents of the `data_shard`.
    pub relation_desc: RelationDesc,
}

impl RustType<ProtoCollectionMetadata> for CollectionMetadata {
    fn into_proto(&self) -> ProtoCollectionMetadata {
        ProtoCollectionMetadata {
            blob_uri: self.persist_location.blob_uri.clone(),
            consensus_uri: self.persist_location.consensus_uri.clone(),
            data_shard: self.data_shard.to_string(),
            remap_shard: self.remap_shard.map(|s| s.to_string()),
            status_shard: self.status_shard.map(|s| s.to_string()),
            relation_desc: Some(self.relation_desc.into_proto()),
        }
    }

    fn from_proto(value: ProtoCollectionMetadata) -> Result<Self, TryFromProtoError> {
        Ok(CollectionMetadata {
            persist_location: PersistLocation {
                blob_uri: value.blob_uri,
                consensus_uri: value.consensus_uri,
            },
            remap_shard: value
                .remap_shard
                .map(|s| s.parse().map_err(TryFromProtoError::InvalidShardId))
                .transpose()?,
            data_shard: value
                .data_shard
                .parse()
                .map_err(TryFromProtoError::InvalidShardId)?,
            status_shard: value
                .status_shard
                .map(|s| s.parse().map_err(TryFromProtoError::InvalidShardId))
                .transpose()?,
            relation_desc: value
                .relation_desc
                .into_rust_if_some("ProtoCollectionMetadata::relation_desc")?,
        })
    }
}

/// A trait that is used to calculate safe _resumption frontiers_ for a source.
///
/// Use [`ResumptionFrontierCalculator::initialize_state`] for creating an
/// opaque state that you should keep around. Then repeatedly call
/// [`ResumptionFrontierCalculator::calculate_resumption_frontier`] with the
/// state to efficiently calculate an up-to-date frontier.
#[async_trait]
pub trait ResumptionFrontierCalculator<T> {
    /// Opaque state that a `ResumptionFrontierCalculator` needs to repeatedly
    /// (and efficiently) calculate a _resumption frontier_.
    type State;

    /// Creates an opaque state type that can be used to efficiently calculate a
    /// new _resumption frontier_ when needed.
    async fn initialize_state(&self, client_cache: &PersistClientCache) -> Self::State;

    /// Calculates a new, safe _resumption frontier_.
    async fn calculate_resumption_frontier(&self, state: &mut Self::State) -> Antichain<T>;
}

/// The subset of [`CollectionMetadata`] that must be durable stored.
#[derive(Arbitrary, Clone, Debug, PartialEq, PartialOrd, Ord, Eq, Serialize, Deserialize)]
pub struct DurableCollectionMetadata {
    // MIGRATION: v0.44 This field can be deleted in a future version of
    // Materialize because we are moving the relationship between a collection
    // and its remap shard into a relationship between a collection and its
    // remap collection, i.e. we will use another collection's data shard as our
    // remap shard, rendering this mapping duplicative.
    pub remap_shard: Option<ShardId>,
    pub data_shard: ShardId,
}

impl RustType<ProtoDurableCollectionMetadata> for DurableCollectionMetadata {
    fn into_proto(&self) -> ProtoDurableCollectionMetadata {
        ProtoDurableCollectionMetadata {
            remap_shard: self.remap_shard.into_proto(),
            data_shard: self.data_shard.to_string(),
        }
    }

    fn from_proto(value: ProtoDurableCollectionMetadata) -> Result<Self, TryFromProtoError> {
        Ok(DurableCollectionMetadata {
            remap_shard: value
                .remap_shard
                .map(|data_shard| {
                    data_shard
                        .parse()
                        .map_err(TryFromProtoError::InvalidShardId)
                })
                .transpose()?,
            data_shard: value
                .data_shard
                .parse()
                .map_err(TryFromProtoError::InvalidShardId)?,
        })
    }
}

#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct DurableExportMetadata<T> {
    pub initial_as_of: SinkAsOf<T>,
}

impl PartialOrd for DurableExportMetadata<mz_repr::Timestamp> {
    fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
        Some(self.cmp(other))
    }
}

impl std::cmp::Ord for DurableExportMetadata<mz_repr::Timestamp> {
    fn cmp(&self, other: &Self) -> std::cmp::Ordering {
        let mut s = vec![];
        let mut o = vec![];
        self.encode(&mut s);
        other.encode(&mut o);
        s.cmp(&o)
    }
}

impl RustType<ProtoDurableExportMetadata> for DurableExportMetadata<mz_repr::Timestamp> {
    fn into_proto(&self) -> ProtoDurableExportMetadata {
        ProtoDurableExportMetadata {
            initial_as_of: Some(self.initial_as_of.into_proto()),
        }
    }

    fn from_proto(proto: ProtoDurableExportMetadata) -> Result<Self, TryFromProtoError> {
        Ok(DurableExportMetadata {
            initial_as_of: proto
                .initial_as_of
                .into_rust_if_some("ProtoDurableExportMetadata::initial_as_of")?,
        })
    }
}

impl DurableExportMetadata<mz_repr::Timestamp> {
    pub fn encode<B: BufMut>(&self, buf: &mut B) {
        self.into_proto()
            .encode(buf)
            .expect("no required fields means no initialization errors");
    }

    pub fn decode(buf: &[u8]) -> Result<Self, String> {
        let proto = ProtoDurableExportMetadata::decode(buf).map_err(|err| err.to_string())?;
        proto.into_rust().map_err(|err| err.to_string())
    }
}

impl Arbitrary for DurableExportMetadata<mz_repr::Timestamp> {
    type Strategy = BoxedStrategy<Self>;
    type Parameters = ();

    fn arbitrary_with(_: Self::Parameters) -> Self::Strategy {
        (any::<SinkAsOf<mz_repr::Timestamp>>(),)
            .prop_map(|(initial_as_of,)| Self { initial_as_of })
            .boxed()
    }
}

/// Controller state maintained for each storage instance.
#[derive(Debug)]
pub struct StorageControllerState<T: Timestamp + Lattice + Codec64 + TimestampManipulation> {
    /// A function that returns the current time.
    now: NowFn,
    /// The fencing token for this instance of the controller.
    envd_epoch: NonZeroI64,

    /// Collections maintained by the storage controller.
    ///
    /// This collection only grows, although individual collections may be rendered unusable.
    /// This is to prevent the re-binding of identifiers to other descriptions.
    pub(super) collections: BTreeMap<GlobalId, CollectionState<T>>,
    pub(super) exports: BTreeMap<GlobalId, ExportState<T>>,
    pub(super) stash: mz_stash::Stash,
    /// Write handle for persist shards.
    pub(super) persist_write_handles: persist_handles::PersistWriteWorker<T>,
    /// Read handles for persist shards.
    ///
    /// These handles are on the other end of a Tokio task, so that work can be done asynchronously
    /// without blocking the storage controller.
    persist_read_handles: persist_handles::PersistReadWorker<T>,
    stashed_response: Option<StorageResponse<T>>,
    /// IDs of sources that were dropped whose statuses should be
    /// updated during the next call to `StorageController::process`.
    pending_source_drops: Vec<GlobalId>,
    /// IDs of sinks that were dropped whose statuses should be
    /// updated during the next call to `StorageController::process`.
    pending_sink_drops: Vec<GlobalId>,
    /// Compaction commands to send during the next call to
    /// `StorageController::process`.
    pending_compaction_commands: Vec<(GlobalId, Antichain<T>, Option<StorageInstanceId>)>,

    /// Interface for managed collections
    pub(super) collection_manager: collection_mgmt::CollectionManager,
    /// Tracks which collection is responsible for which [`IntrospectionType`].
    pub(super) introspection_ids: BTreeMap<IntrospectionType, GlobalId>,
    /// Tokens for tasks that drive updating introspection collections. Dropping
    /// this will make sure that any tasks (or other resources) will stop when
    /// needed.
    // TODO(aljoscha): Should these live somewhere else?
    introspection_tokens: BTreeMap<GlobalId, Box<dyn Any + Send + Sync>>,

    /// Consolidated metrics updates to periodically write. We do not eagerly initialize this,
    /// and its contents are entirely driven by `StorageResponse::StatisticsUpdates`'s.
    source_statistics:
        Arc<std::sync::Mutex<BTreeMap<GlobalId, BTreeMap<usize, SourceStatisticsUpdate>>>>,
    /// Consolidated metrics updates to periodically write. We do not eagerly initialize this,
    /// and its contents are entirely driven by `StorageResponse::StatisticsUpdates`'s.
    sink_statistics:
        Arc<std::sync::Mutex<BTreeMap<GlobalId, BTreeMap<usize, SinkStatisticsUpdate>>>>,

    /// Clients for all known storage instances.
    clients: BTreeMap<StorageInstanceId, RehydratingStorageClient<T>>,
    /// Set to `true` once `initialization_complete` has been called.
    initialized: bool,
    /// Storage configuration to apply to newly provisioned instances.
    config: StorageParameters,
}

/// A storage controller for a storage instance.
#[derive(Debug)]
pub struct Controller<T: Timestamp + Lattice + Codec64 + From<EpochMillis> + TimestampManipulation>
{
    /// The build information for this process.
    build_info: &'static BuildInfo,
    /// The state for the storage controller.
    /// TODO(benesch): why is this a separate struct?
    state: StorageControllerState<T>,
    /// Mechanism for returning frontier advancement for tables.
    internal_response_queue: tokio::sync::mpsc::UnboundedReceiver<StorageResponse<T>>,
    /// The persist location where all storage collections are being written to
    persist_location: PersistLocation,
    /// A persist client used to write to storage collections
    persist: Arc<PersistClientCache>,
    /// Metrics of the Storage controller
    metrics: StorageControllerMetrics,
}

#[derive(Debug)]
pub enum StorageError {
    /// The source identifier was re-created after having been dropped,
    /// or installed with a different description.
    SourceIdReused(GlobalId),
    /// The sink identifier was re-created after having been dropped, or
    /// installed with a different description.
    SinkIdReused(GlobalId),
    /// The source identifier is not present.
    IdentifierMissing(GlobalId),
    /// The update contained in the appended batch was at a timestamp equal or beyond the batch's upper
    UpdateBeyondUpper(GlobalId),
    /// The read was at a timestamp before the collection's since
    ReadBeforeSince(GlobalId),
    /// The expected upper of one or more appends was different from the actual upper of the collection
    InvalidUppers(Vec<GlobalId>),
    /// An error from the underlying client.
    ClientError(anyhow::Error),
    /// An operation failed to read or write state
    IOError(StashError),
    /// Dataflow was not able to process a request
    DataflowError(DataflowError),
    /// The controller API was used in some invalid way. This usually indicates
    /// a bug.
    InvalidUsage(String),
}

impl Error for StorageError {
    fn source(&self) -> Option<&(dyn Error + 'static)> {
        match self {
            Self::SourceIdReused(_) => None,
            Self::SinkIdReused(_) => None,
            Self::IdentifierMissing(_) => None,
            Self::UpdateBeyondUpper(_) => None,
            Self::ReadBeforeSince(_) => None,
            Self::InvalidUppers(_) => None,
            Self::ClientError(_) => None,
            Self::IOError(err) => Some(err),
            Self::DataflowError(err) => Some(err),
            Self::InvalidUsage(_) => None,
        }
    }
}

impl fmt::Display for StorageError {
    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
        f.write_str("storage error: ")?;
        match self {
            Self::SourceIdReused(id) => write!(
                f,
                "source identifier was re-created after having been dropped: {id}"
            ),
            Self::SinkIdReused(id) => write!(
                f,
                "sink identifier was re-created after having been dropped: {id}"
            ),
            Self::IdentifierMissing(id) => write!(f, "collection identifier is not present: {id}"),
            Self::UpdateBeyondUpper(id) => {
                write!(
                    f,
                    "append batch for {id} contained update at or beyond its upper"
                )
            }
            Self::ReadBeforeSince(id) => {
                write!(f, "read for {id} was at a timestamp before its since")
            }
            Self::InvalidUppers(id) => {
                write!(
                    f,
                    "expected upper was different from the actual upper for: {}",
                    id.iter().map(|id| id.to_string()).join(", ")
                )
            }
            Self::ClientError(err) => write!(f, "underlying client error: {:#}", err),
            Self::IOError(err) => write!(f, "failed to read or write state: {err}"),
            Self::DataflowError(err) => write!(f, "dataflow failed to process request: {err}"),
            Self::InvalidUsage(err) => write!(f, "invalid usage: {err}"),
        }
    }
}

impl From<anyhow::Error> for StorageError {
    fn from(error: anyhow::Error) -> Self {
        Self::ClientError(error)
    }
}

impl From<StashError> for StorageError {
    fn from(error: StashError) -> Self {
        Self::IOError(error)
    }
}

impl From<DataflowError> for StorageError {
    fn from(error: DataflowError) -> Self {
        Self::DataflowError(error)
    }
}

impl<T: Timestamp + Lattice + Codec64 + From<EpochMillis> + TimestampManipulation>
    StorageControllerState<T>
{
    pub(super) async fn new(
        postgres_url: String,
        tx: tokio::sync::mpsc::UnboundedSender<StorageResponse<T>>,
        now: NowFn,
        factory: &StashFactory,
        envd_epoch: NonZeroI64,
    ) -> Self {
        let tls = mz_postgres_util::make_tls(
            &tokio_postgres::config::Config::from_str(&postgres_url)
                .expect("invalid postgres url for storage stash"),
        )
        .expect("could not make storage TLS connection");
        let mut stash = factory
            .open(postgres_url, None, tls)
            .await
            .expect("could not connect to postgres storage stash");

        // Ensure all collections are initialized, otherwise they panic if
        // they're read before being written to.
        async fn maybe_get_init_batch<'tx, K, V>(
            tx: &'tx mz_stash::Transaction<'tx>,
            typed: &TypedCollection<K, V>,
        ) -> Option<AppendBatch>
        where
            K: mz_stash::Data,
            V: mz_stash::Data,
        {
            let collection = tx
                .collection::<K, V>(typed.name())
                .await
                .expect("named collection must exist");
            let upper = tx
                .upper(collection.id)
                .await
                .expect("collection known to exist");
            if upper.elements() == [mz_stash::Timestamp::MIN] {
                Some(
                    collection
                        .make_batch_lower(upper)
                        .expect("stash operation must succeed"),
                )
            } else {
                None
            }
        }

        stash
            .with_transaction(move |tx| {
                Box::pin(async move {
                    // Query all collections in parallel. Makes for triplicated
                    // names, but runs quick.
                    let (metadata_collection, metadata_export, shard_finalization) = futures::join!(
                        maybe_get_init_batch(&tx, &METADATA_COLLECTION),
                        maybe_get_init_batch(&tx, &METADATA_EXPORT),
                        maybe_get_init_batch(&tx, &command_wals::SHARD_FINALIZATION),
                    );
                    let batches: Vec<AppendBatch> =
                        [metadata_collection, metadata_export, shard_finalization]
                            .into_iter()
                            .filter_map(|b| b)
                            .collect();

                    tx.append(batches).await
                })
            })
            .await
            .expect("stash operation must succeed");

        let persist_write_handles = persist_handles::PersistWriteWorker::new(tx);
        let collection_manager_write_handle = persist_write_handles.clone();

        let collection_manager =
            collection_mgmt::CollectionManager::new(collection_manager_write_handle, now.clone());

        Self {
            collections: BTreeMap::default(),
            exports: BTreeMap::default(),
            stash,
            persist_write_handles,
            persist_read_handles: persist_handles::PersistReadWorker::new(),
            stashed_response: None,
            pending_source_drops: vec![],
            pending_sink_drops: vec![],
            pending_compaction_commands: vec![],
            collection_manager,
            introspection_ids: BTreeMap::new(),
            introspection_tokens: BTreeMap::new(),
            now,
            envd_epoch,
            source_statistics: Arc::new(std::sync::Mutex::new(BTreeMap::new())),
            sink_statistics: Arc::new(std::sync::Mutex::new(BTreeMap::new())),
            clients: BTreeMap::new(),
            initialized: false,
            config: StorageParameters::default(),
        }
    }
}

#[async_trait(?Send)]
impl<T> StorageController for Controller<T>
where
    T: Timestamp + Lattice + TotalOrder + Codec64 + From<EpochMillis> + TimestampManipulation,
    StorageCommand<T>: RustType<ProtoStorageCommand>,
    StorageResponse<T>: RustType<ProtoStorageResponse>,
    MetadataExportFetcher: MetadataExport<T>,
    DurableExportMetadata<T>: mz_stash::Data,
{
    type Timestamp = T;

    fn initialization_complete(&mut self) {
        self.state.initialized = true;
        for client in self.state.clients.values_mut() {
            client.send(StorageCommand::InitializationComplete);
        }
    }

    fn update_configuration(&mut self, config_params: StorageParameters) {
        config_params.persist.apply(self.persist.cfg());

        for client in self.state.clients.values_mut() {
            client.send(StorageCommand::UpdateConfiguration(config_params.clone()));
        }
        self.state.config.update(config_params);
    }

    fn collection(&self, id: GlobalId) -> Result<&CollectionState<Self::Timestamp>, StorageError> {
        self.state
            .collections
            .get(&id)
            .ok_or(StorageError::IdentifierMissing(id))
    }

    fn collection_mut(
        &mut self,
        id: GlobalId,
    ) -> Result<&mut CollectionState<Self::Timestamp>, StorageError> {
        self.state
            .collections
            .get_mut(&id)
            .ok_or(StorageError::IdentifierMissing(id))
    }

    fn collections(
        &self,
    ) -> Box<dyn Iterator<Item = (&GlobalId, &CollectionState<Self::Timestamp>)> + '_> {
        Box::new(self.state.collections.iter())
    }

    fn create_instance(&mut self, id: StorageInstanceId) {
        let mut client = RehydratingStorageClient::new(
            self.build_info,
            self.metrics.for_instance(id),
            self.state.envd_epoch,
        );
        if self.state.initialized {
            client.send(StorageCommand::InitializationComplete);
        }
        client.send(StorageCommand::UpdateConfiguration(
            self.state.config.clone(),
        ));
        let old_client = self.state.clients.insert(id, client);
        assert!(old_client.is_none(), "storage instance {id} already exists");
    }

    fn drop_instance(&mut self, id: StorageInstanceId) {
        let client = self.state.clients.remove(&id);
        assert!(client.is_some(), "storage instance {id} does not exist");
    }

    fn connect_replica(&mut self, id: StorageInstanceId, location: ClusterReplicaLocation) {
        let client = self
            .state
            .clients
            .get_mut(&id)
            .unwrap_or_else(|| panic!("instance {id} does not exist"));
        client.connect(location);
    }

    // Add new migrations below and precede them with a short summary of the
    // migration's purpose and optional additional commentary about safety or
    // approach.
    //
    // Note that:
    // - The sum of all migrations must be idempotent because all migrations run
    //   every time the catalog opens, unless migrations are explicitly
    //   disabled. This might mean changing code outside the migration itself,
    //   or only executing some migrations when encountering certain versions.
    // - Migrations must preserve backwards compatibility with all past releases
    //   of Materialize.
    #[tracing::instrument(level = "debug", skip_all)]
    async fn migrate_collections(
        &mut self,
        collections: Vec<(GlobalId, CollectionDescription<Self::Timestamp>)>,
    ) -> Result<(), StorageError> {
        let mut durable_metadata = METADATA_COLLECTION.peek_one(&mut self.state.stash).await?;

        // MIGRATION: v0.44 See comments on remap_shard_migration.
        let remap_shard_migration_delta =
            self.remap_shard_migration(&durable_metadata, &collections);

        self.upsert_collection_metadata(&mut durable_metadata, remap_shard_migration_delta)
            .await;

        Ok(())
    }

    // TODO(aljoscha): It would be swell if we could refactor this Leviathan of
    // a method/move individual parts to their own methods.
    #[tracing::instrument(level = "debug", skip_all)]
    async fn create_collections(
        &mut self,
        mut collections: Vec<(GlobalId, CollectionDescription<Self::Timestamp>)>,
    ) -> Result<(), StorageError> {
        // Validate first, to avoid corrupting state.
        // 1. create a dropped identifier, or
        // 2. create an existing identifier with a new description.
        // Make sure to check for errors within `ingestions` as well.
        collections.sort_by_key(|(id, _)| *id);
        collections.dedup();
        for pos in 1..collections.len() {
            if collections[pos - 1].0 == collections[pos].0 {
                return Err(StorageError::SourceIdReused(collections[pos].0));
            }
        }
        for (id, description) in collections.iter() {
            if let Ok(collection) = self.collection(*id) {
                if &collection.description != description {
                    return Err(StorageError::SourceIdReused(*id));
                }
            }
        }

        // Install collection state for each bound description. Note that this
        // method implementation attempts to do AS MUCH work concurrently as
        // possible. There are inline comments explaining the motivation behind
        // each section.
        let mut entries = Vec::with_capacity(collections.len());

        for (id, _desc) in &collections {
            entries.push((
                *id,
                DurableCollectionMetadata {
                    data_shard: ShardId::new(),
                    remap_shard: None,
                },
            ))
        }

        // Perform all stash writes in a single transaction, to minimize transaction overhead and
        // the time spent waiting for stash.
        METADATA_COLLECTION
            .insert_without_overwrite(&mut self.state.stash, entries.into_iter())
            .await?;

        let mut durable_metadata = METADATA_COLLECTION.peek_one(&mut self.state.stash).await?;

        // We first enrich each collection description with some additional metadata...
        use futures::stream::{StreamExt, TryStreamExt};
        let enriched_with_metadata = collections
            .into_iter()
            .map(|(id, description)| {
                let collection_shards = durable_metadata.remove(&id).expect("inserted above");
                // MIGRATION: v0.44
                assert!(collection_shards.remap_shard.is_none(), "remap shards must be migrated to be the data shard of their remap/progress collections or dropped");

                let status_shard =
                    if let Some(status_collection_id) = description.status_collection_id {
                        Some(
                            durable_metadata
                                .get(&status_collection_id)
                                .ok_or(StorageError::IdentifierMissing(status_collection_id))?
                                .data_shard,
                        )
                    } else {
                        None
                    };

                let remap_shard = match &description.data_source {
                    // Only ingestions can have remap shards.
                    DataSource::Ingestion(IngestionDescription {
                        remap_collection_id,
                        ..
                    }) => {
                        // Iff ingestion has a remap collection, its metadata must
                        // exist (and be correct) by this point.
                        Some(
                            durable_metadata
                                .get(remap_collection_id)
                                .ok_or(StorageError::IdentifierMissing(*remap_collection_id))?
                                .data_shard,
                        )
                    }
                    _ => None,
                };

                let metadata = CollectionMetadata {
                    persist_location: self.persist_location.clone(),
                    remap_shard,
                    data_shard: collection_shards.data_shard,
                    status_shard,
                    relation_desc: description.desc.clone(),
                };

                Ok((id, description, metadata))
            })
            .collect_vec();

        // So that we can open `SinceHandle`s for each collections concurrently.
        let persist_client = self
            .persist
            .open(self.persist_location.clone())
            .await
            .unwrap();
        let persist_client = &persist_client;
        // Reborrow the `&mut self` as immutable, as all the concurrent work to be processed in
        // this stream cannot all have exclusive access.
        let this = &*self;
        let to_register: Vec<_> = futures::stream::iter(enriched_with_metadata)
            .map(|data: Result<_, anyhow::Error>| async move {
                let (id, description, metadata) = data?;

                // should be replaced with real introspection (https://github.com/MaterializeInc/materialize/issues/14266)
                // but for now, it's helpful to have this mapping written down somewhere
                debug!(
                    "mapping GlobalId={} to remap shard ({:?}), data shard ({}), status shard ({:?})",
                    id, metadata.remap_shard, metadata.data_shard, metadata.status_shard
                );

                let (write, since_handle) = this
                    .open_data_handles(
                        format!("controller data {}", id).as_str(),
                        metadata.data_shard,
                        metadata.relation_desc.clone(),
                        persist_client,
                    )
                    .await;

                Ok::<_, anyhow::Error>((id, description, write, since_handle, metadata))
            })
            // Poll each future for each collection concurrently, maximum of 50 at a time.
            .buffer_unordered(50)
            // HERE BE DRAGONS:
            //
            // There are at least 2 subtleties in using `FuturesUnordered` (which
            // `buffer_unordered` uses underneath:
            // - One is captured here <https://github.com/rust-lang/futures-rs/issues/2387>
            // - And the other is deadlocking if processing an OUTPUT of a `FuturesUnordered`
            // stream attempts to obtain an async mutex that is also obtained in the futures
            // being polled.
            //
            // Both of these could potentially be issues in all usages of `buffer_unordered` in
            // this method, so we stick the standard advice: only use `try_collect` or
            // `collect`!
            .try_collect()
            .await?;

        let mut to_create = Vec::with_capacity(to_register.len());
        // This work mutates the controller state, so must be done serially. Because there
        // is no io-bound work, its very fast.
        for (id, description, write, since_handle, metadata) in to_register {
            let data_shard_since = since_handle.since().clone();

            let collection_state = CollectionState::new(
                description.clone(),
                data_shard_since,
                write.upper().clone(),
                vec![],
                metadata.clone(),
            );

            self.state.persist_write_handles.register(id, write);
            self.state.persist_read_handles.register(id, since_handle);

            self.state.collections.insert(id, collection_state);

            to_create.push((id, description));
        }

        // Patch up the since of all subsources (which includes the "main"
        // collection) and install read holds from the subsources on the since
        // of the remap collection. We need to do this here because a) the since
        // of the remap collection might be in advance of the since of the data
        // collections because we lazily forward commands to downgrade the since
        // to persist, and b) at the time the subsources are created we know
        // close to nothing about them, not even that they are subsources.
        //
        // N.B. Patching up the since based on the since of the remap collection
        // is correct because the since of the remap collection can advance iff
        // the storage controller allowed it to, which it only does when it
        // would also allow the since of the data collections to advance. It's
        // just that we need to reconcile outselves to the outside world
        // (persist) here.
        //
        // TODO(aljoscha): We should find a way to put this information and the
        // read holds in place when we create the subsource collections. OR, we
        // could create the subsource collections only as part of creating the
        // main source/ingestion.
        for (_id, description) in to_create.iter() {
            match &description.data_source {
                DataSource::Ingestion(ingestion) => {
                    let storage_dependencies = description.get_storage_dependencies();
                    let dependency_since =
                        self.determine_collection_since_joins(&storage_dependencies)?;

                    // Install read capability for all non-remap subsources on
                    // remap collection.
                    //
                    // N.B. The "main" collection of the source is included in
                    // `source_exports`.
                    for id in ingestion.source_exports.keys() {
                        let collection = self.collection(*id).expect("known to exist");

                        // At the time of collection creation, we did not yet
                        // have firm guarantees that the since of our
                        // dependencies was not advanced beyond those of its
                        // dependents, so we need to patch up the
                        // implied_capability/since of the collction.
                        //
                        // TODO(aljoscha): This comes largely from the fact that
                        // subsources are created with a `DataSource::Other`, so
                        // we have no idea (at their creation time) that they
                        // are a subsource, or that they are a subsource of a
                        // source where they need a read hold on that
                        // ingestion's remap collection.
                        if timely::order::PartialOrder::less_than(
                            &collection.implied_capability,
                            &dependency_since,
                        ) {
                            assert!(
                                timely::order::PartialOrder::less_than(
                                    &dependency_since,
                                    &collection.write_frontier
                                ),
                                "write frontier ({:?}) must be in advance dependency collection's since ({:?})",
                                collection.write_frontier,
                                dependency_since,
                            );
                            mz_ore::soft_assert!(
                                matches!(collection.read_policy, ReadPolicy::NoPolicy { .. }),
                                "subsources should not have external read holds installed until \
                                their ingestion is created, but {:?} has read policy {:?}",
                                id,
                                collection.read_policy
                            );

                            // This patches up the implied_capability!
                            self.set_read_policy(vec![(
                                *id,
                                ReadPolicy::NoPolicy {
                                    initial_since: dependency_since.clone(),
                                },
                            )]);

                            // We have to re-borrow.
                            let collection = self.collection(*id).expect("known to exist");
                            assert!(
                                collection.implied_capability == dependency_since,
                                "monkey patching the implied_capability to {:?} did not work, is still {:?}",
                                dependency_since,
                                collection.implied_capability,
                            );
                        }

                        // Fill in the storage dependencies.
                        let collection = self.collection_mut(*id).expect("known to exist");
                        collection
                            .storage_dependencies
                            .extend(storage_dependencies.iter().cloned());

                        assert!(
                            !PartialOrder::less_than(
                                &collection.read_capabilities.frontier(),
                                &collection.implied_capability.borrow()
                            ),
                            "{id}: at this point, there can be no read holds for any time that is not \
                            beyond the implied capability \
                            but we have implied_capability {:?}, read_capabilities {:?}",
                            collection.implied_capability,
                            collection.read_capabilities,
                        );

                        let read_hold = collection.implied_capability.clone();
                        self.install_read_capabilities(*id, &storage_dependencies, read_hold)?;
                    }
                }
                DataSource::Introspection(_) | DataSource::Progress | DataSource::Other => {
                    // No since to patch up and no read holds to install on
                    // dependencies!
                }
            }
        }

        // Reborrow `&mut self` immutably, same reasoning as above.
        let this = &*self;

        this.register_shard_mappings(to_create.iter().map(|(id, _)| *id))
            .await;

        // TODO(guswynn): perform the io in this final section concurrently.
        for (id, description) in to_create {
            match description.data_source {
                DataSource::Ingestion(ingestion) => {
                    // Each ingestion is augmented with the collection metadata.
                    let mut source_imports = BTreeMap::new();
                    for (id, _) in ingestion.source_imports {
                        // This _requires_ that the sub-source collection (with
                        // `DataSource::Other`) was registered BEFORE we process this, the
                        // top-level collection.
                        let metadata = self.collection(id)?.collection_metadata.clone();
                        source_imports.insert(id, metadata);
                    }

                    // The ingestion metadata is simply the collection metadata of the collection with
                    // the associated ingestion
                    let ingestion_metadata = self.collection(id)?.collection_metadata.clone();

                    let mut source_exports = BTreeMap::new();
                    for (id, export) in ingestion.source_exports {
                        // Note that these metadata's have been previously enriched with the
                        // required `RelationDesc` for each sub-source above!
                        let storage_metadata = self.collection(id)?.collection_metadata.clone();
                        source_exports.insert(
                            id,
                            SourceExport {
                                storage_metadata,
                                output_index: export.output_index,
                            },
                        );
                    }

                    let desc = IngestionDescription {
                        source_imports,
                        source_exports,
                        ingestion_metadata,
                        // The rest of the fields are identical
                        desc: ingestion.desc,
                        instance_id: ingestion.instance_id,
                        remap_collection_id: ingestion.remap_collection_id,
                    };
                    let mut state = desc.initialize_state(&self.persist).await;
                    let resume_upper = desc.calculate_resumption_frontier(&mut state).await;

                    // Fetch the client for this ingestion's instance.
                    let client = self
                        .state
                        .clients
                        .get_mut(&ingestion.instance_id)
                        .with_context(|| {
                            format!(
                                "instance {} missing for ingestion {}",
                                ingestion.instance_id, id
                            )
                        })?;
                    let augmented_ingestion = CreateSourceCommand {
                        id,
                        description: desc,
                        resume_upper,
                    };

                    client.send(StorageCommand::CreateSources(vec![augmented_ingestion]));
                }
                DataSource::Introspection(i) => {
                    let prev = self.state.introspection_ids.insert(i, id);
                    assert!(
                        prev.is_none(),
                        "cannot have multiple IDs for introspection type"
                    );

                    self.state.collection_manager.register_collection(id).await;

                    match i {
                        IntrospectionType::ShardMapping => {
                            self.initialize_shard_mapping().await;
                        }
                        IntrospectionType::StorageSourceStatistics => {
                            // Set the collection to empty.
                            self.reconcile_managed_collection(id, vec![]).await;

                            let scraper_token = statistics::spawn_statistics_scraper(
                                id.clone(),
                                // These do a shallow copy.
                                self.state.collection_manager.clone(),
                                Arc::clone(&self.state.source_statistics),
                            );

                            // Make sure this is dropped when the controller is
                            // dropped, so that the internal task will stop.
                            self.state.introspection_tokens.insert(id, scraper_token);
                        }
                        IntrospectionType::StorageSinkStatistics => {
                            // Set the collection to empty.
                            self.reconcile_managed_collection(id, vec![]).await;

                            let scraper_token = statistics::spawn_statistics_scraper(
                                id.clone(),
                                // These do a shallow copy.
                                self.state.collection_manager.clone(),
                                Arc::clone(&self.state.sink_statistics),
                            );

                            // Make sure this is dropped when the controller is
                            // dropped, so that the internal task will stop.
                            self.state.introspection_tokens.insert(id, scraper_token);
                        }
                        IntrospectionType::SourceStatusHistory
                        | IntrospectionType::SinkStatusHistory => {
                            // nothing to do: these collections are append only
                        }
                    }
                }
                DataSource::Progress | DataSource::Other => {}
            }
        }

        Ok(())
    }

    fn export(&self, id: GlobalId) -> Result<&ExportState<Self::Timestamp>, StorageError> {
        self.state
            .exports
            .get(&id)
            .ok_or(StorageError::IdentifierMissing(id))
    }

    fn export_mut(
        &mut self,
        id: GlobalId,
    ) -> Result<&mut ExportState<Self::Timestamp>, StorageError> {
        self.state
            .exports
            .get_mut(&id)
            .ok_or(StorageError::IdentifierMissing(id))
    }

    fn prepare_export(
        &mut self,
        id: GlobalId,
        from_id: GlobalId,
    ) -> Result<CreateExportToken<T>, StorageError> {
        if let Ok(_export) = self.export(id) {
            return Err(StorageError::SourceIdReused(id));
        }

        let dependency_since = self.determine_collection_since_joins(&[from_id])?;
        self.install_read_capabilities(id, &[from_id], dependency_since.clone())?;

        info!(
            sink_id = id.to_string(),
            from_id = from_id.to_string(),
            acquired_since = ?dependency_since,
            "prepare_export: sink acquired read holds"
        );

        Ok(CreateExportToken {
            id,
            from_id,
            acquired_since: dependency_since,
        })
    }

    fn cancel_prepare_export(
        &mut self,
        CreateExportToken {
            id,
            from_id,
            acquired_since,
        }: CreateExportToken<T>,
    ) {
        info!(
            sink_id = id.to_string(),
            from_id = from_id.to_string(),
            acquired_since = ?acquired_since,
            "cancel_prepare_export: sink releasing read holds",
        );
        self.remove_read_capabilities(acquired_since, &[from_id]);
    }

    async fn create_exports(
        &mut self,
        exports: Vec<(
            CreateExportToken<Self::Timestamp>,
            ExportDescription<Self::Timestamp>,
        )>,
    ) -> Result<(), StorageError> {
        // Validate first, to avoid corrupting state.
        let mut dedup_hashmap = BTreeMap::<&_, &_>::new();
        for (export, desc) in exports.iter() {
            let CreateExportToken {
                id,
                from_id,
                acquired_since: _,
            } = export;

            if dedup_hashmap.insert(id, desc).is_some() {
                return Err(StorageError::SinkIdReused(*id));
            }
            if let Ok(export) = self.export(*id) {
                if &export.description != desc {
                    return Err(StorageError::SinkIdReused(*id));
                }
            }
            if desc.sink.from != *from_id {
                return Err(StorageError::InvalidUsage(format!(
                    "sink {id} was prepared using from_id {from_id}, \
                    but is now presented with from_id {}",
                    desc.sink.from
                )));
            }
        }

        for (export, description) in exports {
            let CreateExportToken {
                id,
                from_id,
                acquired_since,
            } = export;

            // It's worth adding a quick note on write frontiers here.
            //
            // The write frontier that sinks communicate back to the controller
            // indicates that all further writes will happen at a time `t` such
            // that `!timely::ParitalOrder::less_than(&t, &write_frontier)` is
            // true.  On restart, the sink will receive an SinkAsOf from this
            // controller indicating that it should ignore everything at or
            // before the `since` of the from collection. This will not miss any
            // records because, if there were records not yet written out that
            // have an uncompacted time of `since`, the write frontier
            // previously reported from the sink must be less than `since` so we
            // would not have compacted up to `since`! This is tested by the
            // kafka persistence tests.
            //
            // TODO: Remove upper frontier manipulation from sinks, the read
            // policy ensures that we can always resume and discern the updates
            // that happened at upper. The comment above is slightly wrong:
            // sinks report `F-1` as the upper when they are at upper `F`
            // (speaking in terms of a timely frontier). We should change sinks
            // to divorce what they write out to the progress topic and what
            // they report back as the write upper. To make sure that the
            // reported write upper conforms with what other parts of the system
            // think how uppers work.
            //
            // Note: This is where the sink code (kafka) calculates the write
            // frontier that it reports back:
            // https://github.com/MaterializeInc/materialize/blob/ec8560a532eb5e7282041757d6c1d650f0ffaa77/src/storage/src/sink/kafka.rs#L857
            let read_policy = ReadPolicy::step_back();

            let from_collection = self.collection(from_id)?;
            let from_storage_metadata = from_collection.collection_metadata.clone();

            let storage_dependencies = vec![from_id];

            let mut durable_export_data = MetadataExportFetcher::get_stash_collection()
                .insert_key_without_overwrite(
                    &mut self.state.stash,
                    id,
                    DurableExportMetadata {
                        initial_as_of: description.sink.as_of.clone(),
                    },
                )
                .await?;

            durable_export_data.initial_as_of.downgrade(&acquired_since);

            info!(
                sink_id = id.to_string(),
                from_id = from_id.to_string(),
                acquired_since = ?acquired_since,
                initial_as_of = ?durable_export_data.initial_as_of,
                "create_exports: creating sink"
            );

            self.state.exports.insert(
                id,
                ExportState::new(
                    description.clone(),
                    acquired_since,
                    read_policy,
                    storage_dependencies,
                ),
            );

            let status_id = if let Some(status_collection_id) = description.sink.status_id {
                Some(
                    self.collection(status_collection_id)?
                        .collection_metadata
                        .data_shard,
                )
            } else {
                None
            };

            let cmd = CreateSinkCommand {
                id,
                description: StorageSinkDesc {
                    from: from_id,
                    from_desc: description.sink.from_desc,
                    connection: description.sink.connection,
                    envelope: description.sink.envelope,
                    as_of: durable_export_data.initial_as_of,
                    status_id,
                    from_storage_metadata,
                },
            };

            // Fetch the client for this exports's cluster.
            let client = self
                .state
                .clients
                .get_mut(&description.instance_id)
                .with_context(|| {
                    format!(
                        "cluster {} missing for export {}",
                        description.instance_id, id
                    )
                })?;

            client.send(StorageCommand::CreateSinks(vec![cmd]));
        }
        Ok(())
    }

    fn drop_sources(&mut self, identifiers: Vec<GlobalId>) -> Result<(), StorageError> {
        self.validate_collection_ids(identifiers.iter().cloned())?;
        self.drop_sources_unvalidated(identifiers);
        Ok(())
    }

    fn drop_sources_unvalidated(&mut self, identifiers: Vec<GlobalId>) {
        // We don't explicitly call `remove_read_capabilities`! Downgrading the
        // frontier of the source to `[]` (the empty Antichain), will propagate
        // to the storage dependencies.
        let policies = identifiers
            .into_iter()
            .filter(|id| self.collection(*id).is_ok())
            .map(|id| (id, ReadPolicy::ValidFrom(Antichain::new())))
            .collect();
        self.set_read_policy(policies);
    }

    /// Drops the read capability for the sinks and allows their resources to be reclaimed.
    fn drop_sinks(&mut self, identifiers: Vec<GlobalId>) -> Result<(), StorageError> {
        self.validate_export_ids(identifiers.iter().cloned())?;
        self.drop_sinks_unvalidated(identifiers);
        Ok(())
    }

    fn drop_sinks_unvalidated(&mut self, identifiers: Vec<GlobalId>) {
        for id in identifiers {
            let export = match self.export(id) {
                Ok(export) => export,
                Err(_) => continue,
            };

            let read_capability = export.read_capability.clone();
            let storage_dependencies = export.storage_dependencies.clone();
            self.remove_read_capabilities(read_capability, &storage_dependencies);

            // Remove sink by removing its write frontier and arranging for deprovisioning.
            self.update_write_frontiers(&[(id, Antichain::new())]);
            self.state.pending_sink_drops.push(id);
        }
    }

    #[tracing::instrument(level = "debug", skip_all)]
    fn append(
        &mut self,
        commands: Vec<(GlobalId, Vec<Update<Self::Timestamp>>, Self::Timestamp)>,
    ) -> Result<tokio::sync::oneshot::Receiver<Result<(), StorageError>>, StorageError> {
        // TODO(petrosagg): validate appends against the expected RelationDesc of the collection
        for (id, updates, batch_upper) in commands.iter() {
            for update in updates.iter() {
                if !update.timestamp.less_than(batch_upper) {
                    return Err(StorageError::UpdateBeyondUpper(*id));
                }
            }
        }

        Ok(self.state.persist_write_handles.append(commands))
    }

    // TODO(petrosagg): This signature is not very useful in the context of partially ordered times
    // where the as_of frontier might have multiple elements. In the current form the mutually
    // incomparable updates will be accumulated together to a state of the collection that never
    // actually existed. We should include the original time in the updates advanced by the as_of
    // frontier in the result and let the caller decide what to do with the information.
    async fn snapshot(
        &self,
        id: GlobalId,
        as_of: Self::Timestamp,
    ) -> Result<Vec<(Row, Diff)>, StorageError> {
        let as_of = Antichain::from_elem(as_of);
        let metadata = &self.collection(id)?.collection_metadata;

        let persist_client = self
            .persist
            .open(metadata.persist_location.clone())
            .await
            .unwrap();

        // We create a new read handle every time someone requests a snapshot and then immediately
        // expire it instead of keeping a read handle permanently in our state to avoid having it
        // heartbeat continously. The assumption is that calls to snapshot are rare and therefore
        // worth it to always create a new handle.
        let mut read_handle = persist_client
            .open_leased_reader::<SourceData, (), _, _>(
                metadata.data_shard,
                &format!("snapshot {}", id),
                Arc::new(metadata.relation_desc.clone()),
                Arc::new(UnitSchema),
            )
            .await
            .expect("invalid persist usage");

        match read_handle.snapshot_and_fetch(as_of).await {
            Ok(contents) => {
                let mut snapshot = Vec::with_capacity(contents.len());
                for ((data, _), _, diff) in contents {
                    // TODO(petrosagg): We should accumulate the errors too and let the user
                    // interprret the result
                    let row = data.expect("invalid protobuf data").0?;
                    snapshot.push((row, diff));
                }
                Ok(snapshot)
            }
            Err(_) => Err(StorageError::ReadBeforeSince(id)),
        }
    }

    #[tracing::instrument(level = "debug", skip(self))]
    fn set_read_policy(&mut self, policies: Vec<(GlobalId, ReadPolicy<Self::Timestamp>)>) {
        let mut read_capability_changes = BTreeMap::default();

        for (id, policy) in policies.into_iter() {
            let collection = self
                .collection_mut(id)
                .expect("Reference to absent collection");

            let mut new_read_capability = policy.frontier(collection.write_frontier.borrow());

            if timely::order::PartialOrder::less_equal(
                &collection.implied_capability,
                &new_read_capability,
            ) {
                let mut update = ChangeBatch::new();
                update.extend(new_read_capability.iter().map(|time| (time.clone(), 1)));
                std::mem::swap(&mut collection.implied_capability, &mut new_read_capability);
                update.extend(new_read_capability.iter().map(|time| (time.clone(), -1)));
                if !update.is_empty() {
                    read_capability_changes.insert(id, update);
                }
            }

            collection.read_policy = policy;
        }

        if !read_capability_changes.is_empty() {
            self.update_read_capabilities(&mut read_capability_changes);
        }
    }

    #[tracing::instrument(level = "debug", skip(self))]
    fn update_write_frontiers(&mut self, updates: &[(GlobalId, Antichain<Self::Timestamp>)]) {
        let mut read_capability_changes = BTreeMap::default();

        for (id, new_upper) in updates.iter() {
            if let Ok(collection) = self.collection_mut(*id) {
                if PartialOrder::less_than(&collection.write_frontier, new_upper) {
                    collection.write_frontier = new_upper.clone();
                }

                let mut new_read_capability = collection
                    .read_policy
                    .frontier(collection.write_frontier.borrow());

                if timely::order::PartialOrder::less_equal(
                    &collection.implied_capability,
                    &new_read_capability,
                ) {
                    let mut update = ChangeBatch::new();
                    update.extend(new_read_capability.iter().map(|time| (time.clone(), 1)));
                    std::mem::swap(&mut collection.implied_capability, &mut new_read_capability);
                    update.extend(new_read_capability.iter().map(|time| (time.clone(), -1)));

                    if !update.is_empty() {
                        read_capability_changes.insert(*id, update);
                    }
                }
            } else if let Ok(export) = self.export_mut(*id) {
                if PartialOrder::less_than(&export.write_frontier, new_upper) {
                    export.write_frontier = new_upper.clone();
                }

                let mut new_read_capability =
                    export.read_policy.frontier(export.write_frontier.borrow());

                if timely::order::PartialOrder::less_equal(
                    &export.read_capability,
                    &new_read_capability,
                ) {
                    let mut update = ChangeBatch::new();
                    update.extend(new_read_capability.iter().map(|time| (time.clone(), 1)));
                    std::mem::swap(&mut export.read_capability, &mut new_read_capability);
                    update.extend(new_read_capability.iter().map(|time| (time.clone(), -1)));

                    if !update.is_empty() {
                        read_capability_changes.insert(*id, update);
                    }
                }
            } else {
                panic!("Reference to absent collection {id}");
            }
        }

        if !read_capability_changes.is_empty() {
            self.update_read_capabilities(&mut read_capability_changes);
        }
    }

    #[tracing::instrument(level = "debug", skip(self))]
    fn update_read_capabilities(
        &mut self,
        updates: &mut BTreeMap<GlobalId, ChangeBatch<Self::Timestamp>>,
    ) {
        // Location to record consequences that we need to act on.
        let mut collections_net = BTreeMap::new();
        let mut exports_net = BTreeMap::new();

        // Repeatedly extract the maximum id, and updates for it.
        while let Some(key) = updates.keys().rev().next().cloned() {
            let mut update = updates.remove(&key).unwrap();
            if let Ok(collection) = self.collection_mut(key) {
                for (time, diff) in update.iter() {
                    assert!(
                        collection.read_capabilities.count_for(time) + diff >= 0,
                        "update {:?} for collection {key} would lead to negative \
                        read capabilities, read capabilities before applying: {:?}",
                        update,
                        collection.read_capabilities
                    );
                }

                let changes = collection.read_capabilities.update_iter(update.drain());
                update.extend(changes);

                for id in collection.storage_dependencies.iter() {
                    updates
                        .entry(*id)
                        .or_insert_with(ChangeBatch::new)
                        .extend(update.iter().cloned());
                }

                let (changes, frontier, _cluster_id) =
                    collections_net.entry(key).or_insert_with(|| {
                        (
                            ChangeBatch::new(),
                            Antichain::new(),
                            collection.cluster_id(),
                        )
                    });

                changes.extend(update.drain());
                *frontier = collection.read_capabilities.frontier().to_owned();
            } else if let Ok(export) = self.export_mut(key) {
                // Exports are not depended upon by other storage objects. We
                // only need to report changes in our own read_capability to our
                // dependencies.
                for id in export.storage_dependencies.iter() {
                    updates
                        .entry(*id)
                        .or_insert_with(ChangeBatch::new)
                        .extend(update.iter().cloned());
                }

                // Make sure we also send `AllowCompaction` commands for sinks,
                // which drives updating the sink's `as_of`, among other things.
                let (changes, frontier, _cluster_id) = exports_net
                    .entry(key)
                    .or_insert_with(|| (ChangeBatch::new(), Antichain::new(), export.cluster_id()));

                changes.extend(update.drain());
                *frontier = export.read_capability.clone();
            } else {
                // This is confusing and we should probably error.
                panic!("Unknown collection identifier {}", key);
            }
        }

        // Translate our net compute actions into `AllowCompaction` commands and
        // downgrade persist sinces. The actual downgrades are performed by a Tokio
        // task asynchorously.
        //
        // N.B. We only downgrade persist sinces for collections because
        // exports/sinks don't have an associated collection. We still _do_ want
        // to sent `AllowCompaction` commands to workers for them, though.
        let mut worker_compaction_commands = BTreeMap::default();
        let mut persist_compaction_commands = BTreeMap::default();
        for (key, (mut changes, frontier, cluster_id)) in collections_net {
            if !changes.is_empty() {
                worker_compaction_commands.insert(key, (frontier.clone(), cluster_id));
                persist_compaction_commands.insert(key, frontier);
            }
        }
        for (key, (mut changes, frontier, cluster_id)) in exports_net {
            if !changes.is_empty() {
                worker_compaction_commands.insert(key, (frontier, cluster_id));
            }
        }

        self.state
            .persist_read_handles
            .downgrade(persist_compaction_commands);

        for (id, (frontier, cluster_id)) in worker_compaction_commands {
            // Acquiring a client for a storage instance requires await, so we
            // instead stash these for later and process when we can.
            self.state
                .pending_compaction_commands
                .push((id, frontier, cluster_id));
        }
    }

    async fn ready(&mut self) {
        let mut clients = self
            .state
            .clients
            .values_mut()
            .map(|client| client.response_stream())
            .enumerate()
            .collect::<StreamMap<_, _>>();

        use tokio_stream::StreamExt;
        let msg = tokio::select! {
            // Order matters here. We want to process internal commands
            // before processing external commands.
            biased;

            Some(m) = self.internal_response_queue.recv() => m,
            Some((_id, m)) = clients.next() => m,
        };

        self.state.stashed_response = Some(msg);
    }

    async fn process(&mut self) -> Result<(), anyhow::Error> {
        match self.state.stashed_response.take() {
            None => (),
            Some(StorageResponse::FrontierUppers(updates)) => {
                self.update_write_frontiers(&updates);
            }
            Some(StorageResponse::DroppedIds(_ids)) => {
                // TODO(petrosagg): It looks like the storage controller never cleans up GlobalIds
                // from its state. It should probably be done as a reaction to this response.
            }
            Some(StorageResponse::StatisticsUpdates(source_stats, sink_stats)) => {
                // Note we only hold the locks while moving some plain-old-data around here.

                let mut shared_stats = self.state.source_statistics.lock().expect("poisoned");
                for stat in source_stats {
                    let shared_stats = shared_stats.entry(stat.id).or_default();
                    // We just write the whole object, as the update from storage represents the
                    // current values.
                    shared_stats.insert(stat.worker_id, stat);
                }

                let mut shared_stats = self.state.sink_statistics.lock().expect("poisoned");
                for stat in sink_stats {
                    let shared_stats = shared_stats.entry(stat.id).or_default();
                    // We just write the whole object, as the update from storage represents the
                    // current values.
                    shared_stats.insert(stat.worker_id, stat);
                }
            }
        }

        // TODO(aljoscha): We could consolidate these before sending to
        // instances, but this seems fine for now.
        for (id, frontier, cluster_id) in self.state.pending_compaction_commands.drain(..) {
            // TODO(petrosagg): make this a strict check
            // TODO(aljoscha): What's up with this TODO?
            let client = cluster_id.and_then(|cluster_id| self.state.clients.get_mut(&cluster_id));

            // Only ingestion collections have actual work to do on drop.
            //
            // Note that while collections are dropped, the `client` may already
            // be cleared out, before we do this post-processing!
            if cluster_id.is_some() && frontier.is_empty() {
                self.state.pending_source_drops.push(id);
            }

            if let Some(client) = client {
                client.send(StorageCommand::AllowCompaction(vec![(
                    id,
                    frontier.clone(),
                )]));
            }
        }

        // Record the drop status for all pending source and sink drops.
        let source_status_history_id =
            self.state.introspection_ids[&IntrospectionType::SourceStatusHistory];
        let mut updates = vec![];
        for id in self.state.pending_source_drops.drain(..) {
            let status_row =
                healthcheck::pack_status_row(id, "dropped", None, (self.state.now)(), None);
            updates.push((status_row, 1));
        }
        self.append_to_managed_collection(source_status_history_id, updates)
            .await;

        // Record the drop status for all pending sink drops.
        let sink_status_history_id =
            self.state.introspection_ids[&IntrospectionType::SinkStatusHistory];
        let mut updates = vec![];
        for id in self.state.pending_sink_drops.drain(..) {
            let status_row =
                healthcheck::pack_status_row(id, "dropped", None, (self.state.now)(), None);
            updates.push((status_row, 1));
        }
        self.append_to_managed_collection(sink_status_history_id, updates)
            .await;

        Ok(())
    }

    async fn reconcile_state(&mut self) {
        self.reconcile_shards().await
    }
}

/// A wrapper struct that presents the adapter token to a format that is understandable by persist
/// and also allows us to differentiate between a token being present versus being set for the
/// first time.
// TODO(aljoscha): Make this crate-public again once the remap operator doesn't
// hold a critical handle anymore.
#[derive(PartialEq, Clone, Debug)]
pub struct PersistEpoch(Option<NonZeroI64>);

impl Opaque for PersistEpoch {
    fn initial() -> Self {
        PersistEpoch(None)
    }
}

impl Codec64 for PersistEpoch {
    fn codec_name() -> String {
        "PersistEpoch".to_owned()
    }

    fn encode(&self) -> [u8; 8] {
        self.0.map(NonZeroI64::get).unwrap_or(0).to_le_bytes()
    }

    fn decode(buf: [u8; 8]) -> Self {
        Self(NonZeroI64::new(i64::from_le_bytes(buf)))
    }
}

impl From<NonZeroI64> for PersistEpoch {
    fn from(epoch: NonZeroI64) -> Self {
        Self(Some(epoch))
    }
}

impl<T> Controller<T>
where
    T: Timestamp + Lattice + TotalOrder + Codec64 + From<EpochMillis> + TimestampManipulation,
    StorageCommand<T>: RustType<ProtoStorageCommand>,
    StorageResponse<T>: RustType<ProtoStorageResponse>,

    Self: StorageController<Timestamp = T>,
{
    /// Create a new storage controller from a client it should wrap.
    ///
    /// Note that when creating a new storage controller, you must also
    /// reconcile it with the previous state.
    pub async fn new(
        build_info: &'static BuildInfo,
        postgres_url: String,
        persist_location: PersistLocation,
        persist_clients: Arc<PersistClientCache>,
        now: NowFn,
        postgres_factory: &StashFactory,
        envd_epoch: NonZeroI64,
        metrics_registry: MetricsRegistry,
    ) -> Self {
        let (tx, rx) = tokio::sync::mpsc::unbounded_channel();

        Self {
            build_info,
            state: StorageControllerState::new(postgres_url, tx, now, postgres_factory, envd_epoch)
                .await,
            internal_response_queue: rx,
            persist_location,
            persist: persist_clients,
            metrics: StorageControllerMetrics::new(metrics_registry),
        }
    }

    /// Validate that a collection exists for all identifiers, and error if any do not.
    fn validate_collection_ids(
        &self,
        ids: impl Iterator<Item = GlobalId>,
    ) -> Result<(), StorageError> {
        for id in ids {
            self.collection(id)?;
        }
        Ok(())
    }

    /// Validate that a collection exists for all identifiers, and error if any do not.
    fn validate_export_ids(&self, ids: impl Iterator<Item = GlobalId>) -> Result<(), StorageError> {
        for id in ids {
            self.export(id)?;
        }
        Ok(())
    }

    /// Return the since frontier at which we can read from all the given
    /// collections.
    ///
    /// The outer error is a potentially recoverable internal error, while the
    /// inner error is appropriate to return to the adapter.
    fn determine_collection_since_joins(
        &mut self,
        collections: &[GlobalId],
    ) -> Result<Antichain<T>, StorageError> {
        let mut joined_since = Antichain::from_elem(T::minimum());
        for id in collections {
            let collection = self.collection(*id)?;

            let since = collection.implied_capability.clone();
            joined_since.join_assign(&since);
        }

        Ok(joined_since)
    }

    /// Install read capabilities on the given `storage_dependencies`.
    #[tracing::instrument(level = "info", skip(self))]
    fn install_read_capabilities(
        &mut self,
        from_id: GlobalId,
        storage_dependencies: &[GlobalId],
        read_capability: Antichain<T>,
    ) -> Result<(), StorageError> {
        let mut changes = ChangeBatch::new();
        for time in read_capability.iter() {
            changes.update(time.clone(), 1);
        }

        // This is lifted out of `update_read_capabilitie` because it seems that
        // the compute controller is currently trying to acquire read holds for
        // times that are not beyond the current frontier of
        // `read_capabilities`.
        for id in storage_dependencies {
            let collection = self.collection(id.clone())?;
            let current_read_capabilities = collection.read_capabilities.frontier().to_owned();

            for time in read_capability.iter() {
                assert!(
                    current_read_capabilities.less_equal(time),
                    "trying to acquire hold on {:?} for collection {id} is trying to \
                    install read capabilities before the current \
                    frontier of read capabilities, read capabilities before applying: {:?}",
                    read_capability,
                    collection.read_capabilities
                );
            }
        }

        let mut storage_read_updates = storage_dependencies
            .iter()
            .map(|id| (*id, changes.clone()))
            .collect();

        self.update_read_capabilities(&mut storage_read_updates);

        Ok(())
    }

    /// Removes read holds that were previously acquired via
    /// `install_read_capabilities`.
    ///
    /// ## Panics
    ///
    /// This panics if there are no read capabilities at `capability` for all
    /// depended-upon collections.
    fn remove_read_capabilities(
        &mut self,
        capability: Antichain<T>,
        storage_dependencies: &[GlobalId],
    ) {
        let mut changes = ChangeBatch::new();
        for time in capability.iter() {
            changes.update(time.clone(), -1);
        }

        // Remove holds for all dependencies, which we previously acquired.
        let mut storage_read_updates = storage_dependencies
            .iter()
            .map(|id| (*id, changes.clone()))
            .collect();

        self.update_read_capabilities(&mut storage_read_updates);
    }

    /// Opens a write and critical since handles for the given `shard`.
    ///
    /// This will `halt!` the process if we cannot successfully acquire a
    /// critical handle with our current epoch.
    async fn open_data_handles(
        &self,
        purpose: &str,
        shard: ShardId,
        relation_desc: RelationDesc,
        persist_client: &PersistClient,
    ) -> (
        WriteHandle<SourceData, (), T, Diff>,
        SinceHandle<SourceData, (), T, Diff, PersistEpoch>,
    ) {
        let write = persist_client
            .open_writer(
                shard,
                purpose,
                Arc::new(relation_desc),
                Arc::new(UnitSchema),
            )
            .await
            .expect("invalid persist usage");

        // Construct the handle in a separate block to ensure all error paths are diverging
        let since_handle = {
            // This block's aim is to ensure the handle is in terms of our epoch
            // by the time we return it.
            let mut handle: SinceHandle<_, _, _, _, PersistEpoch> = persist_client
                .open_critical_since(shard, PersistClient::CONTROLLER_CRITICAL_SINCE, purpose)
                .await
                .expect("invalid persist usage");

            let since = handle.since().clone();

            let our_epoch = self.state.envd_epoch;

            loop {
                let current_epoch: PersistEpoch = handle.opaque().clone();

                // Ensure the current epoch is <= our epoch.
                let unchecked_success = current_epoch.0.map(|e| e <= our_epoch).unwrap_or(true);

                if unchecked_success {
                    // Update the handle's state so that it is in terms of our epoch.
                    let checked_success = handle
                        .compare_and_downgrade_since(
                            &current_epoch,
                            (&PersistEpoch::from(our_epoch), &since),
                        )
                        .await
                        .is_ok();
                    if checked_success {
                        break handle;
                    }
                } else {
                    mz_ore::halt!("fenced by envd @ {current_epoch:?}. ours = {our_epoch}");
                }
            }
        };

        (write, since_handle)
    }

    /// Effectively truncates the `data_shard` associated with `global_id`
    /// effective as of the system time.
    ///
    /// # Panics
    /// - If `id` does not belong to a collection or is not registered as a
    ///   managed collection.
    async fn reconcile_managed_collection(&self, id: GlobalId, updates: Vec<(Row, Diff)>) {
        let mut reconciled_updates = BTreeMap::<Row, Diff>::new();

        for (row, diff) in updates.into_iter() {
            *reconciled_updates.entry(row).or_default() += diff;
        }

        match self.state.collections[&id]
            .write_frontier
            .elements()
            .iter()
            .min()
        {
            Some(f) if f > &T::minimum() => {
                let as_of = f.step_back().unwrap();

                let negate = self.snapshot(id, as_of).await.unwrap();

                for (row, diff) in negate.into_iter() {
                    *reconciled_updates.entry(row).or_default() -= diff;
                }
            }
            // If collection is closed or the frontier is the minimum, we cannot
            // or don't need to truncate (respectively).
            _ => {}
        }

        let updates: Vec<_> = reconciled_updates
            .into_iter()
            .filter(|(_, diff)| *diff != 0)
            .collect();

        if !updates.is_empty() {
            self.append_to_managed_collection(id, updates).await;
        }
    }

    /// Append `updates` to the `data_shard` associated with `global_id`
    /// effective as of the system time.
    ///
    /// # Panics
    /// - If `id` is not registered as a managed collection.
    async fn append_to_managed_collection(&self, id: GlobalId, updates: Vec<(Row, Diff)>) {
        self.state
            .collection_manager
            .append_to_collection(id, updates)
            .await;
    }

    /// Initializes the data expressing which global IDs correspond to which
    /// shards. Necessary because we cannot write any of these mappings that we
    /// discover before the shard mapping collection exists.
    ///
    /// # Panics
    /// - If `IntrospectionType::ShardMapping` is not associated with a
    /// `GlobalId` in `self.state.introspection_ids`.
    /// - If `IntrospectionType::ShardMapping`'s `GlobalId` is not registered as
    ///   a managed collection.
    async fn initialize_shard_mapping(&mut self) {
        let id = self.state.introspection_ids[&IntrospectionType::ShardMapping];

        let mut row_buf = Row::default();
        let mut updates = Vec::with_capacity(self.state.collections.len());
        for (
            global_id,
            CollectionState {
                collection_metadata: CollectionMetadata { data_shard, .. },
                ..
            },
        ) in self.state.collections.iter()
        {
            let mut packer = row_buf.packer();
            packer.push(Datum::from(global_id.to_string().as_str()));
            packer.push(Datum::from(data_shard.to_string().as_str()));
            updates.push((row_buf.clone(), 1));
        }

        self.reconcile_managed_collection(id, updates).await;
    }

    /// Writes a new global ID, shard ID pair to the appropriate collection.
    ///
    /// However, data is written iff we know of the `GlobalId` of the
    /// `IntrospectionType::ShardMapping` collection; in other cases, data is
    /// dropped on the floor. In these cases, the data is later written by
    /// [`Self::initialize_shard_mapping`].
    ///
    /// # Panics
    /// - If `self.state.collections` does not have an entry for `global_id`.
    /// - If `IntrospectionType::ShardMapping`'s `GlobalId` is not registered as
    ///   a managed collection.
    async fn register_shard_mappings<I>(&self, global_ids: I)
    where
        I: Iterator<Item = GlobalId>,
    {
        let id = match self
            .state
            .introspection_ids
            .get(&IntrospectionType::ShardMapping)
        {
            Some(id) => *id,
            _ => return,
        };

        let mut updates = vec![];
        // Pack updates into rows
        let mut row_buf = Row::default();

        for global_id in global_ids {
            let shard_id = self.state.collections[&global_id]
                .collection_metadata
                .data_shard;

            let mut packer = row_buf.packer();
            packer.push(Datum::from(global_id.to_string().as_str()));
            packer.push(Datum::from(shard_id.to_string().as_str()));
            updates.push((row_buf.clone(), 1));
        }

        self.append_to_managed_collection(id, updates).await;
    }

    /// Updates the on-disk and in-memory representation of
    /// `DurableCollectionMetadata` (i.e. KV pairs in `METADATA_COLLECTION`
    /// on-disk and `all_current_metadata` as its in-memory representation) to
    /// include that of `upsert_state`, i.e. upserting the KV pairs in
    /// `upsert_state` into in `all_current_metadata`, as well as
    /// `METADATA_COLLECTION`.
    ///
    /// Any shards no longer referenced after the upsert will be finalized.
    ///
    /// Note that this function expects to be called:
    /// - While no source is currently using the shards identified in the
    ///   current metadata.
    /// - Before any sources begins using the shards identified in
    ///   `new_metadata`.
    async fn upsert_collection_metadata(
        &mut self,
        all_current_metadata: &mut BTreeMap<GlobalId, DurableCollectionMetadata>,
        upsert_state: BTreeMap<GlobalId, DurableCollectionMetadata>,
    ) {
        // If nothing changed, don't do any work, which might include async
        // calls into stash.
        if upsert_state.is_empty() {
            return;
        }

        let mut new_shards = BTreeSet::new();
        let mut dropped_shards = BTreeSet::new();
        let mut data_shards_to_replace = BTreeSet::new();
        let mut remap_shards_to_replace = BTreeSet::new();
        for (id, new_metadata) in upsert_state.iter() {
            assert!(
                new_metadata.remap_shard.is_none(),
                "must not reintroduce remap shards"
            );

            match all_current_metadata.get(id) {
                Some(metadata) => {
                    for (old, new, data_shard) in [
                        (
                            Some(metadata.data_shard),
                            Some(new_metadata.data_shard),
                            true,
                        ),
                        (metadata.remap_shard, new_metadata.remap_shard, false),
                    ] {
                        if old != new {
                            info!(
                                "replacing {:?}'s {} shard {:?} with {:?}",
                                id,
                                if data_shard { "data" } else { "remap" },
                                old,
                                new
                            );

                            if let Some(new) = new {
                                new_shards.insert(new);
                            }

                            if let Some(old) = old {
                                dropped_shards.insert(old);
                            }

                            if data_shard {
                                data_shards_to_replace.insert(*id);
                            } else {
                                remap_shards_to_replace.insert(*id);
                            }
                        }
                    }
                }
                // New collections, which might use an another collection's
                // dropped shard.
                None => {
                    new_shards.insert(new_metadata.data_shard);
                    continue;
                }
            };

            // Update the in-memory representation.
            all_current_metadata.insert(*id, new_metadata.clone());
        }

        // Reconcile dropped shards reference with shards that moved into a new
        // collection.
        dropped_shards.retain(|shard| !new_shards.contains(shard));

        // Ensure we don't leak any shards by tracking all of them we intend to
        // finalize.
        self.register_shards_for_finalization(dropped_shards.iter().cloned())
            .await;

        // Update the on-disk representation.
        METADATA_COLLECTION
            .upsert(&mut self.state.stash, upsert_state.into_iter())
            .await
            .expect("connect to stash");

        // Finalize any shards that are no longer-in use.
        self.finalize_shards(dropped_shards).await;

        // Update in-memory state for remap shards.
        for id in remap_shards_to_replace {
            let c = match self.collection_mut(id) {
                Ok(c) => c,
                Err(_) => continue,
            };

            c.collection_metadata.remap_shard = all_current_metadata[&id].remap_shard;
        }

        // Avoid taking lock if unnecessary
        if data_shards_to_replace.is_empty() {
            return;
        }

        let persist_client = self
            .persist
            .open(self.persist_location.clone())
            .await
            .unwrap();

        // Update the in-memory state for data shards
        for id in data_shards_to_replace {
            let c = match self.collection_mut(id) {
                Ok(c) => c,
                Err(_) => continue,
            };

            let data_shard = all_current_metadata[&id].data_shard;
            c.collection_metadata.data_shard = data_shard;

            let relation_desc = c.collection_metadata.relation_desc.clone();

            // This will halt! if any of the handles cannot be acquired
            // because we're not the leader anymore. But that's fine, we
            // already updated all the persistent state (in stash).
            let (write, since_handle) = self
                .open_data_handles(
                    format!("controller data for {id}").as_str(),
                    data_shard,
                    relation_desc,
                    &persist_client,
                )
                .await;

            self.state.persist_write_handles.update(id, write);
            self.state.persist_read_handles.update(id, since_handle);
        }
    }

    /// Closes the identified shards from further reads or writes.
    #[allow(dead_code)]
    async fn finalize_shards<I>(&mut self, shards: I)
    where
        I: IntoIterator<Item = ShardId> + Clone,
    {
        soft_assert!(
            {
                let mut all_registered = true;
                for shard in shards.clone() {
                    all_registered =
                        self.is_shard_registered_for_finalization(shard).await && all_registered
                }
                all_registered
            },
            "finalized shards must be registered before calling finalize_shards"
        );

        // Open a persist client to delete unused shards.
        let persist_client = self
            .persist
            .open(self.persist_location.clone())
            .await
            .unwrap();

        let persist_client = &persist_client;
        // Reborrow the `&mut self` as immutable, as all the concurrent work to be processed in
        // this stream cannot all have exclusive access.
        let this = &*self;

        use futures::stream::StreamExt;
        let finalized_shards: BTreeSet<_> = futures::stream::iter(shards)
            .map(|shard_id| async move {
                let (mut write, mut critical_since_handle) = this
                    .open_data_handles(
                        "finalizing shards",
                        shard_id,
                        RelationDesc::empty(),
                        persist_client,
                    )
                    .await;

                let our_epoch = PersistEpoch::from(this.state.envd_epoch);

                match critical_since_handle
                    .compare_and_downgrade_since(&our_epoch, (&our_epoch, &Antichain::new()))
                    .await
                {
                    Ok(_) => info!("successfully finalized read handle for shard {shard_id:?}"),
                    Err(e) => mz_ore::halt!("fenced by envd @ {e:?}. ours = {our_epoch:?}"),
                }

                if write.upper().is_empty() {
                    info!("write handle for shard {:?} already finalized", shard_id);
                } else {
                    write
                        .append(
                            Vec::<((crate::types::sources::SourceData, ()), T, Diff)>::new(),
                            write.upper().clone(),
                            Antichain::new(),
                        )
                        .await
                        .expect("failed to connect")
                        .expect("failed to truncate write handle");
                }
                shard_id
            })
            // Poll each future for each collection concurrently, maximum of 10 at a time.
            .buffer_unordered(10)
            // HERE BE DRAGONS: see warning on other uses of buffer_unordered
            // before any changes to `collect`
            .collect()
            .await;

        self.clear_from_shard_finalization_register(finalized_shards)
            .await;
    }
}

/// State maintained about individual collections.
#[derive(Debug)]
pub struct CollectionState<T> {
    /// Description with which the collection was created
    pub description: CollectionDescription<T>,

    /// Accumulation of read capabilities for the collection.
    ///
    /// This accumulation will always contain `self.implied_capability`, but may also contain
    /// capabilities held by others who have read dependencies on this collection.
    pub read_capabilities: MutableAntichain<T>,
    /// The implicit capability associated with collection creation.  This should never be less
    /// than the since of the associated persist collection.
    pub implied_capability: Antichain<T>,
    /// The policy to use to downgrade `self.implied_capability`.
    pub read_policy: ReadPolicy<T>,

    /// Storage identifiers on which this collection depends.
    pub storage_dependencies: Vec<GlobalId>,

    /// Reported write frontier.
    pub write_frontier: Antichain<T>,

    pub collection_metadata: CollectionMetadata,
}

impl<T: Timestamp> CollectionState<T> {
    /// Creates a new collection state, with an initial read policy valid from `since`.
    pub fn new(
        description: CollectionDescription<T>,
        since: Antichain<T>,
        write_frontier: Antichain<T>,
        storage_dependencies: Vec<GlobalId>,
        metadata: CollectionMetadata,
    ) -> Self {
        let mut read_capabilities = MutableAntichain::new();
        read_capabilities.update_iter(since.iter().map(|time| (time.clone(), 1)));
        Self {
            description,
            read_capabilities,
            implied_capability: since.clone(),
            read_policy: ReadPolicy::NoPolicy {
                initial_since: since,
            },
            storage_dependencies,
            write_frontier,
            collection_metadata: metadata,
        }
    }

    /// Returns the cluster to which the collection is bound, if applicable.
    fn cluster_id(&self) -> Option<StorageInstanceId> {
        match &self.description.data_source {
            DataSource::Ingestion(ingestion) => Some(ingestion.instance_id),
            DataSource::Introspection(_) | DataSource::Other | DataSource::Progress => None,
        }
    }
}

/// State maintained about individual exports.
#[derive(Debug)]
pub struct ExportState<T> {
    /// Description with which the export was created
    pub description: ExportDescription<T>,

    /// The capability (hold on the since) that this export needs from its
    /// dependencies (inputs). When the upper of the export changes, we
    /// downgrade this, which in turn downgrades holds we have on our
    /// dependencies' sinces.
    pub read_capability: Antichain<T>,

    /// The policy to use to downgrade `self.read_capability`.
    pub read_policy: ReadPolicy<T>,

    /// Storage identifiers on which this collection depends.
    pub storage_dependencies: Vec<GlobalId>,

    /// Reported write frontier.
    pub write_frontier: Antichain<T>,
}

impl<T: Timestamp> ExportState<T> {
    fn new(
        description: ExportDescription<T>,
        read_capability: Antichain<T>,
        read_policy: ReadPolicy<T>,
        storage_dependencies: Vec<GlobalId>,
    ) -> Self {
        Self {
            description,
            read_capability,
            read_policy,
            storage_dependencies,
            write_frontier: Antichain::from_elem(Timestamp::minimum()),
        }
    }

    /// Returns the cluster to which the export is bound, if applicable.
    fn cluster_id(&self) -> Option<StorageInstanceId> {
        Some(self.description.instance_id)
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    #[test]
    fn lag_writes_by_zero() {
        let policy =
            ReadPolicy::lag_writes_by(mz_repr::Timestamp::default(), mz_repr::Timestamp::default());
        let write_frontier = Antichain::from_elem(mz_repr::Timestamp::from(5));
        assert_eq!(policy.frontier(write_frontier.borrow()), write_frontier);
    }
}