1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
// 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.

//! An abstraction for dealing with storage collections.

use std::collections::{BTreeMap, BTreeSet};
use std::fmt::Debug;
use std::num::NonZeroI64;
use std::sync::{Arc, Mutex};
use std::time::Duration;

use async_trait::async_trait;
use differential_dataflow::lattice::Lattice;
use futures::future::BoxFuture;
use futures::stream::FuturesUnordered;
use futures::{Future, FutureExt, StreamExt};
use itertools::Itertools;

use mz_ore::collections::CollectionExt;
use mz_ore::metrics::MetricsRegistry;
use mz_ore::now::{EpochMillis, NowFn};
use mz_ore::task::AbortOnDropHandle;
use mz_ore::{assert_none, instrument};
use mz_persist_client::cache::PersistClientCache;
use mz_persist_client::cfg::USE_CRITICAL_SINCE_SNAPSHOT;
use mz_persist_client::critical::SinceHandle;
use mz_persist_client::read::ReadHandle;
use mz_persist_client::stats::{SnapshotPartsStats, SnapshotStats};
use mz_persist_client::write::WriteHandle;
use mz_persist_client::{Diagnostics, PersistClient, PersistLocation, ShardId};
use mz_persist_types::codec_impls::UnitSchema;
use mz_persist_types::txn::TxnsCodec;
use mz_persist_types::Codec64;
use mz_repr::{Diff, GlobalId, RelationDesc, TimestampManipulation};
use mz_storage_types::configuration::StorageConfiguration;
use mz_storage_types::connections::inline::InlinedConnection;
use mz_storage_types::connections::ConnectionContext;
use mz_storage_types::controller::{CollectionMetadata, StorageError, TxnsCodecRow};
use mz_storage_types::dyncfgs::STORAGE_DOWNGRADE_SINCE_DURING_FINALIZATION;
use mz_storage_types::parameters::StorageParameters;
use mz_storage_types::read_holds::{ReadHold, ReadHoldError};
use mz_storage_types::read_policy::ReadPolicy;
use mz_storage_types::sources::{
    GenericSourceConnection, IngestionDescription, SourceData, SourceDesc, SourceExport,
};
use mz_txn_wal::metrics::Metrics as TxnMetrics;
use mz_txn_wal::txn_read::{DataSnapshot, TxnsRead};
use mz_txn_wal::txns::TxnsHandle;
use timely::order::TotalOrder;
use timely::progress::frontier::MutableAntichain;
use timely::progress::{Antichain, ChangeBatch, Timestamp as TimelyTimestamp};
use timely::PartialOrder;
use tokio::sync::{mpsc, oneshot};
use tokio::time::MissedTickBehavior;
use tracing::{debug, info, trace, warn};

use crate::controller::{
    CollectionDescription, DataSource, DataSourceOther, PersistEpoch, StorageMetadata, StorageTxn,
};
use crate::storage_collections::metrics::{ShardIdSet, StorageCollectionsMetrics};

mod metrics;

/// An abstraction for keeping track of storage collections and managing access
/// to them.
///
/// Responsibilities:
///
/// - Keeps a critical persist handle for holding the since of collections
///   where it need to be.
///
/// - Drives the since forward based on the upper of a collection and a
///   [ReadPolicy].
///
/// - Hands out [ReadHolds](ReadHold) that prevent a collection's since from
/// advancing while it needs to be read at a specific time.
#[async_trait]
pub trait StorageCollections: Debug {
    type Timestamp: TimelyTimestamp;

    /// On boot, reconcile this [StorageCollections] with outside state. We get
    /// a [StorageTxn] where we can record any durable state that we need.
    ///
    /// We get `init_ids`, which tells us about all collections that currently
    /// exist, so that we can record durable state for those that _we_ don't
    /// know yet about.
    ///
    /// We also get `drop_ids`, which tells us about all collections that we
    /// might have known about before and have now been dropped.
    async fn initialize_state(
        &self,
        txn: &mut (dyn StorageTxn<Self::Timestamp> + Send),
        init_ids: BTreeSet<GlobalId>,
        drop_ids: BTreeSet<GlobalId>,
    ) -> Result<(), StorageError<Self::Timestamp>>;

    /// Update storage configuration with new parameters.
    fn update_parameters(&self, config_params: StorageParameters);

    /// Returns the [CollectionMetadata] of the collection identified by `id`.
    fn collection_metadata(
        &self,
        id: GlobalId,
    ) -> Result<CollectionMetadata, StorageError<Self::Timestamp>>;

    /// Acquire an iterator over [CollectionMetadata] for all active
    /// collections.
    ///
    /// A collection is "active" when it has a non empty frontier of read
    /// capabilties.
    fn active_collection_metadatas(&self) -> Vec<(GlobalId, CollectionMetadata)>;

    /// Returns the frontiers of the identified collection.
    fn collection_frontiers(
        &self,
        id: GlobalId,
    ) -> Result<CollectionFrontiers<Self::Timestamp>, StorageError<Self::Timestamp>> {
        let frontiers = self
            .collections_frontiers(vec![id])?
            .expect_element(|| "known to exist");

        Ok(frontiers)
    }

    /// Atomically gets and returns the frontiers of all the identified
    /// collections.
    fn collections_frontiers(
        &self,
        id: Vec<GlobalId>,
    ) -> Result<Vec<CollectionFrontiers<Self::Timestamp>>, StorageError<Self::Timestamp>>;

    /// Atomically gets and returns the frontiers of all active collections.
    ///
    /// A collection is "active" when it has a non empty frontier of read
    /// capabilties.
    fn active_collection_frontiers(&self) -> Vec<CollectionFrontiers<Self::Timestamp>>;

    /// Checks whether a collection exists under the given `GlobalId`. Returns
    /// an error if the collection does not exist.
    fn check_exists(&self, id: GlobalId) -> Result<(), StorageError<Self::Timestamp>>;

    /// Returns aggregate statistics about the contents of the local input named
    /// `id` at `as_of`.
    async fn snapshot_stats(
        &self,
        id: GlobalId,
        as_of: Antichain<Self::Timestamp>,
    ) -> Result<SnapshotStats, StorageError<Self::Timestamp>>;

    /// Returns aggregate statistics about the contents of the local input named
    /// `id` at `as_of`.
    ///
    /// Note that this async function itself returns a future. We may
    /// need to block on the stats being available, but don't want to hold a reference
    /// to the controller for too long... so the outer future holds a reference to the
    /// controller but returns quickly, and the inner future is slow but does not
    /// reference the controller.
    async fn snapshot_parts_stats(
        &self,
        id: GlobalId,
        as_of: Antichain<Self::Timestamp>,
    ) -> BoxFuture<'static, Result<SnapshotPartsStats, StorageError<Self::Timestamp>>>;

    /// Update the given [`StorageTxn`] with the appropriate metadata given the
    /// IDs to add and drop.
    ///
    /// The data modified in the `StorageTxn` must be made available in all
    /// subsequent calls that require [`StorageMetadata`] as a parameter.
    async fn prepare_state(
        &self,
        txn: &mut (dyn StorageTxn<Self::Timestamp> + Send),
        ids_to_add: BTreeSet<GlobalId>,
        ids_to_drop: BTreeSet<GlobalId>,
    ) -> Result<(), StorageError<Self::Timestamp>>;

    /// Create the collections described by the individual
    /// [CollectionDescriptions](CollectionDescription).
    ///
    /// 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
    /// they 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 [StorageCollections] in an
    /// inconsistent state. It is almost always wrong to do anything but abort
    /// the process on `Err`.
    ///
    /// The `register_ts` is used as the initial timestamp that tables are
    /// available for reads. (We might later give non-tables the same treatment,
    /// but hold off on that initially.) Callers must provide a Some if any of
    /// the collections is a table. A None may be given if none of the
    /// collections are a table (i.e. all materialized views, sources, etc).
    async fn create_collections(
        &self,
        storage_metadata: &StorageMetadata,
        register_ts: Option<Self::Timestamp>,
        collections: Vec<(GlobalId, CollectionDescription<Self::Timestamp>)>,
    ) -> Result<(), StorageError<Self::Timestamp>> {
        self.create_collections_for_bootstrap(
            storage_metadata,
            register_ts,
            collections,
            &BTreeSet::new(),
        )
        .await
    }

    /// Like [`Self::create_collections`], except used specifically for bootstrap.
    ///
    /// `migrated_storage_collections` is a set of migrated storage collections to be excluded
    /// from the txn-wal sub-system.
    async fn create_collections_for_bootstrap(
        &self,
        storage_metadata: &StorageMetadata,
        register_ts: Option<Self::Timestamp>,
        collections: Vec<(GlobalId, CollectionDescription<Self::Timestamp>)>,
        migrated_storage_collections: &BTreeSet<GlobalId>,
    ) -> Result<(), StorageError<Self::Timestamp>>;

    /// Alters the identified ingestion to use the provided [`SourceDesc`].
    ///
    /// NOTE: Ideally, [StorageCollections] would not care about these, but we
    /// have to learn about changes such that when new subsources are created we
    /// can correctly determine a since based on its depenencies' sinces. This
    /// is really only relevant because newly created subsources depend on the
    /// remap shard, and we can't just have them start at since 0.
    async fn alter_ingestion_source_desc(
        &self,
        ingestion_id: GlobalId,
        source_desc: SourceDesc,
    ) -> Result<(), StorageError<Self::Timestamp>>;

    /// Alters each identified collection to use the correlated
    /// [`GenericSourceConnection`].
    ///
    /// See NOTE on [StorageCollections::alter_ingestion_source_desc].
    async fn alter_ingestion_connections(
        &self,
        source_connections: BTreeMap<GlobalId, GenericSourceConnection<InlinedConnection>>,
    ) -> Result<(), StorageError<Self::Timestamp>>;

    /// Updates the [`RelationDesc`] for the specified table.
    async fn alter_table_desc(
        &self,
        table_id: GlobalId,
        new_desc: RelationDesc,
    ) -> Result<(), StorageError<Self::Timestamp>>;

    /// 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_collections_unvalidated(
        &self,
        storage_metadata: &StorageMetadata,
        identifiers: Vec<GlobalId>,
    );

    /// 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.
    ///
    /// This [StorageCollections] may include its own overrides on these
    /// policies.
    ///
    /// Identifiers not present in `policies` retain their existing read
    /// policies.
    fn set_read_policies(&self, policies: Vec<(GlobalId, ReadPolicy<Self::Timestamp>)>);

    /// Acquires and returns the earliest possible read holds for the specified
    /// collections.
    fn acquire_read_holds(
        &self,
        desired_holds: Vec<GlobalId>,
    ) -> Result<Vec<ReadHold<Self::Timestamp>>, ReadHoldError>;
}

/// Frontiers of the collection identified by `id`.
pub struct CollectionFrontiers<T> {
    /// The [GlobalId] of the collection that these frontiers belong to.
    pub id: GlobalId,

