1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258
3259
3260
3261
3262
3263
3264
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274
3275
3276
3277
3278
3279
3280
3281
3282
3283
3284
3285
3286
3287
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311
3312
3313
3314
3315
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363
3364
3365
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375
3376
3377
3378
3379
3380
3381
3382
3383
3384
3385
3386
3387
3388
3389
3390
3391
3392
3393
3394
3395
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405
3406
3407
3408
3409
3410
3411
3412
3413
3414
3415
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
3426
3427
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442
3443
3444
3445
3446
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458
3459
3460
3461
3462
3463
3464
3465
3466
3467
3468
3469
3470
3471
3472
3473
3474
3475
3476
3477
3478
3479
3480
3481
3482
3483
3484
3485
3486
3487
3488
3489
3490
3491
3492
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516
3517
3518
3519
3520
3521
3522
3523
3524
3525
3526
3527
3528
3529
3530
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548
3549
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599
3600
3601
3602
3603
3604
3605
3606
3607
3608
3609
3610
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636
3637
3638
3639
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651
3652
3653
3654
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691
3692
3693
3694
3695
3696
3697
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711
3712
3713
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745
3746
3747
3748
3749
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
// 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.

use anyhow::{ensure, Context};
use async_stream::{stream, try_stream};
use std::borrow::Cow;
use std::cmp::Ordering;
use std::collections::BTreeMap;
use std::fmt::{Debug, Formatter};
use std::marker::PhantomData;
use std::ops::ControlFlow::{self, Break, Continue};
use std::ops::{Deref, DerefMut};
use std::time::Duration;

use arrow::array::{make_array, Array, ArrayData};
use arrow::datatypes::DataType;
use bytes::Bytes;
use differential_dataflow::lattice::Lattice;
use differential_dataflow::trace::implementations::BatchContainer;
use differential_dataflow::trace::Description;
use differential_dataflow::Hashable;
use futures::Stream;
use futures_util::StreamExt;
use mz_dyncfg::Config;
use mz_ore::cast::CastFrom;
use mz_ore::now::EpochMillis;
use mz_ore::soft_panic_or_log;
use mz_ore::vec::PartialOrdVecExt;
use mz_persist::indexed::encoding::BatchColumnarFormat;
use mz_persist::location::{Blob, SeqNo};
use mz_persist_types::arrow::{ArrayBound, ProtoArrayData};
use mz_persist_types::columnar::{ColumnEncoder, Schema2};
use mz_persist_types::schema::{backward_compatible, SchemaId};
use mz_persist_types::{Codec, Codec64, Opaque};
use mz_proto::ProtoType;
use mz_proto::RustType;
use proptest_derive::Arbitrary;
use semver::Version;
use serde::ser::SerializeStruct;
use serde::{Serialize, Serializer};
use timely::order::TotalOrder;
use timely::progress::{Antichain, Timestamp};
use timely::{Container, PartialOrder};
use tracing::info;
use uuid::Uuid;

use crate::critical::CriticalReaderId;
use crate::error::InvalidUsage;
use crate::internal::encoding::{parse_id, LazyInlineBatchPart, LazyPartStats, LazyProto};
use crate::internal::gc::GcReq;
use crate::internal::machine::retry_external;
use crate::internal::metrics::SchemaMetrics;
use crate::internal::paths::{BlobKey, PartId, PartialBatchKey, PartialRollupKey, WriterKey};
use crate::internal::trace::{
    ActiveCompaction, ApplyMergeResult, FueledMergeReq, FueledMergeRes, Trace,
};
use crate::metrics::Metrics;
use crate::read::LeasedReaderId;
use crate::schema::CaESchema;
use crate::write::WriterId;
use crate::{PersistConfig, ShardId};

include!(concat!(
    env!("OUT_DIR"),
    "/mz_persist_client.internal.state.rs"
));

include!(concat!(
    env!("OUT_DIR"),
    "/mz_persist_client.internal.diff.rs"
));

/// Determines how often to write rollups, assigning a maintenance task after
/// `rollup_threshold` seqnos have passed since the last rollup.
///
/// Tuning note: in the absence of a long reader seqno hold, and with
/// incremental GC, this threshold will determine about how many live diffs are
/// held in Consensus. Lowering this value decreases the live diff count at the
/// cost of more maintenance work + blob writes.
pub(crate) const ROLLUP_THRESHOLD: Config<usize> = Config::new(
    "persist_rollup_threshold",
    128,
    "The number of seqnos between rollups.",
);

pub(crate) const WRITE_DIFFS_SUM: Config<bool> = Config::new(
    "persist_write_diffs_sum",
    true,
    "CYA to skip writing the diffs_sum field on HollowBatchPart",
);

/// A token to disambiguate state commands that could not otherwise be
/// idempotent.
#[derive(Arbitrary, Clone, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize)]
#[serde(into = "String")]
pub struct IdempotencyToken(pub(crate) [u8; 16]);

impl std::fmt::Display for IdempotencyToken {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        write!(f, "i{}", Uuid::from_bytes(self.0))
    }
}

impl std::fmt::Debug for IdempotencyToken {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        write!(f, "IdempotencyToken({})", Uuid::from_bytes(self.0))
    }
}

impl std::str::FromStr for IdempotencyToken {
    type Err = String;

    fn from_str(s: &str) -> Result<Self, Self::Err> {
        parse_id('i', "IdempotencyToken", s).map(IdempotencyToken)
    }
}

impl From<IdempotencyToken> for String {
    fn from(x: IdempotencyToken) -> Self {
        x.to_string()
    }
}

impl IdempotencyToken {
    pub(crate) fn new() -> Self {
        IdempotencyToken(*Uuid::new_v4().as_bytes())
    }
    pub(crate) const SENTINEL: IdempotencyToken = IdempotencyToken([17u8; 16]);
}

#[derive(Clone, Debug, PartialEq, Serialize)]
pub struct LeasedReaderState<T> {
    /// The seqno capability of this reader.
    pub seqno: SeqNo,
    /// The since capability of this reader.
    pub since: Antichain<T>,
    /// UNIX_EPOCH timestamp (in millis) of this reader's most recent heartbeat
    pub last_heartbeat_timestamp_ms: u64,
    /// Duration (in millis) allowed after [Self::last_heartbeat_timestamp_ms]
    /// after which this reader may be expired
    pub lease_duration_ms: u64,
    /// For debugging.
    pub debug: HandleDebugState,
}

#[derive(Arbitrary, Clone, Debug, PartialEq, Serialize)]
#[serde(into = "u64")]
pub struct OpaqueState(pub [u8; 8]);

impl From<OpaqueState> for u64 {
    fn from(value: OpaqueState) -> Self {
        u64::from_le_bytes(value.0)
    }
}

#[derive(Clone, Debug, PartialEq, Serialize)]
pub struct CriticalReaderState<T> {
    /// The since capability of this reader.
    pub since: Antichain<T>,
    /// An opaque token matched on by compare_and_downgrade_since.
    pub opaque: OpaqueState,
    /// The [Codec64] used to encode [Self::opaque].
    pub opaque_codec: String,
    /// For debugging.
    pub debug: HandleDebugState,
}

#[derive(Clone, Debug, PartialEq, Serialize)]
pub struct WriterState<T> {
    /// UNIX_EPOCH timestamp (in millis) of this writer's most recent heartbeat
    pub last_heartbeat_timestamp_ms: u64,
    /// Duration (in millis) allowed after [Self::last_heartbeat_timestamp_ms]
    /// after which this writer may be expired
    pub lease_duration_ms: u64,
    /// The idempotency token of the most recent successful compare_and_append
    /// by this writer.
    pub most_recent_write_token: IdempotencyToken,
    /// The upper of the most recent successful compare_and_append by this
    /// writer.
    pub most_recent_write_upper: Antichain<T>,
    /// For debugging.
    pub debug: HandleDebugState,
}

/// Debugging info for a reader or writer.
#[derive(Arbitrary, Clone, Debug, Default, PartialEq, Serialize)]
pub struct HandleDebugState {
    /// Hostname of the persist user that registered this writer or reader. For
    /// critical readers, this is the _most recent_ registration.
    pub hostname: String,
    /// Plaintext description of this writer or reader's intent.
    pub purpose: String,
}

/// Part of the updates in a Batch.
///
/// Either a pointer to ones stored in Blob or the updates themselves inlined.
#[derive(Clone, Debug, PartialEq, Eq, Serialize)]
#[serde(tag = "type")]
pub enum BatchPart<T> {
    Hollow(HollowBatchPart<T>),
    Inline {
        updates: LazyInlineBatchPart,
        ts_rewrite: Option<Antichain<T>>,
        schema_id: Option<SchemaId>,

        /// ID of a schema that has since been deprecated and exists only to cleanly roundtrip.
        deprecated_schema_id: Option<SchemaId>,
    },
}

fn decode_structured_lower(lower: &LazyProto<ProtoArrayData>) -> Option<ArrayBound> {
    let try_decode = |lower: &LazyProto<ProtoArrayData>| {
        let proto = lower.decode()?;
        let data = ArrayData::from_proto(proto)?;
        ensure!(data.len() == 1);
        Ok(ArrayBound::new(make_array(data), 0))
    };

    let decoded: anyhow::Result<ArrayBound> = try_decode(lower);

    match decoded {
        Ok(bound) => Some(bound),
        Err(e) => {
            soft_panic_or_log!("failed to decode bound: {e:#?}");
            None
        }
    }
}

impl<T> BatchPart<T> {
    pub fn hollow_bytes(&self) -> usize {
        match self {
            BatchPart::Hollow(x) => x.encoded_size_bytes,
            BatchPart::Inline { .. } => 0,
        }
    }

    pub fn is_inline(&self) -> bool {
        matches!(self, BatchPart::Inline { .. })
    }

    pub fn inline_bytes(&self) -> usize {
        match self {
            BatchPart::Hollow(_) => 0,
            BatchPart::Inline { updates, .. } => updates.encoded_size_bytes(),
        }
    }

    pub fn writer_key(&self) -> Option<WriterKey> {
        match self {
            BatchPart::Hollow(x) => x.key.split().map(|(writer, _part)| writer),
            BatchPart::Inline { .. } => None,
        }
    }

    pub fn encoded_size_bytes(&self) -> usize {
        match self {
            BatchPart::Hollow(x) => x.encoded_size_bytes,
            BatchPart::Inline { updates, .. } => updates.encoded_size_bytes(),
        }
    }

    // A user-interpretable identifier or description of the part (for logs and
    // such).
    pub fn printable_name(&self) -> &str {
        match self {
            BatchPart::Hollow(x) => x.key.0.as_str(),
            BatchPart::Inline { .. } => "<inline>",
        }
    }

    pub fn stats(&self) -> Option<&LazyPartStats> {
        match self {
            BatchPart::Hollow(x) => x.stats.as_ref(),
            BatchPart::Inline { .. } => None,
        }
    }

    pub fn key_lower(&self) -> &[u8] {
        match self {
            BatchPart::Hollow(x) => x.key_lower.as_slice(),
            // We don't duplicate the lowest key because this can be
            // considerable overhead for small parts.
            //
            // The empty key might not be a tight lower bound, but it is a valid
            // lower bound. If a caller is interested in a tighter lower bound,
            // the data is inline.
            BatchPart::Inline { .. } => &[],
        }
    }

    pub fn structured_key_lower(&self) -> Option<ArrayBound> {
        let part = match self {
            BatchPart::Hollow(part) => part,
            BatchPart::Inline { .. } => return None,
        };

        decode_structured_lower(part.structured_key_lower.as_ref()?)
    }

    pub fn ts_rewrite(&self) -> Option<&Antichain<T>> {
        match self {
            BatchPart::Hollow(x) => x.ts_rewrite.as_ref(),
            BatchPart::Inline { ts_rewrite, .. } => ts_rewrite.as_ref(),
        }
    }

    pub fn schema_id(&self) -> Option<SchemaId> {
        match self {
            BatchPart::Hollow(x) => x.schema_id,
            BatchPart::Inline { schema_id, .. } => *schema_id,
        }
    }
}

/// An ordered list of parts, generally stored as part of a larger run.
#[derive(Debug, Clone)]
pub struct HollowRun<T> {
    /// Pointers usable to retrieve the updates.
    pub(crate) parts: Vec<RunPart<T>>,
}

/// A reference to a [HollowRun], including the key in the blob store and some denormalized
/// metadata.
#[derive(Debug, Eq, PartialEq, Clone, Serialize)]
pub struct HollowRunRef<T> {
    pub key: PartialBatchKey,

    /// The size of the referenced run object, plus all of the hollow objects it contains.
    pub hollow_bytes: usize,

    /// The size of the largest individual part in the run; useful for sizing compaction.
    pub max_part_bytes: usize,

    /// The lower bound of the data in this part, ordered by the codec ordering.
    pub key_lower: Vec<u8>,

    /// The lower bound of the data in this part, ordered by the structured ordering.
    pub structured_key_lower: Option<LazyProto<ProtoArrayData>>,

    pub(crate) _phantom_data: PhantomData<T>,
}
impl<T: Eq> PartialOrd<Self> for HollowRunRef<T> {
    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
        Some(self.cmp(other))
    }
}

impl<T: Eq> Ord for HollowRunRef<T> {
    fn cmp(&self, other: &Self) -> Ordering {
        self.key.cmp(&other.key)
    }
}

impl<T> HollowRunRef<T> {
    pub fn writer_key(&self) -> Option<WriterKey> {
        Some(self.key.split()?.0)
    }
}

impl<T: Timestamp + Codec64> HollowRunRef<T> {
    /// Stores the given runs and returns a [HollowRunRef] that points to them.
    pub async fn set(
        shard_id: ShardId,
        blob: &dyn Blob,
        writer: &WriterKey,
        data: HollowRun<T>,
        metrics: &Metrics,
    ) -> Self {
        let hollow_bytes = data.parts.iter().map(|p| p.hollow_bytes()).sum();
        let max_part_bytes = data
            .parts
            .iter()
            .map(|p| p.max_part_bytes())
            .max()
            .unwrap_or(0);
        let key_lower = data
            .parts
            .first()
            .map_or(vec![], |p| p.key_lower().to_vec());
        let structured_key_lower = match data.parts.first() {
            Some(RunPart::Many(r)) => r.structured_key_lower.clone(),
            Some(RunPart::Single(BatchPart::Hollow(p))) => p.structured_key_lower.clone(),
            Some(RunPart::Single(BatchPart::Inline { .. })) | None => None,
        };

        let key = PartialBatchKey::new(writer, &PartId::new());
        let blob_key = key.complete(&shard_id);
        let bytes = Bytes::from(prost::Message::encode_to_vec(&data.into_proto()));
        let () = retry_external(&metrics.retries.external.hollow_run_set, || {
            blob.set(&blob_key, bytes.clone())
        })
        .await;
        Self {
            key,
            hollow_bytes,
            max_part_bytes,
            key_lower,
            structured_key_lower,
            _phantom_data: Default::default(),
        }
    }