    /// The upper/write frontier of the collection.
    pub write_frontier: Antichain<T>,

    /// The since frontier that is implied by the collection's existence,
    /// disregarding any read holds.
    ///
    /// Concretely, it is the since frontier that is implied by the combination
    /// of the `write_frontier` and a [ReadPolicy]. The implied capability is
    /// derived from the write frontier using the [ReadPolicy].
    pub implied_capability: Antichain<T>,

    /// The frontier of all oustanding [ReadHolds](ReadHold). This includes the
    /// implied capability.
    pub read_capabilities: Antichain<T>,
}

/// Implementation of [StorageCollections] that is shallow-cloneable and uses a
/// background task for doing work concurrently, in the background.
#[derive(Debug, Clone)]
pub struct StorageCollectionsImpl<
    T: TimelyTimestamp + Lattice + Codec64 + From<EpochMillis> + TimestampManipulation,
> {
    /// The fencing token for this instance of [StorageCollections], and really
    /// all of the controllers and Coordinator.
    envd_epoch: NonZeroI64,

    /// Whether or not this [StorageCollections] is in read-only mode.
    ///
    /// When in read-only mode, we are not allowed to affect changes to external
    /// systems, including, for example, acquiring and downgrading critical
    /// [SinceHandles](SinceHandle)
    read_only: bool,

    /// The set of [ShardIds](ShardId) that we have to finalize. These will have
    /// been persisted by the caller of [StorageCollections::prepare_state].
    finalizable_shards: Arc<ShardIdSet>,

    /// The set of [ShardIds](ShardId) that we have finalized. We keep track of
    /// shards here until we are given a chance to let our callers know that
    /// these have been finalized, for example via
    /// [StorageCollections::prepare_state].
    finalized_shards: Arc<ShardIdSet>,

    /// Collections maintained by this [StorageCollections].
    collections: Arc<std::sync::Mutex<BTreeMap<GlobalId, CollectionState<T>>>>,

    /// A shared TxnsCache running in a task and communicated with over a channel.
    txns_read: TxnsRead<T>,

    /// Storage configuration parameters.
    config: Arc<Mutex<StorageConfiguration>>,

    /// The upper of the txn shard as it was when we booted. We forward the
    /// upper of created/registered tables to make sure that their uppers are at
    /// least not less than the initially known txn upper.
    ///
    /// NOTE: This works around a quirk in how the adapter chooses the as_of of
    /// existing indexes when bootstrapping, where tables that have an upper
    /// that is less than the initially known txn upper can lead to indexes that
    /// cannot hydrate in read-only mode.
    initial_txn_upper: Antichain<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>,

    /// For sending commands to our internal task.
    cmd_tx: mpsc::UnboundedSender<BackgroundCmd<T>>,

    /// For sending updates about read holds to our internal task.
    holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch<T>)>,

    /// Handles to tasks we own, making sure they're dropped when we are.
    _background_task: Arc<AbortOnDropHandle<()>>,
    _finalize_shards_task: Arc<AbortOnDropHandle<()>>,
}

// Supporting methods for implementing [StorageCollections].
//
// Almost all internal methods that are the backing implementation for a trait
// method have the `_inner` suffix.
//
// We follow a pattern where `_inner` methods get a mutable reference to the
// shared collections state, and it's the public-facing method that locks the
// state for the duration of its invocation. This allows calling other `_inner`
// methods from within `_inner` methods.
impl<T> StorageCollectionsImpl<T>
where
    T: TimelyTimestamp
        + Lattice
        + Codec64
        + From<EpochMillis>
        + TimestampManipulation
        + Into<mz_repr::Timestamp>,
{
    /// Creates and returns a new [StorageCollections].
    ///
    /// Note that when creating a new [StorageCollections], you must also
    /// reconcile it with the previous state using
    /// [StorageCollections::initialize_state],
    /// [StorageCollections::prepare_state], and
    /// [StorageCollections::create_collections].
    pub async fn new(
        persist_location: PersistLocation,
        persist_clients: Arc<PersistClientCache>,
        metrics_registry: &MetricsRegistry,
        _now: NowFn,
        txns_metrics: Arc<TxnMetrics>,
        envd_epoch: NonZeroI64,
        read_only: bool,
        connection_context: ConnectionContext,
        txn: &dyn StorageTxn<T>,
    ) -> Self {
        let metrics = StorageCollectionsMetrics::register_into(metrics_registry);

        // This value must be already installed because we must ensure it's
        // durably recorded before it is used, otherwise we risk leaking persist
        // state.
        let txns_id = txn
            .get_txn_wal_shard()
            .expect("must call prepare initialization before creating StorageCollections");

        let txns_client = persist_clients
            .open(persist_location.clone())
            .await
            .expect("location should be valid");

        // We have to initialize, so that TxnsRead::start() below does not
        // block.
        let _txns_handle: TxnsHandle<SourceData, (), T, i64, PersistEpoch, TxnsCodecRow> =
            TxnsHandle::open(
                T::minimum(),
                txns_client.clone(),
                txns_client.dyncfgs().clone(),
                Arc::clone(&txns_metrics),
                txns_id,
            )
            .await;

        // For handing to the background task, for listening to upper updates.
        let (txns_key_schema, txns_val_schema) = TxnsCodecRow::schemas();
        let mut txns_write = txns_client
            .open_writer(
                txns_id,
                Arc::new(txns_key_schema),
                Arc::new(txns_val_schema),
                Diagnostics {
                    shard_name: "txns".to_owned(),
                    handle_purpose: "commit txns".to_owned(),
                },
            )
            .await
            .expect("txns schema shouldn't change");

        let txns_read = TxnsRead::start::<TxnsCodecRow>(txns_client.clone(), txns_id).await;

        let collections = Arc::new(std::sync::Mutex::new(BTreeMap::default()));
        let finalizable_shards =
            Arc::new(ShardIdSet::new(metrics.finalization_outstanding.clone()));
        let finalized_shards =
            Arc::new(ShardIdSet::new(metrics.finalization_pending_commit.clone()));
        let config = Arc::new(Mutex::new(StorageConfiguration::new(
            connection_context,
            mz_dyncfgs::all_dyncfgs(),
        )));

        let initial_txn_upper = txns_write.fetch_recent_upper().await.to_owned();

        let (cmd_tx, cmd_rx) = mpsc::unbounded_channel();
        let (holds_tx, holds_rx) = mpsc::unbounded_channel();
        let mut background_task = BackgroundTask {
            config: Arc::clone(&config),
            cmds_tx: cmd_tx.clone(),
            cmds_rx: cmd_rx,
            holds_rx,
            collections: Arc::clone(&collections),
            finalizable_shards: Arc::clone(&finalizable_shards),
            shard_by_id: BTreeMap::new(),
            since_handles: BTreeMap::new(),
            txns_handle: Some(txns_write),
            txns_shards: Default::default(),
        };

        let background_task =
            mz_ore::task::spawn(|| "storage_collections::background_task", async move {
                background_task.run().await
            });

        let finalize_shards_task = mz_ore::task::spawn(
            || "storage_collections::finalize_shards_task",
            finalize_shards_task::<T>(FinalizeShardsTaskConfig {
                envd_epoch: envd_epoch.clone(),
                config: Arc::clone(&config),
                metrics,
                finalizable_shards: Arc::clone(&finalizable_shards),
                finalized_shards: Arc::clone(&finalized_shards),
                persist_location: persist_location.clone(),
                persist: Arc::clone(&persist_clients),
                read_only,
            }),
        );

        Self {
            finalizable_shards,
            finalized_shards,
            collections,
            txns_read,
            envd_epoch,
            read_only,
            config,
            initial_txn_upper,
            persist_location,
            persist: persist_clients,
            cmd_tx,
            holds_tx,
            _background_task: Arc::new(background_task.abort_on_drop()),
            _finalize_shards_task: Arc::new(finalize_shards_task.abort_on_drop()),
        }
    }

    /// Opens a [WriteHandle] and a [SinceHandleWrapper], for holding back the since.
    ///
    /// `since` is an optional since that the read handle will be forwarded to
    /// if it is less than its current since.
    ///
    /// This will `halt!` the process if we cannot successfully acquire a
    /// critical handle with our current epoch.
    async fn open_data_handles(
        &self,
        id: &GlobalId,
        shard: ShardId,
        since: Option<&Antichain<T>>,
        relation_desc: RelationDesc,
        persist_client: &PersistClient,
    ) -> (WriteHandle<SourceData, (), T, Diff>, SinceHandleWrapper<T>) {
        let since_handle = if self.read_only {
            let read_handle = self
                .open_leased_handle(id, shard, relation_desc.clone(), since, persist_client)
                .await;
            SinceHandleWrapper::Leased(read_handle)
        } else {
            let since_handle = self
                .open_critical_handle(id, shard, since, persist_client)
                .await;

            SinceHandleWrapper::Critical(since_handle)
        };

        let mut write_handle = self
            .open_write_handle(id, shard, relation_desc, persist_client)
            .await;

        // N.B.
        // Fetch the most recent upper for the write handle. Otherwise, this may
        // be behind the since of the since handle. Its vital this happens AFTER
        // we create the since handle as it needs to be linearized with that
        // operation. It may be true that creating the write handle after the
        // since handle already ensures this, but we do this out of an abundance
        // of caution.
        //
        // Note that this returns the upper, but also sets it on the handle to
        // be fetched later.
        write_handle.fetch_recent_upper().await;

        (write_handle, since_handle)
    }

    /// Opens a write handle for the given `shard`.
    async fn open_write_handle(
        &self,
        id: &GlobalId,
        shard: ShardId,
        relation_desc: RelationDesc,
        persist_client: &PersistClient,
    ) -> WriteHandle<SourceData, (), T, Diff> {
        let diagnostics = Diagnostics {
            shard_name: id.to_string(),
            handle_purpose: format!("controller data for {}", id),
        };

        let write = persist_client
            .open_writer(
                shard,
                Arc::new(relation_desc),
                Arc::new(UnitSchema),
                diagnostics.clone(),
            )
            .await
            .expect("invalid persist usage");

        write
    }

    /// Opens a critical since handle for the given `shard`.
    ///
    /// `since` is an optional since that the read handle will be forwarded to
    /// if it is less than its current since.
    ///
    /// This will `halt!` the process if we cannot successfully acquire a
    /// critical handle with our current epoch.
    async fn open_critical_handle(
        &self,
        id: &GlobalId,
        shard: ShardId,
        since: Option<&Antichain<T>>,
        persist_client: &PersistClient,
    ) -> SinceHandle<SourceData, (), T, Diff, PersistEpoch> {
        tracing::debug!(%id, ?since, "opening critical handle");

        assert!(
            !self.read_only,
            "attempting to open critical SinceHandle in read-only mode"
        );

        let diagnostics = Diagnostics {
            shard_name: id.to_string(),
            handle_purpose: format!("controller data for {}", id),
        };

        // 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,
                    diagnostics.clone(),
                )
                .await
                .expect("invalid persist usage");

            // Take the join of the handle's since and the provided `since`;
            // this lets materialized views express the since at which their
            // read handles "start."
            let since = handle
                .since()
                .join(since.unwrap_or(&Antichain::from_elem(T::minimum())));

            let our_epoch = self.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}");
                }
            }
        };

        since_handle
    }

    /// Opens a leased [ReadHandle], for the purpose of holding back a since,
    /// for the given `shard`.
    ///
    /// `since` is an optional since that the read handle will be forwarded to
    /// if it is less than its current since.
    async fn open_leased_handle(
        &self,
        id: &GlobalId,
        shard: ShardId,
        relation_desc: RelationDesc,
        since: Option<&Antichain<T>>,
        persist_client: &PersistClient,
    ) -> ReadHandle<SourceData, (), T, Diff> {
        tracing::debug!(%id, ?since, "opening leased handle");

        let diagnostics = Diagnostics {
            shard_name: id.to_string(),
            handle_purpose: format!("controller data for {}", id),
        };

        let use_critical_since = false;
        let mut handle: ReadHandle<_, _, _, _> = persist_client
            .open_leased_reader(
                shard,
                Arc::new(relation_desc),
                Arc::new(UnitSchema),
                diagnostics.clone(),
                use_critical_since,
            )
            .await
            .expect("invalid persist usage");

        // Take the join of the handle's since and the provided `since`;
        // this lets materialized views express the since at which their
        // read handles "start."
        let since = handle
            .since()
            .join(since.unwrap_or(&Antichain::from_elem(T::minimum())));

        handle.downgrade_since(&since).await;

        handle
    }

    fn register_handles(
        &self,
        id: GlobalId,
        is_in_txns: bool,
        since_handle: SinceHandleWrapper<T>,
        write_handle: WriteHandle<SourceData, (), T, Diff>,
    ) {
        self.send(BackgroundCmd::Register {
            id,
            is_in_txns,
            since_handle,
            write_handle,
        });
    }

    fn send(&self, cmd: BackgroundCmd<T>) {
        let _ = self.cmd_tx.send(cmd);
    }

    async fn snapshot_stats_inner(
        &self,
        id: GlobalId,
        as_of: SnapshotStatsAsOf<T>,
    ) -> Result<SnapshotStats, StorageError<T>> {
        // TODO: Pull this out of BackgroundTask. Unlike the other methods, the
        // caller of this one drives it to completion.
        //
        // We'd need to either share the critical handle somehow or maybe have
        // two instances around, one in the worker and one in the
        // StorageCollections.
        let (tx, rx) = oneshot::channel();
        self.send(BackgroundCmd::SnapshotStats(id, as_of, tx));
        rx.await.expect("BackgroundTask should be live").0.await
    }

    /// If this identified collection has a dependency, install a read hold on
    /// it.
    ///
    /// This is necessary to ensure that the dependency's since does not advance
    /// beyond its dependents'.
    fn install_collection_dependency_read_holds_inner(
        &self,
        self_collections: &mut BTreeMap<GlobalId, CollectionState<T>>,
        id: GlobalId,
    ) -> Result<(), StorageError<T>> {
        let (deps, collection_implied_capability) = match self_collections.get(&id) {
            Some(CollectionState {
                storage_dependencies: deps,
                implied_capability,
                ..
            }) => (deps.clone(), implied_capability),
            _ => return Ok(()),
        };

        for dep in deps.iter() {
            let dep_collection = self_collections
                .get(dep)
                .ok_or(StorageError::IdentifierMissing(id))?;

            mz_ore::soft_assert_or_log!(
                PartialOrder::less_equal(
                    &dep_collection.implied_capability,
                    collection_implied_capability
                ),
                "dependency since ({dep}@{:?}) cannot be in advance of dependent's since ({id}@{:?})",
                dep_collection.implied_capability,
                collection_implied_capability,
            );
        }

        self.install_read_capabilities_inner(
            self_collections,
            id,
            &deps,
            collection_implied_capability.clone(),
        )?;

        Ok(())
    }

    /// Determine if this collection has another dependency.
    ///
    /// Currently, collections have either 0 or 1 dependencies.
    fn determine_collection_dependencies(
        &self,
        self_collections: &BTreeMap<GlobalId, CollectionState<T>>,
        data_source: &DataSource,
    ) -> Result<Vec<GlobalId>, StorageError<T>> {
        let dependencies = match &data_source {
            DataSource::Introspection(_)
            | DataSource::Webhook
            | DataSource::Other(DataSourceOther::TableWrites)
            | DataSource::Progress
            | DataSource::Other(DataSourceOther::Compute) => Vec::new(),
            DataSource::IngestionExport { ingestion_id, .. } => {
                // Ingestion exports depend on their primary source's remap
                // collection.
                let source_collection = self_collections
                    .get(ingestion_id)
                    .ok_or(StorageError::IdentifierMissing(*ingestion_id))?;
                match &source_collection.description {
                    CollectionDescription {
                        data_source: DataSource::Ingestion(ingestion_desc),
                        ..
                    } => vec![ingestion_desc.remap_collection_id],
                    _ => unreachable!(
                        "SourceExport must only refer to primary sources that already exist"
                    ),
                }
            }
            // Ingestions depend on their remap collection.
            DataSource::Ingestion(ingestion) => vec![ingestion.remap_collection_id],
        };

        Ok(dependencies)
    }

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

        let user_capabilities = self_collections
            .iter_mut()
            .filter(|(id, _c)| id.is_user())
            .map(|(id, c)| {
                let updates = c.read_capabilities.updates().cloned().collect_vec();
                (*id, c.implied_capability.clone(), updates)
            })
            .collect_vec();

        trace!(
            %from_id,
            ?storage_dependencies,
            ?read_capability,
            ?user_capabilities,
            "install_read_capabilities_inner");

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

        StorageCollectionsImpl::update_read_capabilities_inner(
            &self.cmd_tx,
            self_collections,
            &mut storage_read_updates,
        );

        let user_capabilities = self_collections
            .iter_mut()
            .filter(|(id, _c)| id.is_user())
            .map(|(id, c)| {
                let updates = c.read_capabilities.updates().cloned().collect_vec();
                (*id, c.implied_capability.clone(), updates)
            })
            .collect_vec();

        trace!(
            %from_id,
            ?storage_dependencies,
            ?read_capability,
            ?user_capabilities,
            "after install_read_capabilities_inner!");

        Ok(())
    }

    async fn read_handle_for_snapshot(
        &self,
        metadata: &CollectionMetadata,
        id: GlobalId,
    ) -> Result<ReadHandle<SourceData, (), T, Diff>, StorageError<T>> {
        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 continually.
        // The assumption is that calls to snapshot are rare and therefore worth
        // it to always create a new handle.
        let read_handle = persist_client
            .open_leased_reader::<SourceData, (), _, _>(
                metadata.data_shard,
                Arc::new(metadata.relation_desc.clone()),
                Arc::new(UnitSchema),
                Diagnostics {
                    shard_name: id.to_string(),
                    handle_purpose: format!("snapshot {}", id),
                },
                USE_CRITICAL_SINCE_SNAPSHOT.get(&self.persist.cfg),
            )
            .await
            .expect("invalid persist usage");
        Ok(read_handle)
    }

    fn set_read_policies_inner(
        &self,
        collections: &mut BTreeMap<GlobalId, CollectionState<T>>,
        policies: Vec<(GlobalId, ReadPolicy<T>)>,
    ) {
        trace!("set_read_policies: {:?}", policies);

        let mut read_capability_changes = BTreeMap::default();

        for (id, policy) in policies.into_iter() {
            let collection = match collections.get_mut(&id) {
                Some(c) => c,
                None => {
                    panic!("Reference to absent collection {id}");
                }
            };

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

            if 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;
        }

        for (id, changes) in read_capability_changes.iter() {
            if id.is_user() {
                trace!(%id, ?changes, "in set_read_policies, capability changes");
            }
        }

        if !read_capability_changes.is_empty() {
            StorageCollectionsImpl::update_read_capabilities_inner(
                &self.cmd_tx,
                collections,
                &mut read_capability_changes,
            );
        }
    }

    // This is not an associated function so that we can share it with the task
    // that updates the persist handles and also has a reference to the shared
    // collections state.
    fn update_read_capabilities_inner(
        cmd_tx: &mpsc::UnboundedSender<BackgroundCmd<T>>,
        collections: &mut BTreeMap<GlobalId, CollectionState<T>>,
        updates: &mut BTreeMap<GlobalId, ChangeBatch<T>>,
    ) {
        // Location to record consequences that we need to act on.
        let mut collections_net = BTreeMap::new();

        // We must not rely on any specific relative ordering of `GlobalId`s.
        // That said, it is reasonable to assume that collections generally have
        // greater IDs than their dependencies, so starting with the largest is
        // a useful optimization.
        while let Some(id) = updates.keys().rev().next().cloned() {
            let mut update = updates.remove(&id).unwrap();

            if id.is_user() {
                trace!(id = ?id, update = ?update, "update_read_capabilities");
            }

            let collection = if let Some(c) = collections.get_mut(&id) {
                c
            } else {
                let has_positive_updates = update.iter().any(|(_ts, diff)| *diff > 0);
                if has_positive_updates {
                    panic!(
                        "reference to absent collection {id} but we have positive updates: {:?}",
                        update
                    );
                } else {
                    // Continue purely negative updates. Someone has probably
                    // already dropped this collection!
                    continue;
                }
            };

            let current_read_capabilities = collection.read_capabilities.frontier().to_owned();
            for (time, diff) in update.iter() {
                assert!(
                    collection.read_capabilities.count_for(time) + diff >= 0,
                    "update {:?} for collection {id} would lead to negative \
                        read capabilities, read capabilities before applying: {:?}",
                    update,
                    collection.read_capabilities
                );

                if collection.read_capabilities.count_for(time) + diff > 0 {
                    assert!(
                        current_read_capabilities.less_equal(time),
                        "update {:?} for collection {id} is trying to \
                            install read capabilities before the current \
                            frontier of read capabilities, read capabilities before applying: {:?}",
                        update,
                        collection.read_capabilities
                    );
                }
            }

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

            if id.is_user() {
                trace!(
                %id,
                ?collection.storage_dependencies,
                ?update,
                "forwarding update to storage dependencies");
            }

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

            let (changes, frontier) = collections_net
                .entry(id)
                .or_insert_with(|| (<ChangeBatch<_>>::new(), Antichain::new()));

            changes.extend(update.drain());
            *frontier = collection.read_capabilities.frontier().to_owned();
        }

        // Translate our net compute actions into downgrades of persist sinces.
        // The actual downgrades are performed by a Tokio task asynchronously.
        let mut persist_compaction_commands = Vec::with_capacity(collections_net.len());
        for (key, (mut changes, frontier)) in collections_net {
            if !changes.is_empty() {
                if frontier.is_empty() {
                    info!(id = %key, "removing collection state because the since advanced to []!");
                    collections.remove(&key).expect("must still exist");
                }
                persist_compaction_commands.push((key, frontier));
            }
        }

        if !persist_compaction_commands.is_empty() {
            cmd_tx
                .send(BackgroundCmd::DowngradeSince(persist_compaction_commands))
                .expect("cannot fail to send");
        }
    }

    /// Remove any shards that we know are finalized
    fn synchronize_finalized_shards(&self, storage_metadata: &StorageMetadata) {
        self.finalized_shards
            .lock()
            .retain(|shard| storage_metadata.unfinalized_shards.contains(shard));
    }
}