    /// Retrieve the [HollowRun] that this reference points to.
    /// The caller is expected to ensure that this ref is the result of calling [HollowRunRef::set]
    /// with the same shard id and backing store.
    pub async fn get(
        &self,
        shard_id: ShardId,
        blob: &dyn Blob,
        metrics: &Metrics,
    ) -> Option<HollowRun<T>> {
        let blob_key = self.key.complete(&shard_id);
        let mut bytes = retry_external(&metrics.retries.external.hollow_run_get, || {
            blob.get(&blob_key)
        })
        .await?;
        let proto_runs: ProtoHollowRun =
            prost::Message::decode(&mut bytes).expect("illegal state: invalid proto bytes");
        let runs = proto_runs
            .into_rust()
            .expect("illegal state: invalid encoded runs proto");
        Some(runs)
    }
}

/// Part of the updates in a run.
///
/// Either a pointer to ones stored in Blob or a single part stored inline.
#[derive(Clone, Debug, PartialEq, Eq, Serialize)]
#[serde(untagged)]
pub enum RunPart<T> {
    Single(BatchPart<T>),
    Many(HollowRunRef<T>),
}

impl<T: Ord> PartialOrd<Self> for RunPart<T> {
    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
        Some(self.cmp(other))
    }
}

impl<T: Ord> Ord for RunPart<T> {
    fn cmp(&self, other: &Self) -> Ordering {
        match (self, other) {
            (RunPart::Single(a), RunPart::Single(b)) => a.cmp(b),
            (RunPart::Single(_), RunPart::Many(_)) => Ordering::Less,
            (RunPart::Many(_), RunPart::Single(_)) => Ordering::Greater,
            (RunPart::Many(a), RunPart::Many(b)) => a.cmp(b),
        }
    }
}

impl<T> RunPart<T> {
    #[cfg(test)]
    pub fn expect_hollow_part(&self) -> &HollowBatchPart<T> {
        match self {
            RunPart::Single(BatchPart::Hollow(hollow)) => hollow,
            _ => panic!("expected hollow part!"),
        }
    }

    pub fn hollow_bytes(&self) -> usize {
        match self {
            Self::Single(p) => p.hollow_bytes(),
            Self::Many(r) => r.hollow_bytes,
        }
    }

    pub fn is_inline(&self) -> bool {
        match self {
            Self::Single(p) => p.is_inline(),
            Self::Many(_) => false,
        }
    }

    pub fn inline_bytes(&self) -> usize {
        match self {
            Self::Single(p) => p.inline_bytes(),
            Self::Many(_) => 0,
        }
    }

    pub fn max_part_bytes(&self) -> usize {
        match self {
            Self::Single(p) => p.encoded_size_bytes(),
            Self::Many(r) => r.max_part_bytes,
        }
    }

    pub fn writer_key(&self) -> Option<WriterKey> {
        match self {
            Self::Single(p) => p.writer_key(),
            Self::Many(r) => r.writer_key(),
        }
    }

    pub fn encoded_size_bytes(&self) -> usize {
        match self {
            Self::Single(p) => p.encoded_size_bytes(),
            Self::Many(r) => r.hollow_bytes,
        }
    }

    pub fn schema_id(&self) -> Option<SchemaId> {
        match self {
            Self::Single(p) => p.schema_id(),
            Self::Many(_) => None,
        }
    }

    // A user-interpretable identifier or description of the part (for logs and
    // such).
    pub fn printable_name(&self) -> &str {
        match self {
            Self::Single(p) => p.printable_name(),
            Self::Many(r) => r.key.0.as_str(),
        }
    }

    pub fn stats(&self) -> Option<&LazyPartStats> {
        match self {
            Self::Single(p) => p.stats(),
            // TODO: if we kept stats we could avoid fetching the metadata here.
            Self::Many(_) => None,
        }
    }

    pub fn key_lower(&self) -> &[u8] {
        match self {
            Self::Single(p) => p.key_lower(),
            Self::Many(r) => r.key_lower.as_slice(),
        }
    }

    pub fn structured_key_lower(&self) -> Option<ArrayBound> {
        match self {
            Self::Single(p) => p.structured_key_lower(),
            Self::Many(_) => None,
        }
    }

    pub fn ts_rewrite(&self) -> Option<&Antichain<T>> {
        match self {
            Self::Single(p) => p.ts_rewrite(),
            Self::Many(_) => None,
        }
    }
}

/// A blob was missing!
#[derive(Clone, Debug)]
pub struct MissingBlob(BlobKey);

impl std::fmt::Display for MissingBlob {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        write!(f, "unexpectedly missing key: {}", self.0)
    }
}

impl std::error::Error for MissingBlob {}

impl<T: Timestamp + Codec64 + Sync> RunPart<T> {
    pub fn part_stream<'a>(
        &'a self,
        shard_id: ShardId,
        blob: &'a dyn Blob,
        metrics: &'a Metrics,
    ) -> impl Stream<Item = Result<Cow<'a, BatchPart<T>>, MissingBlob>> + Send + 'a {
        try_stream! {
            match self {
                RunPart::Single(p) => {
                    yield Cow::Borrowed(p);
                }
                RunPart::Many(r) => {
                    let fetched = r.get(shard_id, blob, metrics).await.ok_or_else(|| MissingBlob(r.key.complete(&shard_id)))?;
                    for run_part in fetched.parts {
                        for await batch_part in run_part.part_stream(shard_id, blob, metrics).boxed() {
                            yield Cow::Owned(batch_part?.into_owned());
                        }
                    }
                }
            }
        }
    }
}

impl<T: Ord> PartialOrd for BatchPart<T> {
    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
        Some(self.cmp(other))
    }
}

impl<T: Ord> Ord for BatchPart<T> {
    fn cmp(&self, other: &Self) -> Ordering {
        match (self, other) {
            (BatchPart::Hollow(s), BatchPart::Hollow(o)) => s.cmp(o),
            (
                BatchPart::Inline {
                    updates: s_updates,
                    ts_rewrite: s_ts_rewrite,
                    schema_id: s_schema_id,
                    deprecated_schema_id: s_deprecated_schema_id,
                },
                BatchPart::Inline {
                    updates: o_updates,
                    ts_rewrite: o_ts_rewrite,
                    schema_id: o_schema_id,
                    deprecated_schema_id: o_deprecated_schema_id,
                },
            ) => (
                s_updates,
                s_ts_rewrite.as_ref().map(|x| x.elements()),
                s_schema_id,
                s_deprecated_schema_id,
            )
                .cmp(&(
                    o_updates,
                    o_ts_rewrite.as_ref().map(|x| x.elements()),
                    o_schema_id,
                    o_deprecated_schema_id,
                )),
            (BatchPart::Hollow(_), BatchPart::Inline { .. }) => Ordering::Less,
            (BatchPart::Inline { .. }, BatchPart::Hollow(_)) => Ordering::Greater,
        }
    }
}

/// What order are the parts in this run in?
#[derive(Clone, Copy, Debug, PartialEq, Eq, Ord, PartialOrd, Serialize)]
pub(crate) enum RunOrder {
    /// They're in no particular order.
    Unordered,
    /// They're ordered based on the codec-encoded K/V bytes.
    Codec,
    /// They're ordered by the natural ordering of the structured data.
    Structured,
}

/// Metadata shared across a run.
#[derive(Clone, Debug, Default, PartialEq, Eq, Ord, PartialOrd, Serialize)]
pub struct RunMeta {
    /// If none, Persist should infer the order based on the proto metadata.
    pub(crate) order: Option<RunOrder>,
    /// All parts in a run should have the same schema.
    pub(crate) schema: Option<SchemaId>,

    /// ID of a schema that has since been deprecated and exists only to cleanly roundtrip.
    pub(crate) deprecated_schema: Option<SchemaId>,
}

/// A subset of a [HollowBatch] corresponding 1:1 to a blob.
#[derive(Clone, Debug, PartialEq, Eq, Serialize)]
pub struct HollowBatchPart<T> {
    /// Pointer usable to retrieve the updates.
    pub key: PartialBatchKey,
    /// The encoded size of this part.
    pub encoded_size_bytes: usize,
    /// A lower bound on the keys in the part. (By default, this the minimum
    /// possible key: `vec![]`.)
    #[serde(serialize_with = "serialize_part_bytes")]
    pub key_lower: Vec<u8>,
    /// A lower bound on the keys in the part, stored as structured data.
    #[serde(serialize_with = "serialize_lazy_proto")]
    pub structured_key_lower: Option<LazyProto<ProtoArrayData>>,
    /// Aggregate statistics about data contained in this part.
    #[serde(serialize_with = "serialize_part_stats")]
    pub stats: Option<LazyPartStats>,
    /// A frontier to which timestamps in this part are advanced on read, if
    /// set.
    ///
    /// A value of `Some([T::minimum()])` is functionally the same as `None`,
    /// but we maintain the distinction between the two for some internal sanity
    /// checking of invariants as well as metrics. If this ever becomes an
    /// issue, everything still works with this as just `Antichain<T>`.
    pub ts_rewrite: Option<Antichain<T>>,
    /// A Codec64 encoded sum of all diffs in this part, if known.
    ///
    /// This is `None` if this part was written before we started storing this
    /// information, or if it was written when the dyncfg was off.
    ///
    /// It could also make sense to model this as part of the pushdown stats, if
    /// we later decide that's of some benefit.
    #[serde(serialize_with = "serialize_diffs_sum")]
    pub diffs_sum: Option<[u8; 8]>,
    /// Columnar format that this batch was written in.
    ///
    /// This is `None` if this part was written before we started writing structured
    /// columnar data.
    pub format: Option<BatchColumnarFormat>,
    /// The schemas used to encode the data in this batch part.
    ///
    /// Or None for historical data written before the schema registry was
    /// added.
    pub schema_id: Option<SchemaId>,

    /// ID of a schema that has since been deprecated and exists only to cleanly roundtrip.
    pub deprecated_schema_id: Option<SchemaId>,
}

/// A [Batch] but with the updates themselves stored externally.
///
/// [Batch]: differential_dataflow::trace::BatchReader
#[derive(Clone, PartialEq, Eq)]
pub struct HollowBatch<T> {
    /// Describes the times of the updates in the batch.
    pub desc: Description<T>,
    /// The number of updates in the batch.
    pub len: usize,
    /// Pointers usable to retrieve the updates.
    pub(crate) parts: Vec<RunPart<T>>,
    /// Runs of sequential sorted batch parts, stored as indices into `parts`.
    /// ex.
    /// ```text
    ///     parts=[p1, p2, p3], runs=[]     --> run  is  [p1, p2, p2]
    ///     parts=[p1, p2, p3], runs=[1]    --> runs are [p1] and [p2, p3]
    ///     parts=[p1, p2, p3], runs=[1, 2] --> runs are [p1], [p2], [p3]
    /// ```
    pub(crate) run_splits: Vec<usize>,
    /// Run-level metadata: the first entry has metadata for the first run, and so on.
    /// If there's no corresponding entry for a particular run, it's assumed to be [RunMeta::default()].
    pub(crate) run_meta: Vec<RunMeta>,
}

impl<T: Debug> Debug for HollowBatch<T> {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        let HollowBatch {
            desc,
            parts,
            len,
            run_splits: runs,
            run_meta,
        } = self;
        f.debug_struct("HollowBatch")
            .field(
                "desc",
                &(
                    desc.lower().elements(),
                    desc.upper().elements(),
                    desc.since().elements(),
                ),
            )
            .field("parts", &parts)
            .field("len", &len)
            .field("runs", &runs)
            .field("run_meta", &run_meta)
            .finish()
    }
}

impl<T: Serialize> serde::Serialize for HollowBatch<T> {
    fn serialize<S: Serializer>(&self, s: S) -> Result<S::Ok, S::Error> {
        let HollowBatch {
            desc,
            len,
            // Both parts and runs are covered by the self.runs call.
            parts: _,
            run_splits: _,
            run_meta: _,
        } = self;
        let mut s = s.serialize_struct("HollowBatch", 5)?;
        let () = s.serialize_field("lower", &desc.lower().elements())?;
        let () = s.serialize_field("upper", &desc.upper().elements())?;
        let () = s.serialize_field("since", &desc.since().elements())?;
        let () = s.serialize_field("len", len)?;
        let () = s.serialize_field("part_runs", &self.runs().collect::<Vec<_>>())?;
        s.end()
    }
}

impl<T: Ord> PartialOrd for HollowBatch<T> {
    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
        Some(self.cmp(other))
    }
}

impl<T: Ord> Ord for HollowBatch<T> {
    fn cmp(&self, other: &Self) -> Ordering {
        // Deconstruct self and other so we get a compile failure if new fields
        // are added.
        let HollowBatch {
            desc: self_desc,
            parts: self_parts,
            len: self_len,
            run_splits: self_runs,
            run_meta: self_run_meta,
        } = self;
        let HollowBatch {
            desc: other_desc,
            parts: other_parts,
            len: other_len,
            run_splits: other_runs,
            run_meta: other_run_meta,
        } = other;
        (
            self_desc.lower().elements(),
            self_desc.upper().elements(),
            self_desc.since().elements(),
            self_parts,
            self_len,
            self_runs,
            self_run_meta,
        )
            .cmp(&(
                other_desc.lower().elements(),
                other_desc.upper().elements(),
                other_desc.since().elements(),
                other_parts,
                other_len,
                other_runs,
                other_run_meta,
            ))
    }
}

impl<T: Timestamp + Codec64 + Sync> HollowBatch<T> {
    pub fn part_stream<'a>(
        &'a self,
        shard_id: ShardId,
        blob: &'a dyn Blob,
        metrics: &'a Metrics,
    ) -> impl Stream<Item = Result<Cow<'a, BatchPart<T>>, MissingBlob>> + 'a {
        stream! {
            for part in &self.parts {
                for await part in part.part_stream(shard_id, blob, metrics) {
                    yield part;
                }
            }
        }
    }
}
impl<T> HollowBatch<T> {
    /// Construct an in-memory hollow batch from the given metadata.
    ///
    /// This method checks that `runs` is a sequence of valid indices into `parts`. The caller
    /// is responsible for ensuring that the defined runs are valid.
    ///
    /// `len` should represent the number of valid updates in the referenced parts.
    pub(crate) fn new(
        desc: Description<T>,
        parts: Vec<RunPart<T>>,
        len: usize,
        run_meta: Vec<RunMeta>,
        run_splits: Vec<usize>,
    ) -> Self {
        debug_assert!(
            run_splits.is_strictly_sorted(),
            "run indices should be strictly increasing"
        );
        debug_assert!(
            run_splits.first().map_or(true, |i| *i > 0),
            "run indices should be positive"
        );
        debug_assert!(
            run_splits.last().map_or(true, |i| *i < parts.len()),
            "run indices should be valid indices into parts"
        );
        debug_assert!(
            parts.is_empty() || run_meta.len() == run_splits.len() + 1,
            "all metadata should correspond to a run"
        );

        Self {
            desc,
            len,
            parts,
            run_splits,
            run_meta,
        }
    }