// See comments on the above impl for StorageCollectionsImpl.
#[async_trait]
impl<T> StorageCollections for StorageCollectionsImpl<T>
where
    T: TimelyTimestamp
        + Lattice
        + Codec64
        + From<EpochMillis>
        + TimestampManipulation
        + Into<mz_repr::Timestamp>,
{
    type Timestamp = T;

    async fn initialize_state(
        &self,
        txn: &mut (dyn StorageTxn<T> + Send),
        init_ids: BTreeSet<GlobalId>,
        drop_ids: BTreeSet<GlobalId>,
    ) -> Result<(), StorageError<T>> {
        let metadata = txn.get_collection_metadata();
        let existing_metadata: BTreeSet<_> = metadata.into_iter().map(|(id, _)| id).collect();

        // Determine which collections we do not yet have metadata for.
        let new_collections: BTreeSet<GlobalId> =
            init_ids.difference(&existing_metadata).cloned().collect();

        self.prepare_state(txn, new_collections, drop_ids).await?;

        // All shards that belong to collections dropped in the last epoch are
        // eligible for finalization. This intentionally includes any built-in
        // collections present in `drop_ids`.
        //
        // n.b. this introduces an unlikely race condition: if a collection is
        // dropped from the catalog, but the dataflow is still running on a
        // worker, assuming the shard is safe to finalize on reboot may cause
        // the cluster to panic.
        let unfinalized_shards = txn.get_unfinalized_shards().into_iter().collect_vec();

        info!(?unfinalized_shards, "initializing finalizable_shards");

        self.finalizable_shards.lock().extend(unfinalized_shards);

        Ok(())
    }

    fn update_parameters(&self, config_params: StorageParameters) {
        // We serialize the dyncfg updates in StorageParameters, but configure
        // persist separately.
        config_params.dyncfg_updates.apply(self.persist.cfg());

        self.config
            .lock()
            .expect("lock poisoned")
            .update(config_params);
    }

    fn collection_metadata(
        &self,
        id: GlobalId,
    ) -> Result<CollectionMetadata, StorageError<Self::Timestamp>> {
        let collections = self.collections.lock().expect("lock poisoned");

        collections
            .get(&id)
            .map(|c| c.collection_metadata.clone())
            .ok_or(StorageError::IdentifierMissing(id))
    }

    fn active_collection_metadatas(&self) -> Vec<(GlobalId, CollectionMetadata)> {
        let collections = self.collections.lock().expect("lock poisoned");

        collections
            .iter()
            .filter(|(_id, c)| !c.is_dropped())
            .map(|(id, c)| (*id, c.collection_metadata.clone()))
            .collect()
    }

    fn collections_frontiers(
        &self,
        ids: Vec<GlobalId>,
    ) -> Result<Vec<CollectionFrontiers<Self::Timestamp>>, StorageError<Self::Timestamp>> {
        let collections = self.collections.lock().expect("lock poisoned");

        let res = ids
            .into_iter()
            .map(|id| {
                collections
                    .get(&id)
                    .map(|c| CollectionFrontiers {
                        id: id.clone(),
                        write_frontier: c.write_frontier.clone(),
                        implied_capability: c.implied_capability.clone(),
                        read_capabilities: c.read_capabilities.frontier().to_owned(),
                    })
                    .ok_or(StorageError::IdentifierMissing(id))
            })
            .collect::<Result<Vec<_>, _>>()?;

        Ok(res)
    }

    fn active_collection_frontiers(&self) -> Vec<CollectionFrontiers<Self::Timestamp>> {
        let collections = self.collections.lock().expect("lock poisoned");

        let res = collections
            .iter()
            .filter(|(_id, c)| !c.is_dropped())
            .map(|(id, c)| CollectionFrontiers {
                id: id.clone(),
                write_frontier: c.write_frontier.clone(),
                implied_capability: c.implied_capability.clone(),
                read_capabilities: c.read_capabilities.frontier().to_owned(),
            })
            .collect_vec();

        res
    }

    async fn snapshot_stats(
        &self,
        id: GlobalId,
        as_of: Antichain<Self::Timestamp>,
    ) -> Result<SnapshotStats, StorageError<Self::Timestamp>> {
        let metadata = self.collection_metadata(id)?;

        // See the comments in StorageController::snapshot for what's going on
        // here.
        let as_of = match metadata.txns_shard.as_ref() {
            None => SnapshotStatsAsOf::Direct(as_of),
            Some(txns_id) => {
                assert_eq!(txns_id, self.txns_read.txns_id());
                let as_of = as_of
                    .into_option()
                    .expect("cannot read as_of the empty antichain");
                self.txns_read.update_gt(as_of.clone()).await;
                let data_snapshot = self
                    .txns_read
                    .data_snapshot(metadata.data_shard, as_of.clone())
                    .await;
                SnapshotStatsAsOf::Txns(data_snapshot)
            }
        };
        self.snapshot_stats_inner(id, as_of).await
    }

    async fn snapshot_parts_stats(
        &self,
        id: GlobalId,
        as_of: Antichain<Self::Timestamp>,
    ) -> BoxFuture<'static, Result<SnapshotPartsStats, StorageError<Self::Timestamp>>> {
        let metadata = {
            let self_collections = self.collections.lock().expect("lock poisoned");

            let collection_metadata = self_collections
                .get(&id)
                .ok_or(StorageError::IdentifierMissing(id))
                .map(|c| c.collection_metadata.clone());

            match collection_metadata {
                Ok(m) => m,
                Err(e) => return Box::pin(async move { Err(e) }),
            }
        };

        // See the comments in StorageController::snapshot for what's going on
        // here.
        let read_handle = self.read_handle_for_snapshot(&metadata, id).await;

        let data_snapshot = match (metadata, as_of.as_option()) {
            (
                CollectionMetadata {
                    txns_shard: Some(txns_id),
                    data_shard,
                    ..
                },
                Some(as_of),
            ) => {
                assert_eq!(txns_id, *self.txns_read.txns_id());
                self.txns_read.update_gt(as_of.clone()).await;
                let data_snapshot = self
                    .txns_read
                    .data_snapshot(data_shard, as_of.clone())
                    .await;
                Some(data_snapshot)
            }
            _ => None,
        };

        Box::pin(async move {
            let mut read_handle = read_handle?;
            let result = match data_snapshot {
                Some(data_snapshot) => data_snapshot.snapshot_parts_stats(&mut read_handle).await,
                None => read_handle.snapshot_parts_stats(as_of).await,
            };
            read_handle.expire().await;
            result.map_err(|_| StorageError::ReadBeforeSince(id))
        })
    }

    fn check_exists(&self, id: GlobalId) -> Result<(), StorageError<Self::Timestamp>> {
        let collections = self.collections.lock().expect("lock poisoned");

        if collections.contains_key(&id) {
            Ok(())
        } else {
            Err(StorageError::IdentifierMissing(id))
        }
    }

    async fn prepare_state(
        &self,
        txn: &mut (dyn StorageTxn<Self::Timestamp> + Send),
        ids_to_add: BTreeSet<GlobalId>,
        ids_to_drop: BTreeSet<GlobalId>,
    ) -> Result<(), StorageError<T>> {
        txn.insert_collection_metadata(
            ids_to_add
                .into_iter()
                .map(|id| (id, ShardId::new()))
                .collect(),
        )?;

        // Delete the metadata for any dropped collections.
        let dropped_mappings = txn.delete_collection_metadata(ids_to_drop);

        let dropped_shards = dropped_mappings
            .into_iter()
            .map(|(_id, shard)| shard)
            .collect();

        txn.insert_unfinalized_shards(dropped_shards)?;

        // Reconcile any shards we've successfully finalized with the shard
        // finalization collection.
        let finalized_shards = self.finalized_shards.lock().iter().copied().collect();
        txn.mark_shards_as_finalized(finalized_shards);

        Ok(())
    }

    // TODO(aljoscha): It would be swell if we could refactor this Leviathan of
    // a method/move individual parts to their own methods.
    #[instrument(level = "debug")]
    async fn create_collections_for_bootstrap(
        &self,
        storage_metadata: &StorageMetadata,
        register_ts: Option<Self::Timestamp>,
        mut collections: Vec<(GlobalId, CollectionDescription<Self::Timestamp>)>,
        migrated_storage_collections: &BTreeSet<GlobalId>,
    ) -> Result<(), StorageError<Self::Timestamp>> {
        let is_in_txns = |id, metadata: &CollectionMetadata| {
            metadata.txns_shard.is_some()
                && !(self.read_only && migrated_storage_collections.contains(&id))
        };

        // 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));
            }
        }

        {
            // Early sanity check: if we knew about a collection already it's
            // description must match!
            //
            // NOTE: There could be concurrent modifications to
            // `self.collections`, but this sanity check is better than nothing.
            let self_collections = self.collections.lock().expect("lock poisoned");
            for (id, description) in collections.iter() {
                if let Some(existing_collection) = self_collections.get(id) {
                    if &existing_collection.description != description {
                        return Err(StorageError::SourceIdReused(*id));
                    }
                }
            }
        }

        // We first enrich each collection description with some additional
        // metadata...
        let enriched_with_metadata = collections
            .into_iter()
            .map(|(id, description)| {
                let data_shard = storage_metadata.get_collection_shard::<T>(id)?;

                let get_shard = |id| -> Result<ShardId, StorageError<T>> {
                    let shard = storage_metadata.get_collection_shard::<T>(id)?;
                    Ok(shard)
                };

                let status_shard = match description.status_collection_id {
                    Some(status_collection_id) => Some(get_shard(status_collection_id)?),
                    None => 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(get_shard(*remap_collection_id)?)
                    }
                    _ => None,
                };

                // If the shard is being managed by txn-wal (initially,
                // tables), then we need to pass along the shard id for the txns
                // shard to dataflow rendering.
                let txns_shard = description
                    .data_source
                    .in_txns()
                    .then(|| *self.txns_read.txns_id());

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

                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.
        use futures::stream::{StreamExt, TryStreamExt};
        let this = &*self;
        let mut to_register: Vec<_> = futures::stream::iter(enriched_with_metadata)
            .map(|data: Result<_, StorageError<Self::Timestamp>>| {
                let register_ts = register_ts.clone();
                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, mut since_handle) = this
                    .open_data_handles(
                        &id,
                        metadata.data_shard,
                        description.since.as_ref(),
                        metadata.relation_desc.clone(),
                        persist_client,
                    )
                    .await;

                // Present tables as springing into existence at the register_ts
                // by advancing the since. Otherwise, we could end up in a
                // situation where a table with a long compaction window appears
                // to exist before the environment (and this the table) existed.
                //
                // We could potentially also do the same thing for other
                // sources, in particular storage's internal sources and perhaps
                // others, but leave them for now.
                match description.data_source {
                    DataSource::Introspection(_)
                    | DataSource::IngestionExport { .. }
                    | DataSource::Webhook
                    | DataSource::Ingestion(_)
                    | DataSource::Progress
                    | DataSource::Other(DataSourceOther::Compute) => {},
                    DataSource::Other(DataSourceOther::TableWrites) => {
                        let register_ts = register_ts.expect("caller should have provided a register_ts when creating a table");
                        if since_handle.since().elements() == &[T::minimum()] && !migrated_storage_collections.contains(&id) {
                            debug!("advancing {} to initial since of {:?}", id, register_ts);
                            let token = since_handle.opaque();
                            let _ = since_handle.compare_and_downgrade_since(&token, (&token, &Antichain::from_elem(register_ts.clone()))).await;
                        }
                    }
                }

                Ok::<_, StorageError<Self::Timestamp>>((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?;

        // Reorder in dependency order.
        to_register.sort_by_key(|(id, ..)| *id);

        // We hold this lock for a very short amount of time, just doing some
        // hashmap inserts and unbounded channel sends.
        let mut self_collections = self.collections.lock().expect("lock poisoned");

        for (id, mut description, write_handle, since_handle, metadata) in to_register {
            // Ensure that the ingestion has an export for its primary source.
            // This is done in an awkward spot to appease the borrow checker.
            if let DataSource::Ingestion(ingestion) = &mut description.data_source {
                ingestion
                    .source_exports
                    .insert(id, ingestion.desc.primary_source_export());
            }

            let write_frontier = write_handle.upper();
            let data_shard_since = since_handle.since().clone();

            // Determine if this collection has any dependencies.
            let storage_dependencies = self
                .determine_collection_dependencies(&*self_collections, &description.data_source)?;

            // Determine the initial since of the collection.
            let initial_since = match storage_dependencies
                .iter()
                .at_most_one()
                .expect("should have at most one depdendency")
            {
                Some(dep) => {
                    let dependency_collection = self_collections
                        .get(dep)
                        .ok_or(StorageError::IdentifierMissing(*dep))?;
                    let dependency_since = dependency_collection.implied_capability.clone();

                    // If an item has a dependency, its initial since must be
                    // advanced as far as its dependency, i.e. a dependency's
                    // since may never be in advance of its dependents.
                    //
                    // We have to do this every time we initialize the
                    // collection, though––the invariant might have been upheld
                    // correctly in the previous epoch, but the
                    // `data_shard_since` might not have compacted and, on
                    // establishing a new persist connection, still have data we
                    // said _could_ be compacted.
                    if PartialOrder::less_than(&data_shard_since, &dependency_since) {
                        // The dependency since cannot be beyond the dependent
                        // (our) upper unless the collection is new. In
                        // practice, the depdenency is the remap shard of a
                        // source (export), and if the since is allowed to
                        // "catch up" to the upper, that is `upper <= since`, a
                        // restarting ingestion cannot differentiate between
                        // updates that have already been written out to the
                        // backing persist shard and updates that have yet to be
                        // written. We would write duplicate updates.
                        //
                        // If this check fails, it means that the read hold
                        // installed on the dependency was probably not upheld
                        // –– if it were, the dependency's since could not have
                        // advanced as far the dependent's upper.
                        //
                        // We don't care about the dependency since when the
                        // write frontier is empty. In that case, no-one can
                        // write down any more updates.
                        mz_ore::soft_assert_or_log!(
                            write_frontier.elements() == &[T::minimum()]
                                || write_frontier.is_empty()
                                || PartialOrder::less_than(&dependency_since, write_frontier),
                            "dependency ({dep}) since has advanced past dependent ({id}) upper \n
                            dependent ({id}): since {:?}, upper {:?} \n
                            dependency ({dep}): since {:?}",
                            data_shard_since,
                            write_frontier,
                            dependency_since
                        );

                        dependency_since
                    } else {
                        data_shard_since
                    }
                }
                None => data_shard_since,
            };

            let mut collection_state = CollectionState::new(
                description,
                initial_since,
                write_frontier.clone(),
                storage_dependencies,
                metadata.clone(),
            );

            // Install the collection state in the appropriate spot.
            match &collection_state.description.data_source {
                DataSource::Introspection(_) => {
                    self_collections.insert(id, collection_state);
                }
                DataSource::Webhook => {
                    self_collections.insert(id, collection_state);
                }
                DataSource::IngestionExport {
                    ingestion_id,
                    details,
                    data_config,
                } => {
                    // Adjust the source to contain this export.
                    let source_collection = self_collections
                        .get_mut(ingestion_id)
                        .expect("known to exist");
                    match &mut source_collection.description {
                        CollectionDescription {
                            data_source: DataSource::Ingestion(ingestion_desc),
                            ..
                        } => ingestion_desc.source_exports.insert(
                            id,
                            SourceExport {
                                storage_metadata: (),
                                details: details.clone(),
                                data_config: data_config.clone(),
                            },
                        ),
                        _ => unreachable!(
                            "SourceExport must only refer to primary sources that already exist"
                        ),
                    };

                    self_collections.insert(id, collection_state);
                }
                DataSource::Other(DataSourceOther::TableWrites) => {
                    // See comment on self.initial_txn_upper on why we're doing
                    // this.
                    if is_in_txns(id, &metadata)
                        && PartialOrder::less_than(
                            &collection_state.write_frontier,
                            &self.initial_txn_upper,
                        )
                    {
                        // We could try and be cute and use the join of the txn
                        // upper and the table upper. But that has more
                        // complicated reasoning for why it is or isn't correct,
                        // and we're only dealing with totally ordered times
                        // here.
                        collection_state
                            .write_frontier
                            .clone_from(&self.initial_txn_upper);
                    }
                    self_collections.insert(id, collection_state);
                }
                DataSource::Progress | DataSource::Other(DataSourceOther::Compute) => {
                    self_collections.insert(id, collection_state);
                }
                DataSource::Ingestion(_) => {
                    self_collections.insert(id, collection_state);
                }
            }

            self.register_handles(id, is_in_txns(id, &metadata), since_handle, write_handle);

            // If this collection has a dependency, install a read hold on it.
            self.install_collection_dependency_read_holds_inner(&mut *self_collections, id)?;
        }

        drop(self_collections);

        self.synchronize_finalized_shards(storage_metadata);

        Ok(())
    }

    async fn alter_ingestion_source_desc(
        &self,
        ingestion_id: GlobalId,
        source_desc: SourceDesc,
    ) -> Result<(), StorageError<Self::Timestamp>> {
        // The StorageController checks the validity of these. And we just
        // accept them.

        let mut self_collections = self.collections.lock().expect("lock poisoned");
        let collection = self_collections
            .get_mut(&ingestion_id)
            .ok_or(StorageError::IdentifierMissing(ingestion_id))?;

        let curr_ingestion = match &mut collection.description.data_source {
            DataSource::Ingestion(active_ingestion) => active_ingestion,
            _ => unreachable!("verified collection refers to ingestion"),
        };

        curr_ingestion.desc = source_desc;
        debug!("altered {ingestion_id}'s SourceDesc");

        Ok(())
    }

    async fn alter_ingestion_connections(
        &self,
        source_connections: BTreeMap<GlobalId, GenericSourceConnection<InlinedConnection>>,
    ) -> Result<(), StorageError<Self::Timestamp>> {
        let mut self_collections = self.collections.lock().expect("lock poisoned");

        for (id, conn) in source_connections {
            let collection = self_collections
                .get_mut(&id)
                .ok_or_else(|| StorageError::IdentifierMissing(id))?;

            match &mut collection.description.data_source {
                DataSource::Ingestion(ingestion) => {
                    // If the connection hasn't changed, there's no sense in
                    // re-rendering the dataflow.
                    if ingestion.desc.connection != conn {
                        info!(from = ?ingestion.desc.connection, to = ?conn, "alter_ingestion_connections, updating");
                        ingestion.desc.connection = conn;
                    } else {
                        warn!(
                            "update_source_connection called on {id} but the \
                            connection was the same"
                        );
                    }
                }
                o => {
                    warn!("update_source_connection called on {:?}", o);
                    Err(StorageError::IdentifierInvalid(id))?;
                }
            }
        }

        Ok(())
    }

    async fn alter_table_desc(
        &self,
        table_id: GlobalId,
        new_desc: RelationDesc,
    ) -> Result<(), StorageError<Self::Timestamp>> {
        let mut self_collections = self.collections.lock().expect("lock poisoned");
        let collection = self_collections
            .get_mut(&table_id)
            .ok_or_else(|| StorageError::IdentifierMissing(table_id))?;

        // TODO(parkmycar): To support changing the `RelationDesc` of sources
        // we'll need to cancel the currently running `BackgroundCmd` that
        // fetches recent uppers. See `BackgroundCmd::Register`.
        if !matches!(
            &collection.description.data_source,
            DataSource::Other(DataSourceOther::TableWrites)
        ) {
            return Err(StorageError::IdentifierInvalid(table_id));
        }

        collection.collection_metadata.relation_desc = new_desc.clone();
        collection.description.desc = new_desc.clone();

        debug!("altered table {table_id}'s RelationDesc");

        Ok(())
    }

    fn drop_collections_unvalidated(
        &self,
        storage_metadata: &StorageMetadata,
        identifiers: Vec<GlobalId>,
    ) {
        debug!(?identifiers, "drop_collections_unvalidated");

        let mut self_collections = self.collections.lock().expect("lock poisoned");

        for id in identifiers.iter() {
            let metadata = storage_metadata.get_collection_shard::<T>(*id);
            mz_ore::soft_assert_or_log!(
                matches!(metadata, Err(StorageError::IdentifierMissing(_))),
                "dropping {id}, but drop was not synchronized with storage \
                controller via `synchronize_collections`"
            );

            let dropped_data_source = match self_collections.get(id) {
                Some(col) => col.description.data_source.clone(),
                None => continue,
            };

            // If we are dropping source exports, we need to modify the
            // ingestion that it runs on.
            if let DataSource::IngestionExport { ingestion_id, .. } = dropped_data_source {
                // Adjust the source to remove this export.
                let ingestion = match self_collections.get_mut(&ingestion_id) {
                    Some(ingestion) => ingestion,
                    // Primary ingestion already dropped.
                    None => {
                        tracing::error!(
                            "primary source {ingestion_id} seemingly dropped before subsource {id}",
                        );
                        continue;
                    }
                };

                match &mut ingestion.description {
                    CollectionDescription {
                        data_source: DataSource::Ingestion(ingestion_desc),
                        ..
                    } => {
                        let removed = ingestion_desc.source_exports.remove(id);
                        mz_ore::soft_assert_or_log!(
                            removed.is_some(),
                            "dropped subsource {id} already removed from source exports"
                        );
                    }
                    _ => unreachable!(
                        "SourceExport must only refer to primary sources that already exist"
                    ),
                };
            }
        }

        // Policies that advance the since to the empty antichain. We do still
        // honor outstanding read holds, and collections will only be dropped
        // once those are removed as well.
        //
        // We don't explicitly remove read capabilities! Downgrading the
        // frontier of the source to `[]` (the empty Antichain), will propagate
        // to the storage dependencies.
        let mut finalized_policies = Vec::new();

        for id in identifiers {
            // Make sure it's still there, might already have been deleted.
            if self_collections.contains_key(&id) {
                finalized_policies.push((id, ReadPolicy::ValidFrom(Antichain::new())));
            }
        }
        self.set_read_policies_inner(&mut self_collections, finalized_policies);

        drop(self_collections);

        self.synchronize_finalized_shards(storage_metadata);
    }

    fn set_read_policies(&self, policies: Vec<(GlobalId, ReadPolicy<Self::Timestamp>)>) {
        let mut collections = self.collections.lock().expect("lock poisoned");

        let user_capabilities = collections
            .iter_mut()
            .filter(|(id, _c)| id.is_user())
            .map(|(id, c)| {
                let updates = c.read_capabilities.updates().cloned().collect_vec();
                (*id, c.implied_capability.clone(), updates)
            })
            .collect_vec();

        trace!(?policies, ?user_capabilities, "set_read_policies");

        self.set_read_policies_inner(&mut collections, policies);

        let user_capabilities = collections
            .iter_mut()
            .filter(|(id, _c)| id.is_user())
            .map(|(id, c)| {
                let updates = c.read_capabilities.updates().cloned().collect_vec();
                (*id, c.implied_capability.clone(), updates)
            })
            .collect_vec();

        trace!(?user_capabilities, "after! set_read_policies");
    }

    fn acquire_read_holds(
        &self,
        desired_holds: Vec<GlobalId>,
    ) -> Result<Vec<ReadHold<Self::Timestamp>>, ReadHoldError> {
        let mut collections = self.collections.lock().expect("lock poisoned");

        let mut advanced_holds = Vec::new();
        // We advance the holds by our current since frontier. Can't acquire
        // holds for times that have been compacted away!
        //
        // NOTE: We acquire read holds at the earliest possible time rather than
        // at the implied capability. This is so that, for example, adapter can
        // acquire a read hold to hold back the frontier, giving the COMPUTE
        // controller a chance to also acquire a read hold at that early
        // frontier. If/when we change the interplay between adapter and COMPUTE
        // to pass around ReadHold tokens, we might tighten this up and instead
        // acquire read holds at the implied capability.
        for id in desired_holds.iter() {
            let collection = collections
                .get(id)
                .ok_or(ReadHoldError::CollectionMissing(*id))?;
            let since = collection.read_capabilities.frontier().to_owned();
            advanced_holds.push((*id, since));
        }

        let mut updates = advanced_holds
            .iter()
            .map(|(id, hold)| {
                let mut changes = ChangeBatch::new();
                changes.extend(hold.iter().map(|time| (time.clone(), 1)));
                (*id, changes)
            })
            .collect::<BTreeMap<_, _>>();

        StorageCollectionsImpl::update_read_capabilities_inner(
            &self.cmd_tx,
            &mut collections,
            &mut updates,
        );

        let acquired_holds = advanced_holds
            .into_iter()
            .map(|(id, since)| ReadHold::new(id, since, self.holds_tx.clone()))
            .collect_vec();

        trace!(?desired_holds, ?acquired_holds, "acquire_read_holds");

        Ok(acquired_holds)
    }
}

/// Wraps either a "critical" [SinceHandle] or a leased [ReadHandle].
///
/// When a [StorageCollections] is in read-only mode, we will only ever acquire
/// [ReadHandle], because acquiring the [SinceHandle] and driving forward its
/// since is considered a write. Conversely, when in read-write mode, we acquire
/// [SinceHandle].
#[derive(Debug)]
enum SinceHandleWrapper<T>
where
    T: TimelyTimestamp + Lattice + Codec64,
{
    Critical(SinceHandle<SourceData, (), T, Diff, PersistEpoch>),
    Leased(ReadHandle<SourceData, (), T, Diff>),
}

impl<T> SinceHandleWrapper<T>
where
    T: TimelyTimestamp + Lattice + Codec64 + TotalOrder,
{
    pub fn since(&self) -> &Antichain<T> {
        match self {
            Self::Critical(handle) => handle.since(),
            Self::Leased(handle) => handle.since(),
        }
    }

    pub fn opaque(&self) -> PersistEpoch {
        match self {
            Self::Critical(handle) => handle.opaque().clone(),
            Self::Leased(_handle) => {
                // The opaque is expected to be used with
                // `compare_and_downgrade_since`, and the leased handle doesn't
                // have a notion of an opaque. We pretend here and in
                // `compare_and_downgrade_since`.
                PersistEpoch(None)
            }
        }
    }

    pub async fn compare_and_downgrade_since(
        &mut self,
        expected: &PersistEpoch,
        new: (&PersistEpoch, &Antichain<T>),
    ) -> Result<Antichain<T>, PersistEpoch> {
        match self {
            Self::Critical(handle) => handle.compare_and_downgrade_since(expected, new).await,
            Self::Leased(handle) => {
                let (opaque, since) = new;
                assert_none!(opaque.0);

                handle.downgrade_since(since).await;

                Ok(since.clone())
            }
        }
    }

    pub async fn maybe_compare_and_downgrade_since(
        &mut self,
        expected: &PersistEpoch,
        new: (&PersistEpoch, &Antichain<T>),
    ) -> Option<Result<Antichain<T>, PersistEpoch>> {
        match self {
            Self::Critical(handle) => {
                handle
                    .maybe_compare_and_downgrade_since(expected, new)
                    .await
            }
            Self::Leased(handle) => {
                let (opaque, since) = new;
                assert_none!(opaque.0);

                handle.maybe_downgrade_since(since).await;

                Some(Ok(since.clone()))
            }
        }
    }

    pub fn snapshot_stats(
        &self,
        id: GlobalId,
        as_of: Option<Antichain<T>>,
    ) -> BoxFuture<'static, Result<SnapshotStats, StorageError<T>>> {
        match self {
            Self::Critical(handle) => {
                let res = handle
                    .snapshot_stats(as_of)
                    .map(move |x| x.map_err(|_| StorageError::ReadBeforeSince(id)));
                Box::pin(res)
            }
            Self::Leased(handle) => {
                let res = handle
                    .snapshot_stats(as_of)
                    .map(move |x| x.map_err(|_| StorageError::ReadBeforeSince(id)));
                Box::pin(res)
            }
        }
    }

    pub fn snapshot_stats_from_txn(
        &self,
        id: GlobalId,
        data_snapshot: DataSnapshot<T>,
    ) -> BoxFuture<'static, Result<SnapshotStats, StorageError<T>>> {
        match self {
            Self::Critical(handle) => Box::pin(
                data_snapshot
                    .snapshot_stats_from_critical(handle)
                    .map(move |x| x.map_err(|_| StorageError::ReadBeforeSince(id))),
            ),
            Self::Leased(handle) => Box::pin(
                data_snapshot
                    .snapshot_stats_from_leased(handle)
                    .map(move |x| x.map_err(|_| StorageError::ReadBeforeSince(id))),
            ),
        }
    }
}

/// State maintained about individual collections.
#[derive(Debug)]
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: TimelyTimestamp> 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 whether the collection was dropped.
    pub fn is_dropped(&self) -> bool {
        self.read_capabilities.is_empty()
    }
}

/// A task that keeps persist handles, downgrades sinces when asked,
/// periodically gets recent uppers from them, and updates the shard collection
/// state when needed.
///
/// This shares state with [StorageCollectionsImpl] via `Arcs` and channels.
#[derive(Debug)]
struct BackgroundTask<T: TimelyTimestamp + Lattice + Codec64> {
    config: Arc<Mutex<StorageConfiguration>>,
    cmds_tx: mpsc::UnboundedSender<BackgroundCmd<T>>,
    cmds_rx: mpsc::UnboundedReceiver<BackgroundCmd<T>>,
    holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch<T>)>,
    finalizable_shards: Arc<ShardIdSet>,
    collections: Arc<std::sync::Mutex<BTreeMap<GlobalId, CollectionState<T>>>>,
    // So we know what shard ID corresponds to what global ID, which we need
    // when re-enqueing futures for determining the next upper update.
    shard_by_id: BTreeMap<GlobalId, ShardId>,
    since_handles: BTreeMap<GlobalId, SinceHandleWrapper<T>>,
    txns_handle: Option<WriteHandle<SourceData, (), T, Diff>>,
    txns_shards: BTreeSet<GlobalId>,
}