    /// Construct a batch of a single run with default metadata. Mostly interesting for tests.
    pub(crate) fn new_run(desc: Description<T>, parts: Vec<RunPart<T>>, len: usize) -> Self {
        let run_meta = if parts.is_empty() {
            vec![]
        } else {
            vec![RunMeta::default()]
        };
        Self {
            desc,
            len,
            parts,
            run_splits: vec![],
            run_meta,
        }
    }

    /// An empty hollow batch, representing no updates over the given desc.
    pub(crate) fn empty(desc: Description<T>) -> Self {
        Self {
            desc,
            len: 0,
            parts: vec![],
            run_splits: vec![],
            run_meta: vec![],
        }
    }

    pub(crate) fn runs(&self) -> impl Iterator<Item = (&RunMeta, &[RunPart<T>])> {
        let run_ends = self
            .run_splits
            .iter()
            .copied()
            .chain(std::iter::once(self.parts.len()));
        let run_metas = self.run_meta.iter();
        let run_parts = run_ends
            .scan(0, |start, end| {
                let range = *start..end;
                *start = end;
                Some(range)
            })
            .filter(|range| !range.is_empty())
            .map(|range| &self.parts[range]);
        run_metas.zip(run_parts)
    }

    pub(crate) fn inline_bytes(&self) -> usize {
        self.parts.iter().map(|x| x.inline_bytes()).sum()
    }

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

    pub fn part_count(&self) -> usize {
        self.parts.len()
    }

    /// The sum of the encoded sizes of all parts in the batch.
    pub fn encoded_size_bytes(&self) -> usize {
        self.parts.iter().map(|p| p.encoded_size_bytes()).sum()
    }
}

// See the comment on [Batch::rewrite_ts] for why this is TotalOrder.
impl<T: Timestamp + TotalOrder> HollowBatch<T> {
    pub(crate) fn rewrite_ts(
        &mut self,
        frontier: &Antichain<T>,
        new_upper: Antichain<T>,
    ) -> Result<(), String> {
        if !PartialOrder::less_than(frontier, &new_upper) {
            return Err(format!(
                "rewrite frontier {:?} !< rewrite upper {:?}",
                frontier.elements(),
                new_upper.elements(),
            ));
        }
        if PartialOrder::less_than(&new_upper, self.desc.upper()) {
            return Err(format!(
                "rewrite upper {:?} < batch upper {:?}",
                new_upper.elements(),
                self.desc.upper().elements(),
            ));
        }

        // The following are things that it seems like we could support, but
        // initially we don't because we don't have a use case for them.
        if PartialOrder::less_than(frontier, self.desc.lower()) {
            return Err(format!(
                "rewrite frontier {:?} < batch lower {:?}",
                frontier.elements(),
                self.desc.lower().elements(),
            ));
        }
        if self.desc.since() != &Antichain::from_elem(T::minimum()) {
            return Err(format!(
                "batch since {:?} != minimum antichain {:?}",
                self.desc.since().elements(),
                &[T::minimum()],
            ));
        }
        for part in self.parts.iter() {
            let Some(ts_rewrite) = part.ts_rewrite() else {
                continue;
            };
            if PartialOrder::less_than(frontier, ts_rewrite) {
                return Err(format!(
                    "rewrite frontier {:?} < batch rewrite {:?}",
                    frontier.elements(),
                    ts_rewrite.elements(),
                ));
            }
        }

        self.desc = Description::new(
            self.desc.lower().clone(),
            new_upper,
            self.desc.since().clone(),
        );
        for part in &mut self.parts {
            match part {
                RunPart::Single(BatchPart::Hollow(part)) => {
                    part.ts_rewrite = Some(frontier.clone())
                }
                RunPart::Single(BatchPart::Inline { ts_rewrite, .. }) => {
                    *ts_rewrite = Some(frontier.clone())
                }
                RunPart::Many(runs) => {
                    // Currently unreachable: we only apply rewrites to user batches, and we don't
                    // ever generate runs of >1 part for those.
                    panic!("unexpected rewrite of a hollow runs ref: {runs:?}");
                }
            }
        }
        Ok(())
    }
}

impl<T: Ord> PartialOrd for HollowBatchPart<T> {
    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
        Some(self.cmp(other))
    }
}

impl<T: Ord> Ord for HollowBatchPart<T> {
    fn cmp(&self, other: &Self) -> Ordering {
        // Deconstruct self and other so we get a compile failure if new fields
        // are added.
        let HollowBatchPart {
            key: self_key,
            encoded_size_bytes: self_encoded_size_bytes,
            key_lower: self_key_lower,
            structured_key_lower: self_structured_key_lower,
            stats: self_stats,
            ts_rewrite: self_ts_rewrite,
            diffs_sum: self_diffs_sum,
            format: self_format,
            schema_id: self_schema_id,
            deprecated_schema_id: self_deprecated_schema_id,
        } = self;
        let HollowBatchPart {
            key: other_key,
            encoded_size_bytes: other_encoded_size_bytes,
            key_lower: other_key_lower,
            structured_key_lower: other_structured_key_lower,
            stats: other_stats,
            ts_rewrite: other_ts_rewrite,
            diffs_sum: other_diffs_sum,
            format: other_format,
            schema_id: other_schema_id,
            deprecated_schema_id: other_deprecated_schema_id,
        } = other;
        (
            self_key,
            self_encoded_size_bytes,
            self_key_lower,
            self_structured_key_lower,
            self_stats,
            self_ts_rewrite.as_ref().map(|x| x.elements()),
            self_diffs_sum,
            self_format,
            self_schema_id,
            self_deprecated_schema_id,
        )
            .cmp(&(
                other_key,
                other_encoded_size_bytes,
                other_key_lower,
                other_structured_key_lower,
                other_stats,
                other_ts_rewrite.as_ref().map(|x| x.elements()),
                other_diffs_sum,
                other_format,
                other_schema_id,
                other_deprecated_schema_id,
            ))
    }
}

/// A pointer to a rollup stored externally.
#[derive(Arbitrary, Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Serialize)]
pub struct HollowRollup {
    /// Pointer usable to retrieve the rollup.
    pub key: PartialRollupKey,
    /// The encoded size of this rollup, if known.
    pub encoded_size_bytes: Option<usize>,
}

/// A pointer to a blob stored externally.
#[derive(Debug)]
pub enum HollowBlobRef<'a, T> {
    Batch(&'a HollowBatch<T>),
    Rollup(&'a HollowRollup),
}

/// A sentinel for a state transition that was a no-op.
///
/// Critically, this also indicates that the no-op state transition was not
/// committed through compare_and_append and thus is _not linearized_.
#[derive(Debug)]
#[cfg_attr(any(test, debug_assertions), derive(PartialEq))]
pub struct NoOpStateTransition<T>(pub T);

// TODO: Document invariants.
#[derive(Debug, Clone)]
#[cfg_attr(any(test, debug_assertions), derive(PartialEq))]
pub struct StateCollections<T> {
    // - Invariant: `<= all reader.since`
    // - Invariant: Doesn't regress across state versions.
    pub(crate) last_gc_req: SeqNo,

    // - Invariant: There is a rollup with `seqno <= self.seqno_since`.
    pub(crate) rollups: BTreeMap<SeqNo, HollowRollup>,

    pub(crate) leased_readers: BTreeMap<LeasedReaderId, LeasedReaderState<T>>,
    pub(crate) critical_readers: BTreeMap<CriticalReaderId, CriticalReaderState<T>>,
    pub(crate) writers: BTreeMap<WriterId, WriterState<T>>,
    pub(crate) schemas: BTreeMap<SchemaId, EncodedSchemas>,

    // - Invariant: `trace.since == meet(all reader.since)`
    // - Invariant: `trace.since` doesn't regress across state versions.
    // - Invariant: `trace.upper` doesn't regress across state versions.
    // - Invariant: `trace` upholds its own invariants.
    pub(crate) trace: Trace<T>,
}

/// A key and val [Codec::Schema] encoded via [Codec::encode_schema].
///
/// This strategy of directly serializing the schema objects requires that
/// persist users do the right thing. Specifically, that an encoded schema
/// doesn't in some later version of mz decode to an in-mem object that acts
/// differently. In a sense, the current system (before the introduction of the
/// schema registry) where schemas are passed in unchecked to reader and writer
/// registration calls also has the same defect, so seems fine.
///
/// An alternative is to write down here some persist-specific representation of
/// the schema (e.g. the arrow DataType). This is a lot more work and also has
/// the potential to lead down a similar failure mode to the mz_persist_types
/// `Data` trait, where the boilerplate isn't worth the safety. Given that we
/// can always migrate later by rehydrating these, seems fine to start with the
/// easy thing.
#[derive(Debug, Clone, Serialize, PartialEq)]
pub struct EncodedSchemas {
    /// A full in-mem `K::Schema` impl encoded via [Codec::encode_schema].
    pub key: Bytes,
    /// The arrow `DataType` produced by this `K::Schema` at the time it was
    /// registered, encoded as a `ProtoDataType`.
    pub key_data_type: Bytes,
    /// A full in-mem `V::Schema` impl encoded via [Codec::encode_schema].
    pub val: Bytes,
    /// The arrow `DataType` produced by this `V::Schema` at the time it was
    /// registered, encoded as a `ProtoDataType`.
    pub val_data_type: Bytes,
}

impl EncodedSchemas {
    pub(crate) fn decode_data_type(buf: &[u8]) -> DataType {
        let proto = prost::Message::decode(buf).expect("valid ProtoDataType");
        DataType::from_proto(proto).expect("valid DataType")
    }
}

#[derive(Debug)]
#[cfg_attr(test, derive(PartialEq))]
pub enum CompareAndAppendBreak<T> {
    AlreadyCommitted,
    Upper {
        shard_upper: Antichain<T>,
        writer_upper: Antichain<T>,
    },
    InvalidUsage(InvalidUsage<T>),
    InlineBackpressure,
}

#[derive(Debug)]
#[cfg_attr(test, derive(PartialEq))]
pub enum SnapshotErr<T> {
    AsOfNotYetAvailable(SeqNo, Upper<T>),
    AsOfHistoricalDistinctionsLost(Since<T>),
}

impl<T> StateCollections<T>
where
    T: Timestamp + Lattice + Codec64,
{
    pub fn add_rollup(
        &mut self,
        add_rollup: (SeqNo, &HollowRollup),
    ) -> ControlFlow<NoOpStateTransition<bool>, bool> {
        let (rollup_seqno, rollup) = add_rollup;
        let applied = match self.rollups.get(&rollup_seqno) {
            Some(x) => x.key == rollup.key,
            None => {
                self.rollups.insert(rollup_seqno, rollup.to_owned());
                true
            }
        };
        // This state transition is a no-op if applied is false but we
        // still commit the state change so that this gets linearized
        // (maybe we're looking at old state).
        Continue(applied)
    }

    pub fn remove_rollups(
        &mut self,
        remove_rollups: &[(SeqNo, PartialRollupKey)],
    ) -> ControlFlow<NoOpStateTransition<Vec<SeqNo>>, Vec<SeqNo>> {
        if remove_rollups.is_empty() || self.is_tombstone() {
            return Break(NoOpStateTransition(vec![]));
        }

        let mut removed = vec![];
        for (seqno, key) in remove_rollups {
            let removed_key = self.rollups.remove(seqno);
            debug_assert!(
                removed_key.as_ref().map_or(true, |x| &x.key == key),
                "{} vs {:?}",
                key,
                removed_key
            );

            if removed_key.is_some() {
                removed.push(*seqno);
            }
        }

        Continue(removed)
    }

    pub fn register_leased_reader(
        &mut self,
        hostname: &str,
        reader_id: &LeasedReaderId,
        purpose: &str,
        seqno: SeqNo,
        lease_duration: Duration,
        heartbeat_timestamp_ms: u64,
        use_critical_since: bool,
    ) -> ControlFlow<
        NoOpStateTransition<(LeasedReaderState<T>, SeqNo)>,
        (LeasedReaderState<T>, SeqNo),
    > {
        let since = if use_critical_since {
            self.critical_since().unwrap_or(self.trace.since().clone())
        } else {
            self.trace.since().clone()
        };
        let reader_state = LeasedReaderState {
            debug: HandleDebugState {
                hostname: hostname.to_owned(),
                purpose: purpose.to_owned(),
            },
            seqno,
            since,
            last_heartbeat_timestamp_ms: heartbeat_timestamp_ms,
            lease_duration_ms: u64::try_from(lease_duration.as_millis())
                .expect("lease duration as millis should fit within u64"),
        };

        // If the shard-global upper and since are both the empty antichain,
        // then no further writes can ever commit and no further reads can be
        // served. Optimize this by no-op-ing reader registration so that we can
        // settle the shard into a final unchanging tombstone state.
        if self.is_tombstone() {
            return Break(NoOpStateTransition((reader_state, self.seqno_since(seqno))));
        }

        // TODO: Handle if the reader or writer already exists.
        self.leased_readers
            .insert(reader_id.clone(), reader_state.clone());
        Continue((reader_state, self.seqno_since(seqno)))
    }

    pub fn register_critical_reader<O: Opaque + Codec64>(
        &mut self,
        hostname: &str,
        reader_id: &CriticalReaderId,
        purpose: &str,
    ) -> ControlFlow<NoOpStateTransition<CriticalReaderState<T>>, CriticalReaderState<T>> {
        let state = CriticalReaderState {
            debug: HandleDebugState {
                hostname: hostname.to_owned(),
                purpose: purpose.to_owned(),
            },
            since: self.trace.since().clone(),
            opaque: OpaqueState(Codec64::encode(&O::initial())),
            opaque_codec: O::codec_name(),
        };

        // We expire all readers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            return Break(NoOpStateTransition(state));
        }

        let state = match self.critical_readers.get_mut(reader_id) {
            Some(existing_state) => {
                existing_state.debug = state.debug;
                existing_state.clone()
            }
            None => {
                self.critical_readers
                    .insert(reader_id.clone(), state.clone());
                state
            }
        };
        Continue(state)
    }

    pub fn register_schema<K: Codec, V: Codec>(
        &mut self,
        key_schema: &K::Schema,
        val_schema: &V::Schema,
        metrics: &SchemaMetrics,
    ) -> ControlFlow<NoOpStateTransition<Option<SchemaId>>, Option<SchemaId>> {
        fn encode_data_type(data_type: &DataType) -> Bytes {
            let proto = data_type.into_proto();
            prost::Message::encode_to_vec(&proto).into()
        }

        fn decode_data_type(buf: Bytes) -> Result<DataType, anyhow::Error> {
            let proto: mz_persist_types::arrow::ProtoDataType =
                prost::Message::decode(buf).context("decoding schema DataType")?;
            DataType::from_proto(proto).context("converting ProtoDataType into DataType")
        }

        // Look for an existing registered SchemaId for these schemas.
        //
        // The common case is that this should be a recent one, so as a minor
        // optimization, do this search in reverse order.
        //
        // TODO: Note that this impl is `O(schemas)`. Combined with the
        // possibility of cmd retries, it's possible but unlikely for this to
        // get expensive. We could maintain a reverse map to speed this up in
        // necessary. This would either need to work on the encoded
        // representation (which, we'd have to fall back to the linear scan) or
        // we'd need to add a Hash/Ord bound to Schema.
        let existing_id = self.schemas.iter().rev().find(|(_, x)| {
            K::decode_schema(&x.key) == *key_schema && V::decode_schema(&x.val) == *val_schema
        });
        match existing_id {
            Some((schema_id, encoded_schemas)) => {
                let schema_id = *schema_id;
                let new_k_datatype = mz_persist_types::columnar::data_type::<K>(key_schema)
                    .expect("valid key schema");
                let new_v_datatype = mz_persist_types::columnar::data_type::<V>(val_schema)
                    .expect("valid val schema");

                let new_k_encoded_datatype = encode_data_type(&new_k_datatype);
                let new_v_encoded_datatype = encode_data_type(&new_v_datatype);

                // Check if the generated Arrow DataTypes have changed.
                if encoded_schemas.key_data_type != new_k_encoded_datatype
                    || encoded_schemas.val_data_type != new_v_encoded_datatype
                {
                    let old_k_datatype =
                        decode_data_type(Bytes::clone(&encoded_schemas.key_data_type))
                            .expect("failed to roundtrip Arrow DataType");
                    let old_v_datatype =
                        decode_data_type(Bytes::clone(&encoded_schemas.val_data_type))
                            .expect("failed to roundtrip Arrow DataType");

                    let k_atleast_as_nullable =
                        crate::schema::is_atleast_as_nullable(&old_k_datatype, &new_k_datatype);
                    let v_atleast_as_nullable =
                        crate::schema::is_atleast_as_nullable(&old_v_datatype, &new_v_datatype);

                    // If the Arrow DataType for `k` or `v` has changed, but it's only become more
                    // nullable, then we allow in-place re-writing of the schema.
                    match (k_atleast_as_nullable, v_atleast_as_nullable) {
                        // TODO(parkmycar): Remove this one-time migration after v0.127 ships.
                        (Ok(()), Ok(())) => {
                            let key = Bytes::clone(&encoded_schemas.key);
                            let val = Bytes::clone(&encoded_schemas.val);
                            self.schemas.insert(
                                schema_id,
                                EncodedSchemas {
                                    key,
                                    key_data_type: new_k_encoded_datatype,
                                    val,
                                    val_data_type: new_v_encoded_datatype,
                                },
                            );
                            metrics.one_time_migration_more_nullable.inc();
                            Continue(Some(schema_id))
                        }
                        (k_err, _) => {
                            tracing::info!(
                                "register schemas, Arrow DataType changed\nkey: {:?}\nold: {:?}\nnew: {:?}",
                                k_err,
                                old_k_datatype,
                                new_k_datatype,
                            );
                            Break(NoOpStateTransition(None))
                        }
                    }
                } else {
                    // Everything matches.
                    Break(NoOpStateTransition(Some(schema_id)))
                }
            }
            None if self.is_tombstone() => {
                // TODO: Is this right?
                Break(NoOpStateTransition(None))
            }
            None if self.schemas.is_empty() => {
                // We'll have to do something more sophisticated here to
                // generate the next id if/when we start supporting the removal
                // of schemas.
                let id = SchemaId(self.schemas.len());
                let key_data_type = mz_persist_types::columnar::data_type::<K>(key_schema)
                    .expect("valid key schema");
                let val_data_type = mz_persist_types::columnar::data_type::<V>(val_schema)
                    .expect("valid val schema");
                let prev = self.schemas.insert(
                    id,
                    EncodedSchemas {
                        key: K::encode_schema(key_schema),
                        key_data_type: encode_data_type(&key_data_type),
                        val: V::encode_schema(val_schema),
                        val_data_type: encode_data_type(&val_data_type),
                    },
                );
                assert_eq!(prev, None);
                Continue(Some(id))
            }
            None => {
                info!(
                    "register_schemas got {:?} expected {:?}",
                    key_schema,
                    self.schemas
                        .iter()
                        .map(|(id, x)| (id, K::decode_schema(&x.key)))
                        .collect::<Vec<_>>()
                );
                // Until we implement persist schema changes, only allow at most
                // one registered schema.
                Break(NoOpStateTransition(None))
            }
        }
    }

    pub fn compare_and_evolve_schema<K: Codec, V: Codec>(
        &mut self,
        expected: SchemaId,
        key_schema: &K::Schema,
        val_schema: &V::Schema,
    ) -> ControlFlow<NoOpStateTransition<CaESchema<K, V>>, CaESchema<K, V>> {
        fn data_type<T>(schema: &impl Schema2<T>) -> DataType {
            // To be defensive, create an empty batch and inspect the resulting
            // data type (as opposed to something like allowing the `Schema2` to
            // declare the DataType).
            let array = Schema2::encoder(schema).expect("valid schema").finish();
            Array::data_type(&array).clone()
        }

        let (current_id, current) = self
            .schemas
            .last_key_value()
            .expect("all shards have a schema");
        if *current_id != expected {
            return Break(NoOpStateTransition(CaESchema::ExpectedMismatch {
                schema_id: *current_id,
                key: K::decode_schema(&current.key),
                val: V::decode_schema(&current.val),
            }));
        }

        let current_key = K::decode_schema(&current.key);
        let current_key_dt = EncodedSchemas::decode_data_type(&current.key_data_type);
        let current_val = V::decode_schema(&current.val);
        let current_val_dt = EncodedSchemas::decode_data_type(&current.val_data_type);

        let key_dt = data_type(key_schema);
        let val_dt = data_type(val_schema);

        // If the schema is exactly the same as the current one, no-op.
        if current_key == *key_schema
            && current_key_dt == key_dt
            && current_val == *val_schema
            && current_val_dt == val_dt
        {
            return Break(NoOpStateTransition(CaESchema::Ok(*current_id)));
        }

        let key_fn = backward_compatible(&current_key_dt, &key_dt);
        let val_fn = backward_compatible(&current_val_dt, &val_dt);
        let (Some(key_fn), Some(val_fn)) = (key_fn, val_fn) else {
            return Break(NoOpStateTransition(CaESchema::Incompatible));
        };
        // Persist initially disallows dropping columns. This would require a
        // bunch more work (e.g. not safe to use the latest schema in
        // compaction) and isn't initially necessary in mz.
        if key_fn.contains_drop() || val_fn.contains_drop() {
            return Break(NoOpStateTransition(CaESchema::Incompatible));
        }

        // We'll have to do something more sophisticated here to
        // generate the next id if/when we start supporting the removal
        // of schemas.
        let id = SchemaId(self.schemas.len());
        self.schemas.insert(
            id,
            EncodedSchemas {
                key: K::encode_schema(key_schema),
                key_data_type: prost::Message::encode_to_vec(&key_dt.into_proto()).into(),
                val: V::encode_schema(val_schema),
                val_data_type: prost::Message::encode_to_vec(&val_dt.into_proto()).into(),
            },
        );
        Continue(CaESchema::Ok(id))
    }

    pub fn compare_and_append(
        &mut self,
        batch: &HollowBatch<T>,
        writer_id: &WriterId,
        heartbeat_timestamp_ms: u64,
        lease_duration_ms: u64,
        idempotency_token: &IdempotencyToken,
        debug_info: &HandleDebugState,
        inline_writes_total_max_bytes: usize,
        record_compactions: bool,
        claim_compaction_percent: usize,
        claim_compaction_min_version: Option<&Version>,
    ) -> ControlFlow<CompareAndAppendBreak<T>, Vec<FueledMergeReq<T>>> {
        // We expire all writers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            assert_eq!(self.trace.upper(), &Antichain::new());
            return Break(CompareAndAppendBreak::Upper {
                shard_upper: Antichain::new(),
                // This writer might have been registered before the shard upper
                // was advanced, which would make this pessimistic in the
                // Indeterminate handling of compare_and_append at the machine
                // level, but that's fine.
                writer_upper: Antichain::new(),
            });
        }

        let writer_state = self
            .writers
            .entry(writer_id.clone())
            .or_insert_with(|| WriterState {
                last_heartbeat_timestamp_ms: heartbeat_timestamp_ms,
                lease_duration_ms,
                most_recent_write_token: IdempotencyToken::SENTINEL,
                most_recent_write_upper: Antichain::from_elem(T::minimum()),
                debug: debug_info.clone(),
            });

        if PartialOrder::less_than(batch.desc.upper(), batch.desc.lower()) {
            return Break(CompareAndAppendBreak::InvalidUsage(
                InvalidUsage::InvalidBounds {
                    lower: batch.desc.lower().clone(),
                    upper: batch.desc.upper().clone(),
                },
            ));
        }

        // If the time interval is empty, the list of updates must also be
        // empty.
        if batch.desc.upper() == batch.desc.lower() && !batch.is_empty() {
            return Break(CompareAndAppendBreak::InvalidUsage(
                InvalidUsage::InvalidEmptyTimeInterval {
                    lower: batch.desc.lower().clone(),
                    upper: batch.desc.upper().clone(),
                    keys: batch
                        .parts
                        .iter()
                        .map(|x| x.printable_name().to_owned())
                        .collect(),
                },
            ));
        }

        if idempotency_token == &writer_state.most_recent_write_token {
            // If the last write had the same idempotency_token, then this must
            // have already committed. Sanity check that the most recent write
            // upper matches and that the shard upper is at least the write
            // upper, if it's not something very suspect is going on.
            assert_eq!(batch.desc.upper(), &writer_state.most_recent_write_upper);
            assert!(
                PartialOrder::less_equal(batch.desc.upper(), self.trace.upper()),
                "{:?} vs {:?}",
                batch.desc.upper(),
                self.trace.upper()
            );
            return Break(CompareAndAppendBreak::AlreadyCommitted);
        }

        let shard_upper = self.trace.upper();
        if shard_upper != batch.desc.lower() {
            return Break(CompareAndAppendBreak::Upper {
                shard_upper: shard_upper.clone(),
                writer_upper: writer_state.most_recent_write_upper.clone(),
            });
        }

        let new_inline_bytes = batch.inline_bytes();
        if new_inline_bytes > 0 {
            let mut existing_inline_bytes = 0;
            self.trace
                .map_batches(|x| existing_inline_bytes += x.inline_bytes());
            // TODO: For very small batches, it may actually _increase_ the size
            // of state to flush them out. Consider another threshold under
            // which an inline part can be appended no matter what.
            if existing_inline_bytes + new_inline_bytes >= inline_writes_total_max_bytes {
                return Break(CompareAndAppendBreak::InlineBackpressure);
            }
        }

        let mut merge_reqs = if batch.desc.upper() != batch.desc.lower() {
            self.trace.push_batch(batch.clone())
        } else {
            Vec::new()
        };

        // NB: we don't claim unclaimed compactions when the recording flag is off, even if we'd
        // otherwise be allowed to, to avoid triggering the same compactions in every writer.
        let all_empty_reqs = merge_reqs
            .iter()
            .all(|req| req.inputs.iter().all(|b| b.batch.is_empty()));
        if record_compactions && all_empty_reqs && !batch.is_empty() {
            let mut reqs_to_take = claim_compaction_percent / 100;
            if (usize::cast_from(idempotency_token.hashed()) % 100)
                < (claim_compaction_percent % 100)
            {
                reqs_to_take += 1;
            }
            let threshold_ms = heartbeat_timestamp_ms.saturating_sub(lease_duration_ms);
            let min_writer = claim_compaction_min_version.map(WriterKey::for_version);
            merge_reqs.extend(
                // We keep the oldest `reqs_to_take` batches, under the theory that they're least
                // likely to be compacted soon for other reasons.
                self.trace
                    .fueled_merge_reqs_before_ms(threshold_ms, min_writer)
                    .take(reqs_to_take),
            )
        }

        if record_compactions {
            for req in &merge_reqs {
                self.trace.claim_compaction(
                    req.id,
                    ActiveCompaction {
                        start_ms: heartbeat_timestamp_ms,
                    },
                )
            }
        }

        debug_assert_eq!(self.trace.upper(), batch.desc.upper());
        writer_state.most_recent_write_token = idempotency_token.clone();
        // The writer's most recent upper should only go forward.
        assert!(
            PartialOrder::less_equal(&writer_state.most_recent_write_upper, batch.desc.upper()),
            "{:?} vs {:?}",
            &writer_state.most_recent_write_upper,
            batch.desc.upper()
        );
        writer_state
            .most_recent_write_upper
            .clone_from(batch.desc.upper());

        // Heartbeat the writer state to keep our idempotency token alive.
        writer_state.last_heartbeat_timestamp_ms = std::cmp::max(
            heartbeat_timestamp_ms,
            writer_state.last_heartbeat_timestamp_ms,
        );

        Continue(merge_reqs)
    }

    pub fn apply_merge_res(
        &mut self,
        res: &FueledMergeRes<T>,
    ) -> ControlFlow<NoOpStateTransition<ApplyMergeResult>, ApplyMergeResult> {
        // We expire all writers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            return Break(NoOpStateTransition(ApplyMergeResult::NotAppliedNoMatch));
        }

        let apply_merge_result = self.trace.apply_merge_res(res);
        Continue(apply_merge_result)
    }

    pub fn spine_exert(
        &mut self,
        fuel: usize,
    ) -> ControlFlow<NoOpStateTransition<Vec<FueledMergeReq<T>>>, Vec<FueledMergeReq<T>>> {
        let (merge_reqs, did_work) = self.trace.exert(fuel);
        if did_work {
            Continue(merge_reqs)
        } else {
            assert!(merge_reqs.is_empty());
            // Break if we have nothing useful to do to save the seqno (and
            // resulting crdb traffic)
            Break(NoOpStateTransition(Vec::new()))
        }
    }

    pub fn downgrade_since(
        &mut self,
        reader_id: &LeasedReaderId,
        seqno: SeqNo,
        outstanding_seqno: Option<SeqNo>,
        new_since: &Antichain<T>,
        heartbeat_timestamp_ms: u64,
    ) -> ControlFlow<NoOpStateTransition<Since<T>>, Since<T>> {
        // We expire all readers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            return Break(NoOpStateTransition(Since(Antichain::new())));
        }

        let reader_state = self.leased_reader(reader_id);

        // Also use this as an opportunity to heartbeat the reader and downgrade
        // the seqno capability.
        reader_state.last_heartbeat_timestamp_ms = std::cmp::max(
            heartbeat_timestamp_ms,
            reader_state.last_heartbeat_timestamp_ms,
        );

        let seqno = match outstanding_seqno {
            Some(outstanding_seqno) => {
                assert!(
                    outstanding_seqno >= reader_state.seqno,
                    "SeqNos cannot go backward; however, oldest leased SeqNo ({:?}) \
                    is behind current reader_state ({:?})",
                    outstanding_seqno,
                    reader_state.seqno,
                );
                std::cmp::min(outstanding_seqno, seqno)
            }
            None => seqno,
        };

        reader_state.seqno = seqno;

        let reader_current_since = if PartialOrder::less_than(&reader_state.since, new_since) {
            reader_state.since.clone_from(new_since);
            self.update_since();
            new_since.clone()
        } else {
            // No-op, but still commit the state change so that this gets
            // linearized.
            reader_state.since.clone()
        };

        Continue(Since(reader_current_since))
    }

    pub fn compare_and_downgrade_since<O: Opaque + Codec64>(
        &mut self,
        reader_id: &CriticalReaderId,
        expected_opaque: &O,
        (new_opaque, new_since): (&O, &Antichain<T>),
    ) -> ControlFlow<
        NoOpStateTransition<Result<Since<T>, (O, Since<T>)>>,
        Result<Since<T>, (O, Since<T>)>,
    > {
        // We expire all readers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            // Match the idempotence behavior below of ignoring the token if
            // since is already advanced enough (in this case, because it's a
            // tombstone, we know it's the empty antichain).
            return Break(NoOpStateTransition(Ok(Since(Antichain::new()))));
        }

        let reader_state = self.critical_reader(reader_id);
        assert_eq!(reader_state.opaque_codec, O::codec_name());

        if &O::decode(reader_state.opaque.0) != expected_opaque {
            // No-op, but still commit the state change so that this gets
            // linearized.
            return Continue(Err((
                Codec64::decode(reader_state.opaque.0),
                Since(reader_state.since.clone()),
            )));
        }

        reader_state.opaque = OpaqueState(Codec64::encode(new_opaque));
        if PartialOrder::less_equal(&reader_state.since, new_since) {
            reader_state.since.clone_from(new_since);
            self.update_since();
            Continue(Ok(Since(new_since.clone())))
        } else {
            // no work to be done -- the reader state's `since` is already sufficiently
            // advanced. we may someday need to revisit this branch when it's possible
            // for two `since` frontiers to be incomparable.
            Continue(Ok(Since(reader_state.since.clone())))
        }
    }

    pub fn heartbeat_leased_reader(
        &mut self,
        reader_id: &LeasedReaderId,
        heartbeat_timestamp_ms: u64,
    ) -> ControlFlow<NoOpStateTransition<bool>, bool> {
        // We expire all readers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            return Break(NoOpStateTransition(false));
        }

        match self.leased_readers.get_mut(reader_id) {
            Some(reader_state) => {
                reader_state.last_heartbeat_timestamp_ms = std::cmp::max(
                    heartbeat_timestamp_ms,
                    reader_state.last_heartbeat_timestamp_ms,
                );
                Continue(true)
            }
            // No-op, but we still commit the state change so that this gets
            // linearized (maybe we're looking at old state).
            None => Continue(false),
        }
    }

    pub fn expire_leased_reader(
        &mut self,
        reader_id: &LeasedReaderId,
    ) -> ControlFlow<NoOpStateTransition<bool>, bool> {
        // We expire all readers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            return Break(NoOpStateTransition(false));
        }

        let existed = self.leased_readers.remove(reader_id).is_some();
        if existed {
            // TODO(database-issues#6885): Re-enable this
            //
            // Temporarily disabling this because we think it might be the cause
            // of the remap since bug. Specifically, a clusterd process has a
            // ReadHandle for maintaining the once and one inside a Listen. If
            // we crash and stay down for longer than the read lease duration,
            // it's possible that an expiry of them both in quick succession
            // jumps the since forward to the Listen one.
            //
            // Don't forget to update the downgrade_since when this gets
            // switched back on.
            //
            // self.update_since();
        }
        // No-op if existed is false, but still commit the state change so that
        // this gets linearized.
        Continue(existed)
    }

    pub fn expire_critical_reader(
        &mut self,
        reader_id: &CriticalReaderId,
    ) -> ControlFlow<NoOpStateTransition<bool>, bool> {
        // We expire all readers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            return Break(NoOpStateTransition(false));
        }

        let existed = self.critical_readers.remove(reader_id).is_some();
        if existed {
            // TODO(database-issues#6885): Re-enable this
            //
            // Temporarily disabling this because we think it might be the cause
            // of the remap since bug. Specifically, a clusterd process has a
            // ReadHandle for maintaining the once and one inside a Listen. If
            // we crash and stay down for longer than the read lease duration,
            // it's possible that an expiry of them both in quick succession
            // jumps the since forward to the Listen one.
            //
            // Don't forget to update the downgrade_since when this gets
            // switched back on.
            //
            // self.update_since();
        }
        // This state transition is a no-op if existed is false, but we still
        // commit the state change so that this gets linearized (maybe we're
        // looking at old state).
        Continue(existed)
    }

    pub fn expire_writer(
        &mut self,
        writer_id: &WriterId,
    ) -> ControlFlow<NoOpStateTransition<bool>, bool> {
        // We expire all writers if the upper and since both advance to the
        // empty antichain. Gracefully handle this. At the same time,
        // short-circuit the cmd application so we don't needlessly create new
        // SeqNos.
        if self.is_tombstone() {
            return Break(NoOpStateTransition(false));
        }

        let existed = self.writers.remove(writer_id).is_some();
        // This state transition is a no-op if existed is false, but we still
        // commit the state change so that this gets linearized (maybe we're
        // looking at old state).
        Continue(existed)
    }

    fn leased_reader(&mut self, id: &LeasedReaderId) -> &mut LeasedReaderState<T> {
        self.leased_readers
            .get_mut(id)
            // The only (tm) ways to hit this are (1) inventing a LeasedReaderId
            // instead of getting it from Register or (2) if a lease expired.
            // (1) is a gross mis-use and (2) may happen if a reader did not get
            // to heartbeat for a long time. Readers are expected to
            // heartbeat/downgrade their since regularly.
            .unwrap_or_else(|| {
                panic!(
                    "LeasedReaderId({}) was expired due to inactivity. Did the machine go to sleep?",
                    id
                )
            })
    }

    fn critical_reader(&mut self, id: &CriticalReaderId) -> &mut CriticalReaderState<T> {
        self.critical_readers
            .get_mut(id)
            .unwrap_or_else(|| {
                panic!(
                    "Unknown CriticalReaderId({}). It was either never registered, or has been manually expired.",
                    id
                )
            })
    }

    fn critical_since(&self) -> Option<Antichain<T>> {
        let mut critical_sinces = self.critical_readers.values().map(|r| &r.since);
        let mut since = critical_sinces.next().cloned()?;
        for s in critical_sinces {
            since.meet_assign(s);
        }
        Some(since)
    }

    fn update_since(&mut self) {
        let mut sinces_iter = self
            .leased_readers
            .values()
            .map(|x| &x.since)
            .chain(self.critical_readers.values().map(|x| &x.since));
        let mut since = match sinces_iter.next() {
            Some(since) => since.clone(),
            None => {
                // If there are no current readers, leave `since` unchanged so
                // it doesn't regress.
                return;
            }
        };
        while let Some(s) = sinces_iter.next() {
            since.meet_assign(s);
        }
        self.trace.downgrade_since(&since);
    }

    fn seqno_since(&self, seqno: SeqNo) -> SeqNo {
        let mut seqno_since = seqno;
        for cap in self.leased_readers.values() {
            seqno_since = std::cmp::min(seqno_since, cap.seqno);
        }
        // critical_readers don't hold a seqno capability.
        seqno_since
    }

    fn tombstone_batch() -> HollowBatch<T> {
        HollowBatch::empty(Description::new(
            Antichain::from_elem(T::minimum()),
            Antichain::new(),
            Antichain::new(),
        ))
    }

    pub(crate) fn is_tombstone(&self) -> bool {
        self.trace.upper().is_empty()
            && self.trace.since().is_empty()
            && self.writers.is_empty()
            && self.leased_readers.is_empty()
            && self.critical_readers.is_empty()
    }

    pub(crate) fn is_single_empty_batch(&self) -> bool {
        let mut batch_count = 0;
        let mut is_empty = true;
        self.trace.map_batches(|b| {
            batch_count += 1;
            is_empty &= b.is_empty()
        });
        batch_count <= 1 && is_empty
    }

    pub fn become_tombstone_and_shrink(&mut self) -> ControlFlow<NoOpStateTransition<()>, ()> {
        assert_eq!(self.trace.upper(), &Antichain::new());
        assert_eq!(self.trace.since(), &Antichain::new());

        // Remember our current state, so we can decide whether we have to
        // record a transition in durable state.
        let was_tombstone = self.is_tombstone();

        // Enter the "tombstone" state, if we're not in it already.
        self.writers.clear();
        self.leased_readers.clear();
        self.critical_readers.clear();

        debug_assert!(self.is_tombstone());

        // Now that we're in a "tombstone" state -- ie. nobody can read the data from a shard or write to
        // it -- the actual contents of our batches no longer matter.
        // This method progressively replaces batches in our state with simpler versions, to allow
        // freeing up resources and to reduce the state size. (Since the state is unreadable, this
        // is not visible to clients.) We do this a little bit at a time to avoid really large state
        // transitions... most operations happen incrementally, and large single writes can overwhelm
        // a backing store. See comments for why we believe the relevant diffs are reasonably small.

        let mut to_replace = None;
        let mut batch_count = 0;
        self.trace.map_batches(|b| {
            batch_count += 1;
            if !b.is_empty() && to_replace.is_none() {
                to_replace = Some(b.desc.clone());
            }
        });
        if let Some(desc) = to_replace {
            // We have a nonempty batch: replace it with an empty batch and return.
            // This should not produce an excessively large diff: if it did, we wouldn't have been
            // able to append that batch in the first place.
            let fake_merge = FueledMergeRes {
                output: HollowBatch::empty(desc),
            };
            let result = self.trace.apply_merge_res(&fake_merge);
            assert!(
                result.matched(),
                "merge with a matching desc should always match"
            );
            Continue(())
        } else if batch_count > 1 {
            // All our batches are empty, but we have more than one of them. Replace the whole set
            // with a new single-batch trace.
            // This produces a diff with a size proportional to the number of batches, but since
            // Spine keeps a logarithmic number of batches this should never be excessively large.
            let mut new_trace = Trace::default();
            new_trace.downgrade_since(&Antichain::new());
            let merge_reqs = new_trace.push_batch(Self::tombstone_batch());
            assert_eq!(merge_reqs, Vec::new());
            self.trace = new_trace;
            Continue(())
        } else if !was_tombstone {
            // We were not tombstoned before, so have to make sure this state
            // transition is recorded.
            Continue(())
        } else {
            // All our batches are empty, and there's only one... there's no shrinking this
            // tombstone further.
            Break(NoOpStateTransition(()))
        }
    }
}