#[derive(Debug)]
enum BackgroundCmd<T: TimelyTimestamp + Lattice + Codec64> {
    Register {
        id: GlobalId,
        is_in_txns: bool,
        write_handle: WriteHandle<SourceData, (), T, Diff>,
        since_handle: SinceHandleWrapper<T>,
    },
    DowngradeSince(Vec<(GlobalId, Antichain<T>)>),
    SnapshotStats(
        GlobalId,
        SnapshotStatsAsOf<T>,
        oneshot::Sender<SnapshotStatsRes<T>>,
    ),
}

/// A newtype wrapper to hang a Debug impl off of.
pub(crate) struct SnapshotStatsRes<T>(BoxFuture<'static, Result<SnapshotStats, StorageError<T>>>);

impl<T> Debug for SnapshotStatsRes<T> {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("SnapshotStatsRes").finish_non_exhaustive()
    }
}

impl<T> BackgroundTask<T>
where
    T: TimelyTimestamp
        + Lattice
        + Codec64
        + From<EpochMillis>
        + TimestampManipulation
        + Into<mz_repr::Timestamp>,
{
    async fn run(&mut self) {
        // Futures that fetch the recent upper from all other shards.
        let mut upper_futures: FuturesUnordered<
            std::pin::Pin<
                Box<
                    dyn Future<
                            Output = (GlobalId, WriteHandle<SourceData, (), T, i64>, Antichain<T>),
                        > + Send,
                >,
            >,
        > = FuturesUnordered::new();

        let gen_upper_future = |id: GlobalId, mut handle: WriteHandle<SourceData, (), T, i64>| {
            let fut = async move {
                let current_upper = handle.shared_upper();
                handle.wait_for_upper_past(&current_upper).await;
                let new_upper = handle.shared_upper();
                (id, handle, new_upper)
            };

            fut
        };

        let mut txns_upper_future = match self.txns_handle.take() {
            Some(txns_handle) => {
                let txns_upper_future = gen_upper_future(GlobalId::Transient(1), txns_handle);
                txns_upper_future.boxed()
            }
            None => async { std::future::pending().await }.boxed(),
        };

        loop {
            tokio::select! {
                (id, handle, upper) = &mut txns_upper_future => {
                    trace!("new upper from txns shard: {:?}", upper);
                    let mut uppers = Vec::new();
                    for id in self.txns_shards.iter() {
                        uppers.push((id.clone(), upper.clone()));
                    }
                    self.update_write_frontiers(&uppers).await;

                    let fut = gen_upper_future(id, handle);
                    txns_upper_future = fut.boxed();
                }
                Some((id, handle, upper)) = upper_futures.next() => {
                    if id.is_user() {
                        trace!("new upper for collection {id}: {:?}", upper);
                    }
                    let current_shard = self.shard_by_id.get(&id);
                    if let Some(shard_id) = current_shard {
                        if shard_id == &handle.shard_id() {
                            // Still current, so process the update and enqueue
                            // again!
                            let uppers = vec![(id, upper)];
                            self.update_write_frontiers(&uppers).await;
                            let fut = gen_upper_future(id, handle);
                            upper_futures.push(fut.boxed());
                        } else {
                            // Be polite and expire the write handle. This can
                            // happen when we get an upper update for a write
                            // handle that has since been replaced via Update.
                            handle.expire().await;
                        }
                    }
                }
                cmd = self.cmds_rx.recv() => {
                    let cmd = if let Some(cmd) = cmd {
                        cmd
                    } else {
                        // We're done!
                        break;
                    };

                    match cmd {
                        BackgroundCmd::Register{ id, is_in_txns, write_handle, since_handle } => {
                            debug!("registering handles for {}", id);
                            let previous = self.shard_by_id.insert(id, write_handle.shard_id());
                            if previous.is_some() {
                                panic!("already registered a WriteHandle for collection {id}");
                            }

                            let previous = self.since_handles.insert(id, since_handle);
                            if previous.is_some() {
                                panic!("already registered a SinceHandle for collection {id}");
                            }

                            if is_in_txns {
                                self.txns_shards.insert(id);
                            } else {
                                let fut = gen_upper_future(id, write_handle);
                                upper_futures.push(fut.boxed());
                            }

                        }
                        BackgroundCmd::DowngradeSince(cmds) => {
                            self.downgrade_sinces(cmds).await;
                        }
                        BackgroundCmd::SnapshotStats(id, as_of, tx) => {
                            // NB: The requested as_of could be arbitrarily far
                            // in the future. So, in order to avoid blocking
                            // this loop until it's available and the
                            // `snapshot_stats` call resolves, instead return
                            // the future to the caller and await it there.
                            let res = match self.since_handles.get(&id) {
                                Some(x) => {
                                    let fut: BoxFuture<
                                        'static,
                                        Result<SnapshotStats, StorageError<T>>,
                                    > = match as_of {
                                        SnapshotStatsAsOf::Direct(as_of) => {
                                            x.snapshot_stats(id, Some(as_of))
                                        }
                                        SnapshotStatsAsOf::Txns(data_snapshot) => {
                                            x.snapshot_stats_from_txn(id, data_snapshot)
                                        }
                                    };
                                    SnapshotStatsRes(fut)
                                }
                                None => SnapshotStatsRes(Box::pin(futures::future::ready(Err(
                                    StorageError::IdentifierMissing(id),
                                )))),
                            };
                            // It's fine if the listener hung up.
                            let _ = tx.send(res);
                        }
                    }
                }
                Some(holds_changes) = self.holds_rx.recv() => {
                    let mut batched_changes = BTreeMap::new();
                    batched_changes.insert(holds_changes.0, holds_changes.1);

                    while let Ok(mut holds_changes) = self.holds_rx.try_recv() {
                        let entry = batched_changes.entry(holds_changes.0);
                        entry
                            .and_modify(|existing| existing.extend(holds_changes.1.drain()))
                            .or_insert_with(|| holds_changes.1);
                    }

                    let mut collections = self.collections.lock().expect("lock poisoned");

                    let user_changes = batched_changes
                        .iter()
                        .filter(|(id, _c)| id.is_user())
                        .map(|(id, c)| {
                            (id.clone(), c.clone())
                        })
                        .collect_vec();

                    if !user_changes.is_empty() {
                        trace!(?user_changes, "applying holds changes from channel");
                    }

                    StorageCollectionsImpl::update_read_capabilities_inner(
                        &self.cmds_tx,
                        &mut collections,
                        &mut batched_changes,
                    );
                }
            }
        }

        warn!("BackgroundTask shutting down");
    }

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

        let mut self_collections = self.collections.lock().expect("lock poisoned");

        for (id, new_upper) in updates.iter() {
            let collection = if let Some(c) = self_collections.get_mut(id) {
                c
            } else {
                trace!("Reference to absent collection {id}, due to concurrent removal of that collection");
                continue;
            };

            if PartialOrder::less_than(&collection.write_frontier, new_upper) {
                collection.write_frontier.clone_from(new_upper);
            }

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

            if id.is_user() {
                trace!(
                    %id,
                    implied_capability = ?collection.implied_capability,
                    policy = ?collection.read_policy,
                    write_frontier = ?collection.write_frontier,
                    ?new_read_capability,
                    "update_write_frontiers");
            }

            if 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);
                }
            }
        }

        if !read_capability_changes.is_empty() {
            StorageCollectionsImpl::update_read_capabilities_inner(
                &self.cmds_tx,
                &mut self_collections,
                &mut read_capability_changes,
            );
        }
    }

    async fn downgrade_sinces(&mut self, cmds: Vec<(GlobalId, Antichain<T>)>) {
        for (id, new_since) in cmds {
            let since_handle = if let Some(c) = self.since_handles.get_mut(&id) {
                c
            } else {
                // This can happen when someone concurrently drops a collection.
                trace!("downgrade_sinces: reference to absent collection {id}");
                continue;
            };

            if id.is_user() {
                trace!("downgrading since of {} to {:?}", id, new_since);
            }

            let epoch = since_handle.opaque().clone();
            let result = if new_since.is_empty() {
                // A shard's since reaching the empty frontier is a prereq for
                // being able to finalize a shard, so the final downgrade should
                // never be rate-limited.
                let res = Some(
                    since_handle
                        .compare_and_downgrade_since(&epoch, (&epoch, &new_since))
                        .await,
                );

                info!(%id, "removing persist handles because the since advanced to []!");

                let _since_handle = self.since_handles.remove(&id).expect("known to exist");
                let dropped_shard_id = if let Some(shard_id) = self.shard_by_id.remove(&id) {
                    shard_id
                } else {
                    panic!("missing GlobalId -> ShardId mapping for id {id}");
                };

                // We're not responsible for writes to tables, so we also don't
                // de-register them from the txn system. Whoever is responsible
                // will remove them. We only make sure to remove the table from
                // our tracking.
                self.txns_shards.remove(&id);

                if !self
                    .config
                    .lock()
                    .expect("lock poisoned")
                    .parameters
                    .finalize_shards
                {
                    info!("not triggering shard finalization due to dropped storage object because enable_storage_shard_finalization parameter is false");
                    return;
                }

                info!(%id, %dropped_shard_id, "enqueing shard finalization due to dropped collection and dropped persist handle");

                self.finalizable_shards.lock().insert(dropped_shard_id);

                res
            } else {
                since_handle
                    .maybe_compare_and_downgrade_since(&epoch, (&epoch, &new_since))
                    .await
            };

            if let Some(Err(other_epoch)) = result {
                mz_ore::halt!("fenced by envd @ {other_epoch:?}. ours = {epoch:?}");
            }
        }
    }
}