// TODO: Document invariants.
#[derive(Debug)]
#[cfg_attr(any(test, debug_assertions), derive(Clone, PartialEq))]
pub struct State<T> {
    pub(crate) applier_version: semver::Version,
    pub(crate) shard_id: ShardId,

    pub(crate) seqno: SeqNo,
    /// A strictly increasing wall time of when this state was written, in
    /// milliseconds since the unix epoch.
    pub(crate) walltime_ms: u64,
    /// Hostname of the persist user that created this version of state. For
    /// debugging.
    pub(crate) hostname: String,
    pub(crate) collections: StateCollections<T>,
}

/// A newtype wrapper of State that guarantees the K, V, and D codecs match the
/// ones in durable storage.
pub struct TypedState<K, V, T, D> {
    pub(crate) state: State<T>,

    // According to the docs, PhantomData is to "mark things that act like they
    // own a T". State doesn't actually own K, V, or D, just the ability to
    // produce them. Using the `fn() -> T` pattern gets us the same variance as
    // T [1], but also allows State to correctly derive Send+Sync.
    //
    // [1]:
    //     https://doc.rust-lang.org/nomicon/phantom-data.html#table-of-phantomdata-patterns
    pub(crate) _phantom: PhantomData<fn() -> (K, V, D)>,
}

impl<K, V, T: Clone, D> TypedState<K, V, T, D> {
    #[cfg(any(test, debug_assertions))]
    pub(crate) fn clone(&self, applier_version: Version, hostname: String) -> Self {
        TypedState {
            state: State {
                applier_version,
                shard_id: self.shard_id.clone(),
                seqno: self.seqno.clone(),
                walltime_ms: self.walltime_ms,
                hostname,
                collections: self.collections.clone(),
            },
            _phantom: PhantomData,
        }
    }

    pub(crate) fn clone_for_rollup(&self) -> Self {
        TypedState {
            state: State {
                applier_version: self.applier_version.clone(),
                shard_id: self.shard_id.clone(),
                seqno: self.seqno.clone(),
                walltime_ms: self.walltime_ms,
                hostname: self.hostname.clone(),
                collections: self.collections.clone(),
            },
            _phantom: PhantomData,
        }
    }
}

impl<K, V, T: Debug, D> Debug for TypedState<K, V, T, D> {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        // Deconstruct self so we get a compile failure if new fields
        // are added.
        let TypedState { state, _phantom } = self;
        f.debug_struct("TypedState").field("state", state).finish()
    }
}

// Impl PartialEq regardless of the type params.
#[cfg(any(test, debug_assertions))]
impl<K, V, T: PartialEq, D> PartialEq for TypedState<K, V, T, D> {
    fn eq(&self, other: &Self) -> bool {
        // Deconstruct self and other so we get a compile failure if new fields
        // are added.
        let TypedState {
            state: self_state,
            _phantom,
        } = self;
        let TypedState {
            state: other_state,
            _phantom,
        } = other;
        self_state == other_state
    }
}

impl<K, V, T, D> Deref for TypedState<K, V, T, D> {
    type Target = State<T>;

    fn deref(&self) -> &Self::Target {
        &self.state
    }
}

impl<K, V, T, D> DerefMut for TypedState<K, V, T, D> {
    fn deref_mut(&mut self) -> &mut Self::Target {
        &mut self.state
    }
}

impl<K, V, T, D> TypedState<K, V, T, D>
where
    K: Codec,
    V: Codec,
    T: Timestamp + Lattice + Codec64,
    D: Codec64,
{
    pub fn new(
        applier_version: Version,
        shard_id: ShardId,
        hostname: String,
        walltime_ms: u64,
    ) -> Self {
        let state = State {
            applier_version,
            shard_id,
            seqno: SeqNo::minimum(),
            walltime_ms,
            hostname,
            collections: StateCollections {
                last_gc_req: SeqNo::minimum(),
                rollups: BTreeMap::new(),
                leased_readers: BTreeMap::new(),
                critical_readers: BTreeMap::new(),
                writers: BTreeMap::new(),
                schemas: BTreeMap::new(),
                trace: Trace::default(),
            },
        };
        TypedState {
            state,
            _phantom: PhantomData,
        }
    }

    pub fn clone_apply<R, E, WorkFn>(
        &self,
        cfg: &PersistConfig,
        work_fn: &mut WorkFn,
    ) -> ControlFlow<E, (R, Self)>
    where
        WorkFn: FnMut(SeqNo, &PersistConfig, &mut StateCollections<T>) -> ControlFlow<E, R>,
    {
        // Now that we support one minor version of forward compatibility, tag
        // each version of state with the _max_ version of code that has ever
        // contributed to it. Otherwise, we'd erroneously allow rolling back an
        // arbitrary number of versions if they were done one-by-one.
        let new_applier_version = std::cmp::max(&self.applier_version, &cfg.build_version);
        let mut new_state = State {
            applier_version: new_applier_version.clone(),
            shard_id: self.shard_id,
            seqno: self.seqno.next(),
            walltime_ms: (cfg.now)(),
            hostname: cfg.hostname.clone(),
            collections: self.collections.clone(),
        };
        // Make sure walltime_ms is strictly increasing, in case clocks are
        // offset.
        if new_state.walltime_ms <= self.walltime_ms {
            new_state.walltime_ms = self.walltime_ms + 1;
        }

        let work_ret = work_fn(new_state.seqno, cfg, &mut new_state.collections)?;
        let new_state = TypedState {
            state: new_state,
            _phantom: PhantomData,
        };
        Continue((work_ret, new_state))
    }
}

impl<T> State<T>
where
    T: Timestamp + Lattice + Codec64,
{
    pub fn shard_id(&self) -> ShardId {
        self.shard_id
    }

    pub fn seqno(&self) -> SeqNo {
        self.seqno
    }

    pub fn since(&self) -> &Antichain<T> {
        self.collections.trace.since()
    }

    pub fn upper(&self) -> &Antichain<T> {
        self.collections.trace.upper()
    }

    pub fn spine_batch_count(&self) -> usize {
        self.collections.trace.num_spine_batches()
    }

    pub fn size_metrics(&self) -> StateSizeMetrics {
        let mut ret = StateSizeMetrics::default();
        self.blobs().for_each(|x| match x {
            HollowBlobRef::Batch(x) => {
                ret.hollow_batch_count += 1;
                ret.batch_part_count += x.part_count();
                ret.num_updates += x.len;

                let batch_size = x.encoded_size_bytes();
                for x in x.parts.iter() {
                    if x.ts_rewrite().is_some() {
                        ret.rewrite_part_count += 1;
                    }
                    if x.is_inline() {
                        ret.inline_part_count += 1;
                        ret.inline_part_bytes += x.inline_bytes();
                    }
                }
                ret.largest_batch_bytes = std::cmp::max(ret.largest_batch_bytes, batch_size);
                ret.state_batches_bytes += batch_size;
            }
            HollowBlobRef::Rollup(x) => {
                ret.state_rollup_count += 1;
                ret.state_rollups_bytes += x.encoded_size_bytes.unwrap_or_default()
            }
        });
        ret
    }

    pub fn latest_rollup(&self) -> (&SeqNo, &HollowRollup) {
        // We maintain the invariant that every version of state has at least
        // one rollup.
        self.collections
            .rollups
            .iter()
            .rev()
            .next()
            .expect("State should have at least one rollup if seqno > minimum")
    }

    pub(crate) fn seqno_since(&self) -> SeqNo {
        self.collections.seqno_since(self.seqno)
    }

    // Returns whether the cmd proposing this state has been selected to perform
    // background garbage collection work.
    //
    // If it was selected, this information is recorded in the state itself for
    // commit along with the cmd's state transition. This helps us to avoid
    // redundant work.
    //
    // Correctness does not depend on a gc assignment being executed, nor on
    // them being executed in the order they are given. But it is expected that
    // gc assignments are best-effort respected. In practice, cmds like
    // register_foo or expire_foo, where it would be awkward, ignore gc.
    pub fn maybe_gc(&mut self, is_write: bool) -> Option<GcReq> {
        // This is an arbitrary-ish threshold that scales with seqno, but never
        // gets particularly big. It probably could be much bigger and certainly
        // could use a tuning pass at some point.
        let gc_threshold = std::cmp::max(
            1,
            u64::from(self.seqno.0.next_power_of_two().trailing_zeros()),
        );
        let new_seqno_since = self.seqno_since();
        let should_gc = new_seqno_since
            .0
            .saturating_sub(self.collections.last_gc_req.0)
            >= gc_threshold;
        // Assign GC traffic preferentially to writers, falling back to anyone
        // generating new state versions if there are no writers.
        let should_gc = should_gc && (is_write || self.collections.writers.is_empty());
        // Always assign GC work to a tombstoned shard to have the chance to
        // clean up any residual blobs. This is safe (won't cause excess gc)
        // as the only allowed command after becoming a tombstone is to write
        // the final rollup.
        let tombstone_needs_gc = self.collections.is_tombstone();
        let should_gc = should_gc || tombstone_needs_gc;
        if should_gc {
            self.collections.last_gc_req = new_seqno_since;
            Some(GcReq {
                shard_id: self.shard_id,
                new_seqno_since,
            })
        } else {
            None
        }
    }

    /// Return the number of gc-ineligible state versions.
    pub fn seqnos_held(&self) -> usize {
        usize::cast_from(self.seqno.0.saturating_sub(self.seqno_since().0))
    }

    /// Expire all readers and writers up to the given walltime_ms.
    pub fn expire_at(&mut self, walltime_ms: EpochMillis) -> ExpiryMetrics {
        let mut metrics = ExpiryMetrics::default();
        let shard_id = self.shard_id();
        self.collections.leased_readers.retain(|k, v| {
            let retain = v.last_heartbeat_timestamp_ms + v.lease_duration_ms >= walltime_ms;
            if !retain {
                info!("Force expiring reader ({k}) of shard ({shard_id}) due to inactivity");
                metrics.readers_expired += 1;
            }
            retain
        });
        // critical_readers don't need forced expiration. (In fact, that's the point!)
        self.collections.writers.retain(|k, v| {
            let retain = (v.last_heartbeat_timestamp_ms + v.lease_duration_ms) >= walltime_ms;
            if !retain {
                info!("Force expiring writer ({k}) of shard ({shard_id}) due to inactivity");
                metrics.writers_expired += 1;
            }
            retain
        });
        metrics
    }

    /// Returns the batches that contain updates up to (and including) the given `as_of`. The
    /// result `Vec` contains blob keys, along with a [`Description`] of what updates in the
    /// referenced parts are valid to read.
    pub fn snapshot(&self, as_of: &Antichain<T>) -> Result<Vec<HollowBatch<T>>, SnapshotErr<T>> {
        if PartialOrder::less_than(as_of, self.collections.trace.since()) {
            return Err(SnapshotErr::AsOfHistoricalDistinctionsLost(Since(
                self.collections.trace.since().clone(),
            )));
        }
        let upper = self.collections.trace.upper();
        if PartialOrder::less_equal(upper, as_of) {
            return Err(SnapshotErr::AsOfNotYetAvailable(
                self.seqno,
                Upper(upper.clone()),
            ));
        }

        let batches = self
            .collections
            .trace
            .batches()
            .filter(|b| !PartialOrder::less_than(as_of, b.desc.lower()))
            .cloned()
            .collect();
        Ok(batches)
    }

    // NB: Unlike the other methods here, this one is read-only.
    pub fn verify_listen(&self, as_of: &Antichain<T>) -> Result<Result<(), Upper<T>>, Since<T>> {
        if PartialOrder::less_than(as_of, self.collections.trace.since()) {
            return Err(Since(self.collections.trace.since().clone()));
        }
        let upper = self.collections.trace.upper();
        if PartialOrder::less_equal(upper, as_of) {
            return Ok(Err(Upper(upper.clone())));
        }
        Ok(Ok(()))
    }

    pub fn next_listen_batch(&self, frontier: &Antichain<T>) -> Result<HollowBatch<T>, SeqNo> {
        // TODO: Avoid the O(n^2) here: `next_listen_batch` is called once per
        // batch and this iterates through all batches to find the next one.
        self.collections
            .trace
            .batches()
            .find(|b| {
                PartialOrder::less_equal(b.desc.lower(), frontier)
                    && PartialOrder::less_than(frontier, b.desc.upper())
            })
            .cloned()
            .ok_or(self.seqno)
    }

    pub fn need_rollup(&self, threshold: usize) -> Option<SeqNo> {
        let (latest_rollup_seqno, _) = self.latest_rollup();

        // Tombstoned shards require one final rollup. However, because we
        // write a rollup as of SeqNo X and then link it in using a state
        // transition (in this case from X to X+1), the minimum number of
        // live diffs is actually two. Detect when we're in this minimal
        // two diff state and stop the (otherwise) infinite iteration.
        if self.collections.is_tombstone() && latest_rollup_seqno.next() < self.seqno {
            return Some(self.seqno);
        }

        let seqnos_since_last_rollup = self.seqno.0.saturating_sub(latest_rollup_seqno.0);
        // every `threshold` seqnos since the latest rollup, assign rollup maintenance.
        // we avoid assigning rollups to every seqno past the threshold to avoid handles
        // racing / performing redundant work.
        if seqnos_since_last_rollup > 0 && seqnos_since_last_rollup % u64::cast_from(threshold) == 0
        {
            return Some(self.seqno);
        }

        // however, since maintenance is best-effort and could fail, do assign rollup
        // work to every seqno after a fallback threshold to ensure one is written.
        if seqnos_since_last_rollup
            > u64::cast_from(
                threshold * PersistConfig::DEFAULT_FALLBACK_ROLLUP_THRESHOLD_MULTIPLIER,
            )
        {
            return Some(self.seqno);
        }

        None
    }

    pub(crate) fn blobs(&self) -> impl Iterator<Item = HollowBlobRef<T>> {
        let batches = self.collections.trace.batches().map(HollowBlobRef::Batch);
        let rollups = self.collections.rollups.values().map(HollowBlobRef::Rollup);
        batches.chain(rollups)
    }
}

fn serialize_part_bytes<S: Serializer>(val: &[u8], s: S) -> Result<S::Ok, S::Error> {
    let val = hex::encode(val);
    val.serialize(s)
}

fn serialize_lazy_proto<S: Serializer, T: prost::Message + Default>(
    val: &Option<LazyProto<T>>,
    s: S,
) -> Result<S::Ok, S::Error> {
    val.as_ref()
        .map(|lazy| hex::encode(&lazy.into_proto()))
        .serialize(s)
}

fn serialize_part_stats<S: Serializer>(
    val: &Option<LazyPartStats>,
    s: S,
) -> Result<S::Ok, S::Error> {
    let val = val.as_ref().map(|x| x.decode().key);
    val.serialize(s)
}

fn serialize_diffs_sum<S: Serializer>(val: &Option<[u8; 8]>, s: S) -> Result<S::Ok, S::Error> {
    // This is only used for debugging, so hack to assume that D is i64.
    let val = val.map(i64::decode);
    val.serialize(s)
}

// This Serialize impl is used for debugging/testing and exposed via SQL. It's
// intentionally gated from users, so not strictly subject to our backward
// compatibility guarantees, but still probably best to be thoughtful about
// making unnecessary changes. Additionally, it's nice to make the output as
// nice to use as possible without tying our hands for the actual code usages.
impl<T: Serialize + Timestamp + Lattice> Serialize for State<T> {
    fn serialize<S: Serializer>(&self, s: S) -> Result<S::Ok, S::Error> {
        let State {
            applier_version,
            shard_id,
            seqno,
            walltime_ms,
            hostname,
            collections:
                StateCollections {
                    last_gc_req,
                    rollups,
                    leased_readers,
                    critical_readers,
                    writers,
                    schemas,
                    trace,
                },
        } = self;
        let mut s = s.serialize_struct("State", 13)?;
        let () = s.serialize_field("applier_version", &applier_version.to_string())?;
        let () = s.serialize_field("shard_id", shard_id)?;
        let () = s.serialize_field("seqno", seqno)?;
        let () = s.serialize_field("walltime_ms", walltime_ms)?;
        let () = s.serialize_field("hostname", hostname)?;
        let () = s.serialize_field("last_gc_req", last_gc_req)?;
        let () = s.serialize_field("rollups", rollups)?;
        let () = s.serialize_field("leased_readers", leased_readers)?;
        let () = s.serialize_field("critical_readers", critical_readers)?;
        let () = s.serialize_field("writers", writers)?;
        let () = s.serialize_field("schemas", schemas)?;
        let () = s.serialize_field("since", &trace.since().elements())?;
        let () = s.serialize_field("upper", &trace.upper().elements())?;
        let trace = trace.flatten();
        let () = s.serialize_field("batches", &trace.legacy_batches.keys().collect::<Vec<_>>())?;
        let () = s.serialize_field("hollow_batches", &trace.hollow_batches)?;
        let () = s.serialize_field("spine_batches", &trace.spine_batches)?;
        let () = s.serialize_field("merges", &trace.merges)?;
        s.end()
    }
}

#[derive(Debug, Default)]
pub struct StateSizeMetrics {
    pub hollow_batch_count: usize,
    pub batch_part_count: usize,
    pub rewrite_part_count: usize,
    pub num_updates: usize,
    pub largest_batch_bytes: usize,
    pub state_batches_bytes: usize,
    pub state_rollups_bytes: usize,
    pub state_rollup_count: usize,
    pub inline_part_count: usize,
    pub inline_part_bytes: usize,
}

#[derive(Default)]
pub struct ExpiryMetrics {
    pub(crate) readers_expired: usize,
    pub(crate) writers_expired: usize,
}

/// Wrapper for Antichain that represents a Since
#[derive(Debug, Clone, PartialEq)]
pub struct Since<T>(pub Antichain<T>);

/// Wrapper for Antichain that represents an Upper
#[derive(Debug, PartialEq)]
pub struct Upper<T>(pub Antichain<T>);

#[cfg(test)]
pub(crate) mod tests {
    use std::ops::Range;

    use bytes::Bytes;
    use mz_build_info::DUMMY_BUILD_INFO;
    use mz_dyncfg::ConfigUpdates;
    use mz_ore::now::SYSTEM_TIME;
    use mz_ore::{assert_none, assert_ok};
    use mz_proto::RustType;
    use proptest::prelude::*;
    use proptest::strategy::ValueTree;

    use crate::cache::PersistClientCache;
    use crate::internal::encoding::any_some_lazy_part_stats;
    use crate::internal::paths::RollupId;
    use crate::internal::trace::tests::any_trace;
    use crate::tests::new_test_client_cache;
    use crate::InvalidUsage::{InvalidBounds, InvalidEmptyTimeInterval};
    use crate::PersistLocation;

    use super::*;

    const LEASE_DURATION_MS: u64 = 900 * 1000;
    fn debug_state() -> HandleDebugState {
        HandleDebugState {
            hostname: "debug".to_owned(),
            purpose: "finding the bugs".to_owned(),
        }
    }