struct FinalizeShardsTaskConfig {
    envd_epoch: NonZeroI64,
    config: Arc<Mutex<StorageConfiguration>>,
    metrics: StorageCollectionsMetrics,
    finalizable_shards: Arc<ShardIdSet>,
    finalized_shards: Arc<ShardIdSet>,
    persist_location: PersistLocation,
    persist: Arc<PersistClientCache>,
    read_only: bool,
}

async fn finalize_shards_task<T>(
    FinalizeShardsTaskConfig {
        envd_epoch,
        config,
        metrics,
        finalizable_shards,
        finalized_shards,
        persist_location,
        persist,
        read_only,
    }: FinalizeShardsTaskConfig,
) where
    T: TimelyTimestamp + Lattice + Codec64,
{
    if read_only {
        info!("disabling shard finalization in read only mode");
        return;
    }

    let mut interval = tokio::time::interval(Duration::from_secs(5));
    interval.set_missed_tick_behavior(MissedTickBehavior::Delay);
    loop {
        interval.tick().await;

        if !config
            .lock()
            .expect("lock poisoned")
            .parameters
            .finalize_shards
        {
            debug!("not triggering shard finalization due to dropped storage object because enable_storage_shard_finalization parameter is false");
            continue;
        }

        let current_finalizable_shards = {
            // We hold the lock for as short as possible and pull our cloned set
            // of shards.
            finalizable_shards.lock().iter().cloned().collect_vec()
        };

        if current_finalizable_shards.is_empty() {
            debug!("no shards to finalize");
            continue;
        }

        debug!(?current_finalizable_shards, "attempting to finalize shards");

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

        let metrics = &metrics;
        let finalizable_shards = &finalizable_shards;
        let finalized_shards = &finalized_shards;
        let persist_client = &persist_client;
        let diagnostics = &Diagnostics::from_purpose("finalizing shards");

        let force_downgrade_since = STORAGE_DOWNGRADE_SINCE_DURING_FINALIZATION
            .get(config.lock().expect("lock poisoned").config_set());

        let epoch = &PersistEpoch::from(envd_epoch);

        futures::stream::iter(current_finalizable_shards.clone())
            .map(|shard_id| async move {
                let persist_client = persist_client.clone();
                let diagnostics = diagnostics.clone();
                let epoch = epoch.clone();

                metrics.finalization_started.inc();

                let is_finalized = persist_client
                    .is_finalized::<SourceData, (), T, Diff>(shard_id, diagnostics)
                    .await
                    .expect("invalid persist usage");

                if is_finalized {
                    debug!(%shard_id, "shard is already finalized!");
                    Some(shard_id)
                } else {
                    debug!(%shard_id, "finalizing shard");
                        let finalize = || async move {
                        // TODO: thread the global ID into the shard finalization WAL
                        let diagnostics = Diagnostics::from_purpose("finalizing shards");

                        let schemas = persist_client.latest_schema::<SourceData, (), T, Diff>(shard_id, diagnostics.clone()).await.expect("codecs have not changed");
                        let (key_schema, val_schema) = match schemas {
                            Some((_, key_schema, val_schema)) => (key_schema, val_schema),
                            None => (RelationDesc::empty(), UnitSchema),
                        };

                        let empty_batch: Vec<((SourceData, ()), T, Diff)> = vec![];
                        let mut write_handle: WriteHandle<SourceData, (), T, Diff> =
                            persist_client
                                .open_writer(
                                    shard_id,
                                    Arc::new(key_schema),
                                    Arc::new(val_schema),
                                    diagnostics,
                                )
                                .await
                                .expect("invalid persist usage");

                        let upper = write_handle.upper();

                        if !upper.is_empty() {
                            let append = write_handle
                                .append(empty_batch, upper.clone(), Antichain::new())
                                .await?;

                            if let Err(e) = append {
                                warn!(%shard_id, "tried to finalize a shard with an advancing upper: {e:?}");
                                return Ok(());
                            }
                        }
                        write_handle.expire().await;

                        if force_downgrade_since {
                            let mut since_handle: SinceHandle<
                                SourceData,
                                (),
                                T,
                                Diff,
                                PersistEpoch,
                            > = persist_client
                                .open_critical_since(
                                    shard_id,
                                    PersistClient::CONTROLLER_CRITICAL_SINCE,
                                    Diagnostics::from_purpose("finalizing shards"),
                                )
                                .await
                                .expect("invalid persist usage");
                            let handle_epoch = since_handle.opaque().clone();
                            let our_epoch = epoch.clone();
                            let epoch = if our_epoch.0 > handle_epoch.0 {
                                // We're newer, but it's fine to use the
                                // handle's old epoch to try and downgrade.
                                handle_epoch
                            } else {
                                // Good luck, buddy! The downgrade below will
                                // not succeed. There's a process with a newer
                                // epoch out there and someone at some juncture
                                // will fence out this process.
                                our_epoch
                            };
                            let new_since = Antichain::new();
                            let downgrade = since_handle
                                .compare_and_downgrade_since(&epoch, (&epoch, &new_since))
                                .await;
                            if let Err(e) = downgrade {
                                warn!(
                                    "tried to finalize a shard with an advancing epoch: {e:?}"
                                );
                                return Ok(());
                            }
                            // Not available now, so finalization is broken.
                            // since_handle.expire().await;
                        }

                        persist_client
                            .finalize_shard::<SourceData, (), T, Diff>(
                                shard_id,
                                Diagnostics::from_purpose("finalizing shards"),
                            )
                            .await
                    };

                    match finalize().await {
                        Err(e) => {
                            // Rather than error, just leave this shard as
                            // one to finalize later.
                            warn!("error during finalization of shard {shard_id}: {e:?}");
                            None
                        }
                        Ok(()) => {
                            debug!(%shard_id, "finalize success!");
                            Some(shard_id)
                        }
                    }
                }
            })
            // Poll each future for each collection concurrently, maximum of 10
            // at a time.
            // TODO(benesch): the concurrency here should be configurable
            // via LaunchDarkly.
            .buffer_unordered(10)
            // HERE BE DRAGONS: see warning on other uses of buffer_unordered.
            // The closure passed to `for_each` must remain fast or we risk
            // starving the finalization futures of calls to `poll`.
            .for_each(|shard_id| async move {
                match shard_id {
                    None => metrics.finalization_failed.inc(),
                    Some(shard_id) => {
                        // We make successfully finalized shards available for
                        // removal from the finalization WAL one by one, so that
                        // a handful of stuck shards don't prevent us from
                        // removing the shards that have made progress. The
                        // overhead of repeatedly acquiring and releasing the
                        // locks is negligible.
                        {
                            let mut finalizable_shards = finalizable_shards.lock();
                            let mut finalized_shards = finalized_shards.lock();
                            finalizable_shards.remove(&shard_id);
                            finalized_shards.insert(shard_id);
                        }

                        metrics.finalization_succeeded.inc();
                    }
                }
            })
            .await;

        debug!("done finalizing shards");
    }
}