    pub fn any_hollow_batch<T: Arbitrary + Timestamp>() -> impl Strategy<Value = HollowBatch<T>> {
        Strategy::prop_map(
            (
                any::<T>(),
                any::<T>(),
                any::<T>(),
                proptest::collection::vec(any_run_part::<T>(), 0..3),
                any::<usize>(),
                any::<bool>(),
            ),
            |(t0, t1, since, parts, len, runs)| {
                let (lower, upper) = if t0 <= t1 {
                    (Antichain::from_elem(t0), Antichain::from_elem(t1))
                } else {
                    (Antichain::from_elem(t1), Antichain::from_elem(t0))
                };
                let since = Antichain::from_elem(since);
                if runs && parts.len() > 2 {
                    let split_at = parts.len() / 2;
                    HollowBatch::new(
                        Description::new(lower, upper, since),
                        parts,
                        len % 10,
                        vec![RunMeta::default(), RunMeta::default()],
                        vec![split_at],
                    )
                } else {
                    HollowBatch::new_run(Description::new(lower, upper, since), parts, len % 10)
                }
            },
        )
    }

    pub fn any_batch_part<T: Arbitrary + Timestamp>() -> impl Strategy<Value = BatchPart<T>> {
        Strategy::prop_map(
            (
                any::<bool>(),
                any_hollow_batch_part(),
                any::<Option<T>>(),
                any::<Option<SchemaId>>(),
                any::<Option<SchemaId>>(),
            ),
            |(is_hollow, hollow, ts_rewrite, schema_id, deprecated_schema_id)| {
                if is_hollow {
                    BatchPart::Hollow(hollow)
                } else {
                    let updates = LazyInlineBatchPart::from_proto(Bytes::new()).unwrap();
                    let ts_rewrite = ts_rewrite.map(Antichain::from_elem);
                    BatchPart::Inline {
                        updates,
                        ts_rewrite,
                        schema_id,
                        deprecated_schema_id,
                    }
                }
            },
        )
    }

    pub fn any_run_part<T: Arbitrary + Timestamp>() -> impl Strategy<Value = RunPart<T>> {
        Strategy::prop_map(any_batch_part(), |part| RunPart::Single(part))
    }

    pub fn any_hollow_batch_part<T: Arbitrary + Timestamp>(
    ) -> impl Strategy<Value = HollowBatchPart<T>> {
        Strategy::prop_map(
            (
                any::<PartialBatchKey>(),
                any::<usize>(),
                any::<Vec<u8>>(),
                any_some_lazy_part_stats(),
                any::<Option<T>>(),
                any::<[u8; 8]>(),
                any::<Option<BatchColumnarFormat>>(),
                any::<Option<SchemaId>>(),
                any::<Option<SchemaId>>(),
            ),
            |(
                key,
                encoded_size_bytes,
                key_lower,
                stats,
                ts_rewrite,
                diffs_sum,
                format,
                schema_id,
                deprecated_schema_id,
            )| {
                HollowBatchPart {
                    key,
                    encoded_size_bytes,
                    key_lower,
                    structured_key_lower: None,
                    stats,
                    ts_rewrite: ts_rewrite.map(Antichain::from_elem),
                    diffs_sum: Some(diffs_sum),
                    format,
                    schema_id,
                    deprecated_schema_id,
                }
            },
        )
    }

    pub fn any_leased_reader_state<T: Arbitrary>() -> impl Strategy<Value = LeasedReaderState<T>> {
        Strategy::prop_map(
            (
                any::<SeqNo>(),
                any::<Option<T>>(),
                any::<u64>(),
                any::<u64>(),
                any::<HandleDebugState>(),
            ),
            |(seqno, since, last_heartbeat_timestamp_ms, mut lease_duration_ms, debug)| {
                // lease_duration_ms of 0 means this state was written by an old
                // version of code, which means we'll migrate it in the decode
                // path. Avoid.
                if lease_duration_ms == 0 {
                    lease_duration_ms += 1;
                }
                LeasedReaderState {
                    seqno,
                    since: since.map_or_else(Antichain::new, Antichain::from_elem),
                    last_heartbeat_timestamp_ms,
                    lease_duration_ms,
                    debug,
                }
            },
        )
    }

    pub fn any_critical_reader_state<T: Arbitrary>() -> impl Strategy<Value = CriticalReaderState<T>>
    {
        Strategy::prop_map(
            (
                any::<Option<T>>(),
                any::<OpaqueState>(),
                any::<String>(),
                any::<HandleDebugState>(),
            ),
            |(since, opaque, opaque_codec, debug)| CriticalReaderState {
                since: since.map_or_else(Antichain::new, Antichain::from_elem),
                opaque,
                opaque_codec,
                debug,
            },
        )
    }

    pub fn any_writer_state<T: Arbitrary>() -> impl Strategy<Value = WriterState<T>> {
        Strategy::prop_map(
            (
                any::<u64>(),
                any::<u64>(),
                any::<IdempotencyToken>(),
                any::<Option<T>>(),
                any::<HandleDebugState>(),
            ),
            |(
                last_heartbeat_timestamp_ms,
                lease_duration_ms,
                most_recent_write_token,
                most_recent_write_upper,
                debug,
            )| WriterState {
                last_heartbeat_timestamp_ms,
                lease_duration_ms,
                most_recent_write_token,
                most_recent_write_upper: most_recent_write_upper
                    .map_or_else(Antichain::new, Antichain::from_elem),
                debug,
            },
        )
    }

    pub fn any_encoded_schemas() -> impl Strategy<Value = EncodedSchemas> {
        Strategy::prop_map(
            (
                any::<Vec<u8>>(),
                any::<Vec<u8>>(),
                any::<Vec<u8>>(),
                any::<Vec<u8>>(),
            ),
            |(key, key_data_type, val, val_data_type)| EncodedSchemas {
                key: Bytes::from(key),
                key_data_type: Bytes::from(key_data_type),
                val: Bytes::from(val),
                val_data_type: Bytes::from(val_data_type),
            },
        )
    }

    pub fn any_state<T: Arbitrary + Timestamp + Lattice>(
        num_trace_batches: Range<usize>,
    ) -> impl Strategy<Value = State<T>> {
        Strategy::prop_map(
            (
                any::<ShardId>(),
                any::<SeqNo>(),
                any::<u64>(),
                any::<String>(),
                any::<SeqNo>(),
                proptest::collection::btree_map(any::<SeqNo>(), any::<HollowRollup>(), 1..3),
                proptest::collection::btree_map(
                    any::<LeasedReaderId>(),
                    any_leased_reader_state::<T>(),
                    1..3,
                ),
                proptest::collection::btree_map(
                    any::<CriticalReaderId>(),
                    any_critical_reader_state::<T>(),
                    1..3,
                ),
                proptest::collection::btree_map(any::<WriterId>(), any_writer_state::<T>(), 0..3),
                proptest::collection::btree_map(any::<SchemaId>(), any_encoded_schemas(), 0..3),
                any_trace::<T>(num_trace_batches),
            ),
            |(
                shard_id,
                seqno,
                walltime_ms,
                hostname,
                last_gc_req,
                rollups,
                leased_readers,
                critical_readers,
                writers,
                schemas,
                trace,
            )| State {
                applier_version: semver::Version::new(1, 2, 3),
                shard_id,
                seqno,
                walltime_ms,
                hostname,
                collections: StateCollections {
                    last_gc_req,
                    rollups,
                    leased_readers,
                    critical_readers,
                    writers,
                    trace,
                    schemas,
                },
            },
        )
    }

    pub(crate) fn hollow<T: Timestamp>(
        lower: T,
        upper: T,
        keys: &[&str],
        len: usize,
    ) -> HollowBatch<T> {
        HollowBatch::new_run(
            Description::new(
                Antichain::from_elem(lower),
                Antichain::from_elem(upper),
                Antichain::from_elem(T::minimum()),
            ),
            keys.iter()
                .map(|x| {
                    RunPart::Single(BatchPart::Hollow(HollowBatchPart {
                        key: PartialBatchKey((*x).to_owned()),
                        encoded_size_bytes: 0,
                        key_lower: vec![],
                        structured_key_lower: None,
                        stats: None,
                        ts_rewrite: None,
                        diffs_sum: None,
                        format: None,
                        schema_id: None,
                        deprecated_schema_id: None,
                    }))
                })
                .collect(),
            len,
        )
    }

    #[mz_ore::test]
    fn downgrade_since() {
        let mut state = TypedState::<(), (), u64, i64>::new(
            DUMMY_BUILD_INFO.semver_version(),
            ShardId::new(),
            "".to_owned(),
            0,
        );
        let reader = LeasedReaderId::new();
        let seqno = SeqNo::minimum();
        let now = SYSTEM_TIME.clone();
        let _ = state.collections.register_leased_reader(
            "",
            &reader,
            "",
            seqno,
            Duration::from_secs(10),
            now(),
            false,
        );

        // The shard global since == 0 initially.
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(0));

        // Greater
        assert_eq!(
            state.collections.downgrade_since(
                &reader,
                seqno,
                None,
                &Antichain::from_elem(2),
                now()
            ),
            Continue(Since(Antichain::from_elem(2)))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(2));
        // Equal (no-op)
        assert_eq!(
            state.collections.downgrade_since(
                &reader,
                seqno,
                None,
                &Antichain::from_elem(2),
                now()
            ),
            Continue(Since(Antichain::from_elem(2)))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(2));
        // Less (no-op)
        assert_eq!(
            state.collections.downgrade_since(
                &reader,
                seqno,
                None,
                &Antichain::from_elem(1),
                now()
            ),
            Continue(Since(Antichain::from_elem(2)))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(2));

        // Create a second reader.
        let reader2 = LeasedReaderId::new();
        let _ = state.collections.register_leased_reader(
            "",
            &reader2,
            "",
            seqno,
            Duration::from_secs(10),
            now(),
            false,
        );

        // Shard since doesn't change until the meet (min) of all reader sinces changes.
        assert_eq!(
            state.collections.downgrade_since(
                &reader2,
                seqno,
                None,
                &Antichain::from_elem(3),
                now()
            ),
            Continue(Since(Antichain::from_elem(3)))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(2));
        // Shard since == 3 when all readers have since >= 3.
        assert_eq!(
            state.collections.downgrade_since(
                &reader,
                seqno,
                None,
                &Antichain::from_elem(5),
                now()
            ),
            Continue(Since(Antichain::from_elem(5)))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(3));

        // Shard since unaffected readers with since > shard since expiring.
        assert_eq!(
            state.collections.expire_leased_reader(&reader),
            Continue(true)
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(3));

        // Create a third reader.
        let reader3 = LeasedReaderId::new();
        let _ = state.collections.register_leased_reader(
            "",
            &reader3,
            "",
            seqno,
            Duration::from_secs(10),
            now(),
            false,
        );

        // Shard since doesn't change until the meet (min) of all reader sinces changes.
        assert_eq!(
            state.collections.downgrade_since(
                &reader3,
                seqno,
                None,
                &Antichain::from_elem(10),
                now()
            ),
            Continue(Since(Antichain::from_elem(10)))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(3));

        // Shard since advances when reader with the minimal since expires.
        assert_eq!(
            state.collections.expire_leased_reader(&reader2),
            Continue(true)
        );
        // TODO(database-issues#6885): expiry temporarily doesn't advance since
        // Switch this assertion back when we re-enable this.
        //
        // assert_eq!(state.collections.trace.since(), &Antichain::from_elem(10));
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(3));

        // Shard since unaffected when all readers are expired.
        assert_eq!(
            state.collections.expire_leased_reader(&reader3),
            Continue(true)
        );
        // TODO(database-issues#6885): expiry temporarily doesn't advance since
        // Switch this assertion back when we re-enable this.
        //
        // assert_eq!(state.collections.trace.since(), &Antichain::from_elem(10));
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(3));
    }

    #[mz_ore::test]
    fn compare_and_downgrade_since() {
        let mut state = TypedState::<(), (), u64, i64>::new(
            DUMMY_BUILD_INFO.semver_version(),
            ShardId::new(),
            "".to_owned(),
            0,
        );
        let reader = CriticalReaderId::new();
        let _ = state
            .collections
            .register_critical_reader::<u64>("", &reader, "");

        // The shard global since == 0 initially.
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(0));
        // The initial opaque value should be set.
        assert_eq!(
            u64::decode(state.collections.critical_reader(&reader).opaque.0),
            u64::initial()
        );

        // Greater
        assert_eq!(
            state.collections.compare_and_downgrade_since::<u64>(
                &reader,
                &u64::initial(),
                (&1, &Antichain::from_elem(2)),
            ),
            Continue(Ok(Since(Antichain::from_elem(2))))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(2));
        assert_eq!(
            u64::decode(state.collections.critical_reader(&reader).opaque.0),
            1
        );
        // Equal (no-op)
        assert_eq!(
            state.collections.compare_and_downgrade_since::<u64>(
                &reader,
                &1,
                (&2, &Antichain::from_elem(2)),
            ),
            Continue(Ok(Since(Antichain::from_elem(2))))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(2));
        assert_eq!(
            u64::decode(state.collections.critical_reader(&reader).opaque.0),
            2
        );
        // Less (no-op)
        assert_eq!(
            state.collections.compare_and_downgrade_since::<u64>(
                &reader,
                &2,
                (&3, &Antichain::from_elem(1)),
            ),
            Continue(Ok(Since(Antichain::from_elem(2))))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(2));
        assert_eq!(
            u64::decode(state.collections.critical_reader(&reader).opaque.0),
            3
        );
    }

    #[mz_ore::test]
    fn compare_and_append() {
        let state = &mut TypedState::<String, String, u64, i64>::new(
            DUMMY_BUILD_INFO.semver_version(),
            ShardId::new(),
            "".to_owned(),
            0,
        )
        .collections;

        let writer_id = WriterId::new();
        let now = SYSTEM_TIME.clone();

        // State is initially empty.
        assert_eq!(state.trace.num_spine_batches(), 0);
        assert_eq!(state.trace.num_hollow_batches(), 0);
        assert_eq!(state.trace.num_updates(), 0);

        // Cannot insert a batch with a lower != current shard upper.
        assert_eq!(
            state.compare_and_append(
                &hollow(1, 2, &["key1"], 1),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            ),
            Break(CompareAndAppendBreak::Upper {
                shard_upper: Antichain::from_elem(0),
                writer_upper: Antichain::from_elem(0)
            })
        );

        // Insert an empty batch with an upper > lower..
        assert!(state
            .compare_and_append(
                &hollow(0, 5, &[], 0),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());

        // Cannot insert a batch with a upper less than the lower.
        assert_eq!(
            state.compare_and_append(
                &hollow(5, 4, &["key1"], 1),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            ),
            Break(CompareAndAppendBreak::InvalidUsage(InvalidBounds {
                lower: Antichain::from_elem(5),
                upper: Antichain::from_elem(4)
            }))
        );

        // Cannot insert a nonempty batch with an upper equal to lower.
        assert_eq!(
            state.compare_and_append(
                &hollow(5, 5, &["key1"], 1),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            ),
            Break(CompareAndAppendBreak::InvalidUsage(
                InvalidEmptyTimeInterval {
                    lower: Antichain::from_elem(5),
                    upper: Antichain::from_elem(5),
                    keys: vec!["key1".to_owned()],
                }
            ))
        );

        // Can insert an empty batch with an upper equal to lower.
        assert!(state
            .compare_and_append(
                &hollow(5, 5, &[], 0),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());
    }

    #[mz_ore::test]
    fn snapshot() {
        let now = SYSTEM_TIME.clone();

        let mut state = TypedState::<String, String, u64, i64>::new(
            DUMMY_BUILD_INFO.semver_version(),
            ShardId::new(),
            "".to_owned(),
            0,
        );
        // Cannot take a snapshot with as_of == shard upper.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(0)),
            Err(SnapshotErr::AsOfNotYetAvailable(
                SeqNo(0),
                Upper(Antichain::from_elem(0))
            ))
        );

        // Cannot take a snapshot with as_of > shard upper.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(5)),
            Err(SnapshotErr::AsOfNotYetAvailable(
                SeqNo(0),
                Upper(Antichain::from_elem(0))
            ))
        );

        let writer_id = WriterId::new();

        // Advance upper to 5.
        assert!(state
            .collections
            .compare_and_append(
                &hollow(0, 5, &["key1"], 1),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());

        // Can take a snapshot with as_of < upper.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(0)),
            Ok(vec![hollow(0, 5, &["key1"], 1)])
        );

        // Can take a snapshot with as_of >= shard since, as long as as_of < shard_upper.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(4)),
            Ok(vec![hollow(0, 5, &["key1"], 1)])
        );

        // Cannot take a snapshot with as_of >= upper.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(5)),
            Err(SnapshotErr::AsOfNotYetAvailable(
                SeqNo(0),
                Upper(Antichain::from_elem(5))
            ))
        );
        assert_eq!(
            state.snapshot(&Antichain::from_elem(6)),
            Err(SnapshotErr::AsOfNotYetAvailable(
                SeqNo(0),
                Upper(Antichain::from_elem(5))
            ))
        );

        let reader = LeasedReaderId::new();
        // Advance the since to 2.
        let _ = state.collections.register_leased_reader(
            "",
            &reader,
            "",
            SeqNo::minimum(),
            Duration::from_secs(10),
            now(),
            false,
        );
        assert_eq!(
            state.collections.downgrade_since(
                &reader,
                SeqNo::minimum(),
                None,
                &Antichain::from_elem(2),
                now()
            ),
            Continue(Since(Antichain::from_elem(2)))
        );
        assert_eq!(state.collections.trace.since(), &Antichain::from_elem(2));
        // Cannot take a snapshot with as_of < shard_since.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(1)),
            Err(SnapshotErr::AsOfHistoricalDistinctionsLost(Since(
                Antichain::from_elem(2)
            )))
        );

        // Advance the upper to 10 via an empty batch.
        assert!(state
            .collections
            .compare_and_append(
                &hollow(5, 10, &[], 0),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());

        // Can still take snapshots at times < upper.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(7)),
            Ok(vec![hollow(0, 5, &["key1"], 1), hollow(5, 10, &[], 0)])
        );

        // Cannot take snapshots with as_of >= upper.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(10)),
            Err(SnapshotErr::AsOfNotYetAvailable(
                SeqNo(0),
                Upper(Antichain::from_elem(10))
            ))
        );

        // Advance upper to 15.
        assert!(state
            .collections
            .compare_and_append(
                &hollow(10, 15, &["key2"], 1),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());

        // Filter out batches whose lowers are less than the requested as of (the
        // batches that are too far in the future for the requested as_of).
        assert_eq!(
            state.snapshot(&Antichain::from_elem(9)),
            Ok(vec![hollow(0, 5, &["key1"], 1), hollow(5, 10, &[], 0)])
        );

        // Don't filter out batches whose lowers are <= the requested as_of.
        assert_eq!(
            state.snapshot(&Antichain::from_elem(10)),
            Ok(vec![
                hollow(0, 5, &["key1"], 1),
                hollow(5, 10, &[], 0),
                hollow(10, 15, &["key2"], 1)
            ])
        );

        assert_eq!(
            state.snapshot(&Antichain::from_elem(11)),
            Ok(vec![
                hollow(0, 5, &["key1"], 1),
                hollow(5, 10, &[], 0),
                hollow(10, 15, &["key2"], 1)
            ])
        );
    }

    #[mz_ore::test]
    fn next_listen_batch() {
        let mut state = TypedState::<String, String, u64, i64>::new(
            DUMMY_BUILD_INFO.semver_version(),
            ShardId::new(),
            "".to_owned(),
            0,
        );

        // Empty collection never has any batches to listen for, regardless of the
        // current frontier.
        assert_eq!(
            state.next_listen_batch(&Antichain::from_elem(0)),
            Err(SeqNo(0))
        );
        assert_eq!(state.next_listen_batch(&Antichain::new()), Err(SeqNo(0)));

        let writer_id = WriterId::new();
        let now = SYSTEM_TIME.clone();

        // Add two batches of data, one from [0, 5) and then another from [5, 10).
        assert!(state
            .collections
            .compare_and_append(
                &hollow(0, 5, &["key1"], 1),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());
        assert!(state
            .collections
            .compare_and_append(
                &hollow(5, 10, &["key2"], 1),
                &writer_id,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());

        // All frontiers in [0, 5) return the first batch.
        for t in 0..=4 {
            assert_eq!(
                state.next_listen_batch(&Antichain::from_elem(t)),
                Ok(hollow(0, 5, &["key1"], 1))
            );
        }

        // All frontiers in [5, 10) return the second batch.
        for t in 5..=9 {
            assert_eq!(
                state.next_listen_batch(&Antichain::from_elem(t)),
                Ok(hollow(5, 10, &["key2"], 1))
            );
        }

        // There is no batch currently available for t = 10.
        assert_eq!(
            state.next_listen_batch(&Antichain::from_elem(10)),
            Err(SeqNo(0))
        );

        // By definition, there is no frontier ever at the empty antichain which
        // is the time after all possible times.
        assert_eq!(state.next_listen_batch(&Antichain::new()), Err(SeqNo(0)));
    }

    #[mz_ore::test]
    fn expire_writer() {
        let mut state = TypedState::<String, String, u64, i64>::new(
            DUMMY_BUILD_INFO.semver_version(),
            ShardId::new(),
            "".to_owned(),
            0,
        );
        let now = SYSTEM_TIME.clone();

        let writer_id_one = WriterId::new();

        let writer_id_two = WriterId::new();

        // Writer is eligible to write
        assert!(state
            .collections
            .compare_and_append(
                &hollow(0, 2, &["key1"], 1),
                &writer_id_one,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());

        assert!(state
            .collections
            .expire_writer(&writer_id_one)
            .is_continue());

        // Other writers should still be able to write
        assert!(state
            .collections
            .compare_and_append(
                &hollow(2, 5, &["key2"], 1),
                &writer_id_two,
                now(),
                LEASE_DURATION_MS,
                &IdempotencyToken::new(),
                &debug_state(),
                0,
                true,
                100,
                None
            )
            .is_continue());
    }

    #[mz_ore::test]
    fn maybe_gc() {
        let mut state = TypedState::<String, String, u64, i64>::new(
            DUMMY_BUILD_INFO.semver_version(),
            ShardId::new(),
            "".to_owned(),
            0,
        );

        // Empty state doesn't need gc, regardless of is_write.
        assert_eq!(state.maybe_gc(true), None);
        assert_eq!(state.maybe_gc(false), None);

        // Artificially advance the seqno so the seqno_since advances past our
        // internal gc_threshold.
        state.seqno = SeqNo(100);
        assert_eq!(state.seqno_since(), SeqNo(100));

        // When a writer is present, non-writes don't gc.
        let writer_id = WriterId::new();
        let now = SYSTEM_TIME.clone();
        state.collections.compare_and_append(
            &hollow(1, 2, &["key1"], 1),
            &writer_id,
            now(),
            LEASE_DURATION_MS,
            &IdempotencyToken::new(),
            &debug_state(),
            0,
            true,
            100,
            None,
        );
        assert_eq!(state.maybe_gc(false), None);

        // A write will gc though.
        assert_eq!(
            state.maybe_gc(true),
            Some(GcReq {
                shard_id: state.shard_id,
                new_seqno_since: SeqNo(100)
            })
        );

        // Artificially advance the seqno (again) so the seqno_since advances
        // past our internal gc_threshold (again).
        state.seqno = SeqNo(200);
        assert_eq!(state.seqno_since(), SeqNo(200));

        // If there are no writers, even a non-write will gc.
        let _ = state.collections.expire_writer(&writer_id);
        assert_eq!(
            state.maybe_gc(true),
            Some(GcReq {
                shard_id: state.shard_id,
                new_seqno_since: SeqNo(200)
            })
        );
    }

    #[mz_ore::test]
    fn need_rollup() {
        const ROLLUP_THRESHOLD: usize = 3;
        mz_ore::test::init_logging();
        let mut state = TypedState::<String, String, u64, i64>::new(
            DUMMY_BUILD_INFO.semver_version(),
            ShardId::new(),
            "".to_owned(),
            0,
        );

        let rollup_seqno = SeqNo(5);
        let rollup = HollowRollup {
            key: PartialRollupKey::new(rollup_seqno, &RollupId::new()),
            encoded_size_bytes: None,
        };

        assert!(state
            .collections
            .add_rollup((rollup_seqno, &rollup))
            .is_continue());

        // shouldn't need a rollup at the seqno of the rollup
        state.seqno = SeqNo(5);
        assert_none!(state.need_rollup(ROLLUP_THRESHOLD));

        // shouldn't need a rollup at seqnos less than our threshold
        state.seqno = SeqNo(6);
        assert_none!(state.need_rollup(ROLLUP_THRESHOLD));
        state.seqno = SeqNo(7);
        assert_none!(state.need_rollup(ROLLUP_THRESHOLD));

        // hit our threshold! we should need a rollup
        state.seqno = SeqNo(8);
        assert_eq!(
            state.need_rollup(ROLLUP_THRESHOLD).expect("rollup"),
            SeqNo(8)
        );

        // but we don't need rollups for every seqno > the threshold
        state.seqno = SeqNo(9);
        assert_none!(state.need_rollup(ROLLUP_THRESHOLD));

        // we only need a rollup each `ROLLUP_THRESHOLD` beyond our current seqno
        state.seqno = SeqNo(11);
        assert_eq!(
            state.need_rollup(ROLLUP_THRESHOLD).expect("rollup"),
            SeqNo(11)
        );

        // add another rollup and ensure we're always picking the latest
        let rollup_seqno = SeqNo(6);
        let rollup = HollowRollup {
            key: PartialRollupKey::new(rollup_seqno, &RollupId::new()),
            encoded_size_bytes: None,
        };
        assert!(state
            .collections
            .add_rollup((rollup_seqno, &rollup))
            .is_continue());

        state.seqno = SeqNo(8);
        assert_none!(state.need_rollup(ROLLUP_THRESHOLD));
        state.seqno = SeqNo(9);
        assert_eq!(
            state.need_rollup(ROLLUP_THRESHOLD).expect("rollup"),
            SeqNo(9)
        );

        // and ensure that after a fallback point, we assign every seqno work
        let fallback_seqno = SeqNo(
            rollup_seqno.0
                * u64::cast_from(PersistConfig::DEFAULT_FALLBACK_ROLLUP_THRESHOLD_MULTIPLIER),
        );
        state.seqno = fallback_seqno;
        assert_eq!(
            state.need_rollup(ROLLUP_THRESHOLD).expect("rollup"),
            fallback_seqno
        );
        state.seqno = fallback_seqno.next();
        assert_eq!(
            state.need_rollup(ROLLUP_THRESHOLD).expect("rollup"),
            fallback_seqno.next()
        );
    }

    #[mz_ore::test]
    fn idempotency_token_sentinel() {
        assert_eq!(
            IdempotencyToken::SENTINEL.to_string(),
            "i11111111-1111-1111-1111-111111111111"
        );
    }

    /// This test generates an "arbitrary" State, but uses a fixed seed for the
    /// randomness, so that it's deterministic. This lets us assert the
    /// serialization of that State against a golden file that's committed,
    /// making it easy to see what the serialization (used in an upcoming
    /// INSPECT feature) looks like.
    ///
    /// This golden will have to be updated each time we change State, but
    /// that's a feature, not a bug.
    #[mz_ore::test]
    fn state_inspect_serde_json() {
        const STATE_SERDE_JSON: &str = include_str!("state_serde.json");
        let mut runner = proptest::test_runner::TestRunner::deterministic();
        let tree = any_state::<u64>(6..8).new_tree(&mut runner).unwrap();
        let json = serde_json::to_string_pretty(&tree.current()).unwrap();
        assert_eq!(
            json.trim(),
            STATE_SERDE_JSON.trim(),
            "\n\nNEW GOLDEN\n{}\n",
            json
        );
    }

    #[mz_persist_proc::test(tokio::test)]
    #[cfg_attr(miri, ignore)] // too slow
    async fn sneaky_downgrades(dyncfgs: ConfigUpdates) {
        let mut clients = new_test_client_cache(&dyncfgs);
        let shard_id = ShardId::new();

        async fn open_and_write(
            clients: &mut PersistClientCache,
            version: semver::Version,
            shard_id: ShardId,
        ) -> Result<(), tokio::task::JoinError> {
            clients.cfg.build_version = version.clone();
            clients.clear_state_cache();
            let client = clients.open(PersistLocation::new_in_mem()).await.unwrap();
            // Run in a task so we can catch the panic.
            mz_ore::task::spawn(|| version.to_string(), async move {
                let (mut write, _) = client.expect_open::<String, (), u64, i64>(shard_id).await;
                let current = *write.upper().as_option().unwrap();
                // Do a write so that we tag the state with the version.
                write
                    .expect_compare_and_append_batch(&mut [], current, current + 1)
                    .await;
            })
            .await
        }

        // Start at v0.10.0.
        let res = open_and_write(&mut clients, Version::new(0, 10, 0), shard_id).await;
        assert_ok!(res);

        // Upgrade to v0.11.0 is allowed.
        let res = open_and_write(&mut clients, Version::new(0, 11, 0), shard_id).await;
        assert_ok!(res);

        // Downgrade to v0.10.0 is allowed.
        let res = open_and_write(&mut clients, Version::new(0, 10, 0), shard_id).await;
        assert_ok!(res);

        // Downgrade to v0.9.0 is _NOT_ allowed.
        let res = open_and_write(&mut clients, Version::new(0, 9, 0), shard_id).await;
        assert!(res.unwrap_err().is_panic());
    }
}