#[derive(Debug)]
pub(crate) enum SnapshotStatsAsOf<T: TimelyTimestamp + Lattice + Codec64> {
    /// Stats for a shard with an "eager" upper (one that continually advances
    /// as time passes, even if no writes are coming in).
    Direct(Antichain<T>),
    /// Stats for a shard with a "lazy" upper (one that only physically advances
    /// in response to writes).
    Txns(DataSnapshot<T>),
}

#[cfg(test)]
mod tests {
    use std::sync::Arc;

    use mz_build_info::DUMMY_BUILD_INFO;
    use mz_dyncfg::ConfigSet;
    use mz_ore::assert_err;
    use mz_ore::metrics::{MetricsRegistry, UIntGauge};
    use mz_ore::now::SYSTEM_TIME;
    use mz_persist_client::cache::PersistClientCache;
    use mz_persist_client::cfg::PersistConfig;
    use mz_persist_client::rpc::PubSubClientConnection;
    use mz_persist_client::{Diagnostics, PersistClient, PersistLocation, ShardId};
    use mz_persist_types::codec_impls::UnitSchema;
    use mz_repr::{RelationDesc, Row};
    use mz_secrets::InMemorySecretsController;

    use super::*;

    #[mz_ore::test(tokio::test)]
    #[cfg_attr(miri, ignore)] // unsupported operation: integer-to-pointer casts and `ptr::from_exposed_addr`
    async fn test_snapshot_stats(&self) {
        let persist_location = PersistLocation {
            blob_uri: "mem://".to_owned(),
            consensus_uri: "mem://".to_owned(),
        };
        let persist_client = PersistClientCache::new(
            PersistConfig::new_default_configs(&DUMMY_BUILD_INFO, SYSTEM_TIME.clone()),
            &MetricsRegistry::new(),
            |_, _| PubSubClientConnection::noop(),
        );
        let persist_client = Arc::new(persist_client);

        let (cmds_tx, mut background_task) =
            BackgroundTask::new_for_test(persist_location.clone(), Arc::clone(&persist_client));
        let background_task =
            mz_ore::task::spawn(|| "storage_collections::background_task", async move {
                background_task.run().await
            });

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

        let shard_id = ShardId::new();
        let since_handle = persist
            .open_critical_since(
                shard_id,
                PersistClient::CONTROLLER_CRITICAL_SINCE,
                Diagnostics::for_tests(),
            )
            .await
            .unwrap();
        let write_handle = persist
            .open_writer::<SourceData, (), mz_repr::Timestamp, i64>(
                shard_id,
                Arc::new(RelationDesc::empty()),
                Arc::new(UnitSchema),
                Diagnostics::for_tests(),
            )
            .await
            .unwrap();

        cmds_tx
            .send(BackgroundCmd::Register {
                id: GlobalId::User(1),
                is_in_txns: false,
                since_handle: SinceHandleWrapper::Critical(since_handle),
                write_handle,
            })
            .unwrap();

        let mut write_handle = persist
            .open_writer::<SourceData, (), mz_repr::Timestamp, i64>(
                shard_id,
                Arc::new(RelationDesc::empty()),
                Arc::new(UnitSchema),
                Diagnostics::for_tests(),
            )
            .await
            .unwrap();

        // No stats for unknown GlobalId.
        let stats =
            snapshot_stats(&cmds_tx, GlobalId::User(2), Antichain::from_elem(0.into())).await;
        assert_err!(stats);

        // Stats don't resolve for as_of past the upper.
        let stats_fut = snapshot_stats(&cmds_tx, GlobalId::User(1), Antichain::from_elem(1.into()));
        assert_none!(stats_fut.now_or_never());

        // // Call it again because now_or_never consumed our future and it's not clone-able.
        let stats_ts1_fut =
            snapshot_stats(&cmds_tx, GlobalId::User(1), Antichain::from_elem(1.into()));

        // Write some data.
        let data = (
            (SourceData(Ok(Row::default())), ()),
            mz_repr::Timestamp::from(0),
            1i64,
        );
        let () = write_handle
            .compare_and_append(
                &[data],
                Antichain::from_elem(0.into()),
                Antichain::from_elem(1.into()),
            )
            .await
            .unwrap()
            .unwrap();

        // Verify that we can resolve stats for ts 0 while the ts 1 stats call is outstanding.
        let stats = snapshot_stats(&cmds_tx, GlobalId::User(1), Antichain::from_elem(0.into()))
            .await
            .unwrap();
        assert_eq!(stats.num_updates, 1);

        // Write more data and unblock the ts 1 call
        let data = (
            (SourceData(Ok(Row::default())), ()),
            mz_repr::Timestamp::from(1),
            1i64,
        );
        let () = write_handle
            .compare_and_append(
                &[data],
                Antichain::from_elem(1.into()),
                Antichain::from_elem(2.into()),
            )
            .await
            .unwrap()
            .unwrap();

        let stats = stats_ts1_fut.await.unwrap();
        assert_eq!(stats.num_updates, 2);

        // Make sure it runs until at least here.
        drop(background_task);
    }

    async fn snapshot_stats<T: TimelyTimestamp + Lattice + Codec64>(
        cmds_tx: &mpsc::UnboundedSender<BackgroundCmd<T>>,
        id: GlobalId,
        as_of: Antichain<T>,
    ) -> Result<SnapshotStats, StorageError<T>> {
        let (tx, rx) = oneshot::channel();
        cmds_tx
            .send(BackgroundCmd::SnapshotStats(
                id,
                SnapshotStatsAsOf::Direct(as_of),
                tx,
            ))
            .unwrap();
        let res = rx.await.expect("BackgroundTask should be live").0;

        res.await
    }

    impl<T: TimelyTimestamp + Lattice + Codec64> BackgroundTask<T> {
        fn new_for_test(
            _persist_location: PersistLocation,
            _persist_client: Arc<PersistClientCache>,
        ) -> (mpsc::UnboundedSender<BackgroundCmd<T>>, Self) {
            let (cmds_tx, cmds_rx) = mpsc::unbounded_channel();
            let (_holds_tx, holds_rx) = mpsc::unbounded_channel();
            let connection_context =
                ConnectionContext::for_tests(Arc::new(InMemorySecretsController::new()));

            let task = Self {
                config: Arc::new(Mutex::new(StorageConfiguration::new(
                    connection_context,
                    ConfigSet::default(),
                ))),
                cmds_tx: cmds_tx.clone(),
                cmds_rx,
                holds_rx,
                finalizable_shards: Arc::new(ShardIdSet::new(
                    UIntGauge::new("finalizable_shards", "dummy gauge for tests").unwrap(),
                )),
                collections: Arc::new(Mutex::new(BTreeMap::new())),
                shard_by_id: BTreeMap::new(),
                since_handles: BTreeMap::new(),
                txns_handle: None,
                txns_shards: BTreeSet::new(),
            };

            (cmds_tx, task)
        }
    }
}