slatedb 0.5.2

A cloud native embedded storage engine built on object storage.
Documentation
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
//! This module provides the core database functionality for SlateDB.
//! It provides methods for reading and writing to the database, as well as for flushing the database to disk.
//!
//! The `Db` struct represents a database.
//!
//! # Examples
//!
//! Basic usage of the `Db` struct:
//!
//! ```
//! use slatedb::{Db, SlateDBError};
//! use slatedb::object_store::{ObjectStore, memory::InMemory};
//! use std::sync::Arc;
//!
//! #[tokio::main]
//! async fn main() -> Result<(), SlateDBError> {
//!     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
//!     let db = Db::open("test_db", object_store).await?;
//!     Ok(())
//! }
//! ```

use std::collections::HashMap;
use std::ops::RangeBounds;
use std::sync::Arc;

use bytes::Bytes;
use fail_parallel::FailPointRegistry;
use object_store::path::Path;
use object_store::ObjectStore;
use parking_lot::{Mutex, RwLock};
use tokio::runtime::Handle;
use tokio::sync::mpsc::UnboundedSender;

use crate::batch::WriteBatch;
use crate::batch_write::{WriteBatchMsg, WriteBatchRequest};
use crate::bytes_range::BytesRange;
use crate::cached_object_store::stats::CachedObjectStoreStats;
use crate::cached_object_store::CachedObjectStore;
use crate::cached_object_store::FsCacheStorage;
use crate::compactor::Compactor;
use crate::config::{DbOptions, PutOptions, ReadOptions, ScanOptions, WriteOptions};
use crate::db_cache::{DbCache, DbCacheWrapper};
use crate::db_iter::DbIterator;
use crate::db_state::{CoreDbState, DbState, SsTableId};
use crate::db_stats::DbStats;
use crate::error::SlateDBError;
use crate::flush::WalFlushMsg;
use crate::garbage_collector::GarbageCollector;
use crate::manifest::store::{DirtyManifest, FenceableManifest, ManifestStore, StoredManifest};
use crate::mem_table::WritableKVTable;
use crate::mem_table_flush::MemtableFlushMsg;
use crate::paths::PathResolver;
use crate::reader::Reader;
use crate::sst::SsTableFormat;
use crate::sst_iter::SstIteratorOptions;
use crate::stats::StatRegistry;
use crate::tablestore::TableStore;
use crate::utils::{bg_task_result_into_err, MonotonicClock};
use crate::wal_replay::{WalReplayIterator, WalReplayOptions};
use tracing::{info, warn};

pub(crate) struct DbInner {
    pub(crate) state: Arc<RwLock<DbState>>,
    pub(crate) options: DbOptions,
    pub(crate) table_store: Arc<TableStore>,
    pub(crate) wal_flush_notifier: UnboundedSender<WalFlushMsg>,
    pub(crate) memtable_flush_notifier: UnboundedSender<MemtableFlushMsg>,
    pub(crate) write_notifier: UnboundedSender<WriteBatchMsg>,
    pub(crate) db_stats: DbStats,
    pub(crate) stat_registry: Arc<StatRegistry>,
    pub(crate) mono_clock: Arc<MonotonicClock>,
    pub(crate) reader: Reader,
}

impl DbInner {
    pub async fn new(
        options: DbOptions,
        table_store: Arc<TableStore>,
        manifest: DirtyManifest,
        wal_flush_notifier: UnboundedSender<WalFlushMsg>,
        memtable_flush_notifier: UnboundedSender<MemtableFlushMsg>,
        write_notifier: UnboundedSender<WriteBatchMsg>,
        stat_registry: Arc<StatRegistry>,
    ) -> Result<Self, SlateDBError> {
        let mono_clock = Arc::new(MonotonicClock::new(
            options.clock.clone(),
            manifest.core.last_l0_clock_tick,
        ));
        let state = Arc::new(RwLock::new(DbState::new(manifest)));
        let db_stats = DbStats::new(stat_registry.as_ref());

        let reader = Reader {
            table_store: Arc::clone(&table_store),
            db_stats: db_stats.clone(),
            mono_clock: Arc::clone(&mono_clock),
        };

        let db_inner = Self {
            state,
            options,
            table_store,
            wal_flush_notifier,
            memtable_flush_notifier,
            write_notifier,
            db_stats,
            mono_clock,
            stat_registry,
            reader,
        };
        Ok(db_inner)
    }

    /// Get the value for a given key.
    pub async fn get_with_options<K: AsRef<[u8]>>(
        &self,
        key: K,
        options: &ReadOptions,
    ) -> Result<Option<Bytes>, SlateDBError> {
        self.check_error()?;
        let snapshot = self.state.read().snapshot();
        self.reader.get_with_options(key, options, &snapshot).await
    }

    pub async fn scan_with_options<'a>(
        &'a self,
        range: BytesRange,
        options: &ScanOptions,
    ) -> Result<DbIterator<'a>, SlateDBError> {
        self.check_error()?;
        let snapshot = self.state.read().snapshot();
        self.reader
            .scan_with_options(range, options, &snapshot)
            .await
    }

    /// Fences all writers with an older epoch than the provided `manifest` by flushing an empty WAL file that acts
    /// as a barrier. Any parallel old writers will fail with `SlateDBError::Fenced` when trying to "re-write" this file.
    async fn fence_writers(
        &self,
        manifest: &mut FenceableManifest,
        next_wal_id: u64,
    ) -> Result<(), SlateDBError> {
        let mut empty_wal_id = next_wal_id;

        loop {
            let empty_wal = WritableKVTable::new();
            match self
                .flush_imm_table(&SsTableId::Wal(empty_wal_id), empty_wal.table().clone())
                .await
            {
                Ok(_) => {
                    return Ok(());
                }
                Err(SlateDBError::Fenced) => {
                    manifest.refresh().await?;
                    self.state
                        .write()
                        .merge_remote_manifest(manifest.prepare_dirty()?);
                    empty_wal_id += 1;
                }
                Err(e) => {
                    return Err(e);
                }
            }
        }
    }

    pub(crate) fn wal_enabled(&self) -> bool {
        #[cfg(feature = "wal_disable")]
        return self.options.wal_enabled;
        #[cfg(not(feature = "wal_disable"))]
        return true;
    }

    pub async fn write_with_options(
        &self,
        batch: WriteBatch,
        options: &WriteOptions,
    ) -> Result<(), SlateDBError> {
        self.check_error()?;

        if batch.ops.is_empty() {
            return Ok(());
        }

        let (tx, rx) = tokio::sync::oneshot::channel();
        let batch_msg = WriteBatchMsg::WriteBatch(WriteBatchRequest { batch, done: tx });

        self.maybe_apply_backpressure().await?;
        self.write_notifier
            .send(batch_msg)
            .expect("write notifier closed");

        // if the write pipeline task exits then this call to rx.await will fail because tx is dropped
        let current_table = rx.await??;

        if options.await_durable {
            current_table.await_durable().await?;
        }

        Ok(())
    }

    #[inline]
    pub(crate) async fn maybe_apply_backpressure(&self) -> Result<(), SlateDBError> {
        loop {
            let mem_size_bytes = {
                let guard = self.state.read();
                // Exclude active memtable and WAL to avoid a write lock.
                let imm_wal_size = guard
                    .state()
                    .imm_wal
                    .iter()
                    .map(|imm| imm.table().size())
                    .sum::<usize>();
                let imm_memtable_size = guard
                    .state()
                    .imm_memtable
                    .iter()
                    .map(|imm| imm.table().size())
                    .sum::<usize>();
                imm_wal_size + imm_memtable_size
            };
            if mem_size_bytes >= self.options.max_unflushed_bytes {
                let (wal_table, mem_table) = {
                    let guard = self.state.read();
                    (
                        guard.state().imm_wal.back().map(|imm| imm.table().clone()),
                        guard.state().imm_memtable.back().cloned(),
                    )
                };
                tracing::warn!(
                    "Unflushed memtable and WAL size {} >= max_unflushed_bytes {}. Applying backpressure.",
                    mem_size_bytes, self.options.max_unflushed_bytes,
                );

                match (wal_table, mem_table) {
                    (Some(wal_table), Some(mem_table)) => {
                        tokio::select! {
                            result = wal_table.await_durable() => {
                                result?;
                            }
                            result = mem_table.await_flush_to_l0() => {
                                result?;
                            }
                        }
                    }
                    (Some(wal_table), None) => {
                        wal_table.await_durable().await?;
                    }
                    (None, Some(mem_table)) => {
                        mem_table.await_flush_to_l0().await?;
                    }
                    _ => {
                        // No tables to flush, so backpressure is no longer needed.
                        break;
                    }
                }
            } else {
                break;
            }
        }
        Ok(())
    }

    async fn flush_wals(&self) -> Result<(), SlateDBError> {
        let (tx, rx) = tokio::sync::oneshot::channel();
        self.wal_flush_notifier
            .send(WalFlushMsg::FlushImmutableWals { sender: Some(tx) })
            .map_err(|_| SlateDBError::WalFlushChannelError)?;
        rx.await?
    }

    // use to manually flush memtables
    async fn flush_immutable_memtables(&self) -> Result<(), SlateDBError> {
        let (tx, rx) = tokio::sync::oneshot::channel();
        self.memtable_flush_notifier
            .send(MemtableFlushMsg::FlushImmutableMemtables { sender: Some(tx) })
            .map_err(|_| SlateDBError::MemtableFlushChannelError)?;
        rx.await?
    }

    async fn flush_memtables(&self) -> Result<(), SlateDBError> {
        {
            let mut guard = self.state.write();
            if !guard.memtable().is_empty() {
                let last_wal_id = guard.last_written_wal_id();
                guard.freeze_memtable(last_wal_id)?;
            }
        }
        self.flush_immutable_memtables().await
    }

    async fn replay_wal(&self) -> Result<(), SlateDBError> {
        let sst_iter_options = SstIteratorOptions {
            max_fetch_tasks: 1,
            blocks_to_fetch: 256,
            cache_blocks: true,
            eager_spawn: true,
        };

        let replay_options = WalReplayOptions {
            sst_batch_size: 4,
            min_memtable_bytes: self.options.l0_sst_size_bytes,
            max_memtable_bytes: usize::MAX,
            sst_iter_options,
        };

        let db_state = self.state.read().state().core().clone();
        let mut replay_iter =
            WalReplayIterator::new(&db_state, replay_options, Arc::clone(&self.table_store))
                .await?;

        while let Some(replayed_table) = replay_iter.next().await? {
            self.replay_memtable(replayed_table)?;
        }

        Ok(())
    }

    /// Return an error if the state has encountered
    /// an unrecoverable error.
    pub(crate) fn check_error(&self) -> Result<(), SlateDBError> {
        let error_reader = {
            let state = self.state.read();
            state.error_reader()
        };
        if let Some(error) = error_reader.read() {
            return Err(error.clone());
        }
        Ok(())
    }
}

pub struct Db {
    pub(crate) inner: Arc<DbInner>,
    /// The handle for the flush thread.
    wal_flush_task: Mutex<Option<tokio::task::JoinHandle<Result<(), SlateDBError>>>>,
    memtable_flush_task: Mutex<Option<tokio::task::JoinHandle<Result<(), SlateDBError>>>>,
    write_task: Mutex<Option<tokio::task::JoinHandle<Result<(), SlateDBError>>>>,
    compactor: Mutex<Option<Compactor>>,
    garbage_collector: Mutex<Option<GarbageCollector>>,
}

impl Db {
    /// Open a new database with default options.
    ///
    /// ## Arguments
    /// - `path`: the path to the database
    /// - `object_store`: the object store to use for the database
    ///
    /// ## Returns
    /// - `Db`: the database
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error opening the database
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn open<P: Into<Path>>(
        path: P,
        object_store: Arc<dyn ObjectStore>,
    ) -> Result<Self, SlateDBError> {
        Self::open_with_opts(path, DbOptions::default(), object_store).await
    }

    /// Open a new database with custom `DbOptions`.
    ///
    /// ## Arguments
    /// - `path`: the path to the database
    /// - `options`: the options to use for the database
    /// - `object_store`: the object store to use for the database
    ///
    /// ## Returns
    /// - `Db`: the database
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error opening the database
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, config::DbOptions, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open_with_opts("test_db", DbOptions::default(), object_store).await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn open_with_opts<P: Into<Path>>(
        path: P,
        options: DbOptions,
        object_store: Arc<dyn ObjectStore>,
    ) -> Result<Self, SlateDBError> {
        Self::open_with_fp_registry(
            path,
            options,
            object_store,
            Arc::new(FailPointRegistry::new()),
        )
        .await
    }

    /// Open a new database with a custom `FailPointRegistry`.
    ///
    /// ## Arguments
    /// - `path`: the path to the database
    /// - `options`: the options to use for the database
    /// - `object_store`: the object store to use for the database
    /// - `fp_registry`: the failpoint registry to use for the database
    ///
    /// ## Returns
    /// - `Db`: the database
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error opening the database
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, config::DbOptions, SlateDBError};
    /// use slatedb::fail_parallel::FailPointRegistry;
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let fp_registry = Arc::new(FailPointRegistry::new());
    ///     let db = Db::open_with_fp_registry("test_db", DbOptions::default(), object_store, fp_registry).await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn open_with_fp_registry<P: Into<Path>>(
        path: P,
        options: DbOptions,
        object_store: Arc<dyn ObjectStore>,
        fp_registry: Arc<FailPointRegistry>,
    ) -> Result<Self, SlateDBError> {
        let path = path.into();
        if let Ok(options_json) = options.to_json_string() {
            tracing::info!(?path, options = options_json, "Opening SlateDB database");
        } else {
            tracing::info!(?path, ?options, "Opening SlateDB database");
        }

        let stat_registry = Arc::new(StatRegistry::new());
        let sst_format = SsTableFormat {
            min_filter_keys: options.min_filter_keys,
            filter_bits_per_key: options.filter_bits_per_key,
            compression_codec: options.compression_codec,
            ..SsTableFormat::default()
        };
        let maybe_cached_object_store = match &options.object_store_cache_options.root_folder {
            None => object_store.clone(),
            Some(cache_root_folder) => {
                let stats = Arc::new(CachedObjectStoreStats::new(stat_registry.as_ref()));
                let cache_storage = Arc::new(FsCacheStorage::new(
                    cache_root_folder.clone(),
                    options.object_store_cache_options.max_cache_size_bytes,
                    options.object_store_cache_options.scan_interval,
                    stats.clone(),
                ));

                let cached_object_store = CachedObjectStore::new(
                    object_store.clone(),
                    cache_storage,
                    options.object_store_cache_options.part_size_bytes,
                    stats.clone(),
                )?;
                cached_object_store.start_evictor().await;
                cached_object_store
            }
        };

        let manifest_store = Arc::new(ManifestStore::new(&path, maybe_cached_object_store.clone()));
        let latest_manifest = StoredManifest::try_load(manifest_store.clone()).await?;

        let external_ssts = match &latest_manifest {
            Some(latest_stored_manifest) => {
                let mut external_ssts = HashMap::new();
                for external_db in &latest_stored_manifest.manifest().external_dbs {
                    for id in &external_db.sst_ids {
                        external_ssts.insert(*id, external_db.path.clone().into());
                    }
                }
                external_ssts
            }
            None => HashMap::new(),
        };

        let path_resolver = PathResolver::new_with_external_ssts(path.clone(), external_ssts);
        let table_store = Arc::new(TableStore::new_with_fp_registry(
            maybe_cached_object_store.clone(),
            sst_format.clone(),
            path_resolver.clone(),
            fp_registry.clone(),
            options.block_cache.as_ref().map(|c| {
                Arc::new(DbCacheWrapper::new(c.clone(), stat_registry.as_ref())) as Arc<dyn DbCache>
            }),
        ));

        // get the next wal id before writing manifest.
        let wal_id_last_compacted = match &latest_manifest {
            Some(latest_stored_manifest) => {
                latest_stored_manifest.db_state().last_compacted_wal_sst_id
            }
            None => 0,
        };
        let next_wal_id = table_store.next_wal_sst_id(wal_id_last_compacted).await?;

        let mut manifest = Self::init_db(&manifest_store, latest_manifest).await?;
        let (memtable_flush_tx, memtable_flush_rx) = tokio::sync::mpsc::unbounded_channel();
        let (wal_flush_tx, wal_flush_rx) = tokio::sync::mpsc::unbounded_channel();
        let (write_tx, write_rx) = tokio::sync::mpsc::unbounded_channel();
        let inner = Arc::new(
            DbInner::new(
                options.clone(),
                table_store.clone(),
                manifest.prepare_dirty()?,
                wal_flush_tx,
                memtable_flush_tx,
                write_tx,
                stat_registry,
            )
            .await?,
        );
        if inner.wal_enabled() {
            inner.fence_writers(&mut manifest, next_wal_id).await?;
        }
        inner.replay_wal().await?;
        let tokio_handle = Handle::current();
        let flush_task = if inner.wal_enabled() {
            Some(inner.spawn_flush_task(wal_flush_rx, &tokio_handle))
        } else {
            None
        };
        let memtable_flush_task =
            inner.spawn_memtable_flush_task(manifest, memtable_flush_rx, &tokio_handle);
        let write_task = inner.spawn_write_task(write_rx, &tokio_handle);
        let mut compactor = None;
        if let Some(compactor_options) = &inner.options.compactor_options {
            // not to pollute the cache during compaction
            let uncached_table_store = Arc::new(TableStore::new_with_fp_registry(
                object_store.clone(),
                sst_format,
                path_resolver,
                fp_registry.clone(),
                None,
            ));
            let cleanup_inner = inner.clone();
            compactor = Some(
                Compactor::new(
                    manifest_store.clone(),
                    uncached_table_store.clone(),
                    compactor_options.clone(),
                    Handle::current(),
                    inner.stat_registry.as_ref(),
                    move |result: &Result<(), SlateDBError>| {
                        let err = bg_task_result_into_err(result);
                        warn!("compactor thread exited with {:?}", err);
                        let mut state = cleanup_inner.state.write();
                        state.record_fatal_error(err.clone())
                    },
                )
                .await?,
            )
        }
        let mut garbage_collector = None;
        if let Some(gc_options) = &inner.options.garbage_collector_options {
            let cleanup_inner = inner.clone();
            garbage_collector = Some(
                GarbageCollector::new(
                    manifest_store.clone(),
                    table_store.clone(),
                    gc_options.clone(),
                    Handle::current(),
                    inner.stat_registry.clone(),
                    move |result| {
                        let err = bg_task_result_into_err(result);
                        warn!("GC thread exited with {:?}", err);
                        let mut state = cleanup_inner.state.write();
                        state.record_fatal_error(err.clone())
                    },
                )
                .await,
            )
        };
        Ok(Self {
            inner,
            wal_flush_task: Mutex::new(flush_task),
            memtable_flush_task: Mutex::new(memtable_flush_task),
            write_task: Mutex::new(write_task),
            compactor: Mutex::new(compactor),
            garbage_collector: Mutex::new(garbage_collector),
        })
    }

    async fn init_db(
        manifest_store: &Arc<ManifestStore>,
        latest_stored_manifest: Option<StoredManifest>,
    ) -> Result<FenceableManifest, SlateDBError> {
        let stored_manifest = match latest_stored_manifest {
            Some(manifest) => manifest,
            None => {
                StoredManifest::create_new_db(manifest_store.clone(), CoreDbState::new()).await?
            }
        };
        FenceableManifest::init_writer(stored_manifest).await
    }

    /// Close the database.
    ///
    /// ## Returns
    /// - `Result<(), SlateDBError>`: if there was an error closing the database
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.close().await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn close(&self) -> Result<(), SlateDBError> {
        if let Some(compactor) = {
            let mut maybe_compactor = self.compactor.lock();
            maybe_compactor.take()
        } {
            compactor.close().await;
        }

        if let Some(gc) = {
            let mut maybe_gc = self.garbage_collector.lock();
            maybe_gc.take()
        } {
            gc.close().await;
        }

        // Shutdown the write batch thread.
        self.inner.write_notifier.send(WriteBatchMsg::Shutdown).ok();

        if let Some(write_task) = {
            let mut write_task = self.write_task.lock();
            write_task.take()
        } {
            let result = write_task.await.expect("Failed to join write thread");
            info!("write task exited with {:?}", result);
        }

        // Shutdown the WAL flush thread.
        self.inner
            .wal_flush_notifier
            .send(WalFlushMsg::Shutdown)
            .ok();

        if let Some(flush_task) = {
            let mut flush_task = self.wal_flush_task.lock();
            flush_task.take()
        } {
            let result = flush_task.await.expect("Failed to join flush thread");
            info!("flush task exited with {:?}", result);
        }

        // Shutdown the memtable flush thread.
        self.inner
            .memtable_flush_notifier
            .send(MemtableFlushMsg::Shutdown)
            .ok();

        if let Some(memtable_flush_task) = {
            let mut memtable_flush_task = self.memtable_flush_task.lock();
            memtable_flush_task.take()
        } {
            let result = memtable_flush_task
                .await
                .expect("Failed to join memtable flush thread");
            info!("mem table flush task exited with: {:?}", result);
        }

        Ok(())
    }

    /// Get a value from the database with default read options.
    ///
    /// The `Bytes` object returned contains a slice of an entire
    /// 4 KiB block. The block will be held in memory as long as the
    /// caller holds a reference to the `Bytes` object. Consider
    /// copying the data if you need to hold it for a long time.
    ///
    /// ## Arguments
    /// - `key`: the key to get
    ///
    /// ## Returns
    /// - `Result<Option<Bytes>, SlateDBError>`:
    ///     - `Some(Bytes)`: the value if it exists
    ///     - `None`: if the value does not exist
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error getting the value
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.put(b"key", b"value").await?;
    ///     assert_eq!(db.get(b"key").await?, Some("value".into()));
    ///     Ok(())
    /// }
    /// ```
    pub async fn get<K: AsRef<[u8]> + Send>(&self, key: K) -> Result<Option<Bytes>, SlateDBError> {
        self.get_with_options(key, &ReadOptions::default()).await
    }

    /// Get a value from the database with custom read options.
    ///
    /// The `Bytes` object returned contains a slice of an entire
    /// 4 KiB block. The block will be held in memory as long as the
    /// caller holds a reference to the `Bytes` object. Consider
    /// copying the data if you need to hold it for a long time.
    ///
    /// ## Arguments
    /// - `key`: the key to get
    /// - `options`: the read options to use (Note that [`ReadOptions::read_level`] has no effect for readers, which
    ///    can only observe committed state).
    ///
    /// ## Returns
    /// - `Result<Option<Bytes>, SlateDBError>`:
    ///     - `Some(Bytes)`: the value if it exists
    ///     - `None`: if the value does not exist
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error getting the value
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, config::ReadOptions, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.put(b"key", b"value").await?;
    ///     assert_eq!(db.get_with_options(b"key", &ReadOptions::default()).await?, Some("value".into()));
    ///     Ok(())
    /// }
    /// ```
    pub async fn get_with_options<K: AsRef<[u8]> + Send>(
        &self,
        key: K,
        options: &ReadOptions,
    ) -> Result<Option<Bytes>, SlateDBError> {
        self.inner.get_with_options(key, options).await
    }

    /// Scan a range of keys using the default scan options.
    ///
    /// returns a `DbIterator`
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error scanning the range of keys
    ///
    /// ## Returns
    /// - `Result<DbIterator, SlateDBError>`: An iterator with the results of the scan
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.put(b"a", b"a_value").await?;
    ///     db.put(b"b", b"b_value").await?;
    ///
    ///     let mut iter = db.scan("a".."b").await?;
    ///     assert_eq!(Some((b"a", b"a_value").into()), iter.next().await?);
    ///     assert_eq!(None, iter.next().await?);
    ///     Ok(())
    /// }
    /// ```
    pub async fn scan<K, T>(&self, range: T) -> Result<DbIterator, SlateDBError>
    where
        K: AsRef<[u8]> + Send,
        T: RangeBounds<K> + Send,
    {
        self.scan_with_options(range, &ScanOptions::default()).await
    }

    /// Scan a range of keys with the provided options.
    ///
    /// returns a `DbIterator`
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error scanning the range of keys
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, config::ScanOptions, config::ReadLevel, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.put(b"a", b"a_value").await?;
    ///     db.put(b"b", b"b_value").await?;
    ///
    ///     let mut iter = db.scan_with_options("a".."b", &ScanOptions {
    ///         read_level: ReadLevel::Uncommitted,
    ///         ..ScanOptions::default()
    ///     }).await?;
    ///     assert_eq!(Some((b"a", b"a_value").into()), iter.next().await?);
    ///     assert_eq!(None, iter.next().await?);
    ///     Ok(())
    /// }
    /// ```
    pub async fn scan_with_options<K, T>(
        &self,
        range: T,
        options: &ScanOptions,
    ) -> Result<DbIterator, SlateDBError>
    where
        K: AsRef<[u8]> + Send,
        T: RangeBounds<K> + Send,
    {
        let start = range
            .start_bound()
            .map(|b| Bytes::copy_from_slice(b.as_ref()));
        let end = range
            .end_bound()
            .map(|b| Bytes::copy_from_slice(b.as_ref()));
        let range = (start, end);
        self.inner
            .scan_with_options(BytesRange::from(range), options)
            .await
    }

    /// Write a value into the database with default `WriteOptions`.
    ///
    /// ## Arguments
    /// - `key`: the key to write
    /// - `value`: the value to write
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error writing the value.
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.put(b"key", b"value").await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn put<K, V>(&self, key: K, value: V) -> Result<(), SlateDBError>
    where
        K: AsRef<[u8]>,
        V: AsRef<[u8]>,
    {
        let mut batch = WriteBatch::new();
        batch.put(key, value);
        self.write(batch).await
    }

    /// Write a value into the database with custom `PutOptions` and `WriteOptions`.
    ///
    /// ## Arguments
    /// - `key`: the key to write
    /// - `value`: the value to write
    /// - `put_opts`: the put options to use
    /// - `write_opts`: the write options to use
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error writing the value.
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, config::{PutOptions, WriteOptions}, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.put_with_options(b"key", b"value", &PutOptions::default(), &WriteOptions::default()).await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn put_with_options<K, V>(
        &self,
        key: K,
        value: V,
        put_opts: &PutOptions,
        write_opts: &WriteOptions,
    ) -> Result<(), SlateDBError>
    where
        K: AsRef<[u8]>,
        V: AsRef<[u8]>,
    {
        let mut batch = WriteBatch::new();
        batch.put_with_options(key, value, put_opts);
        self.write_with_options(batch, write_opts).await
    }

    /// Delete a key from the database with default `WriteOptions`.
    ///
    /// ## Arguments
    /// - `key`: the key to delete
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error deleting the key.
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.delete(b"key").await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn delete<K: AsRef<[u8]>>(&self, key: K) -> Result<(), SlateDBError> {
        let mut batch = WriteBatch::new();
        batch.delete(key.as_ref());
        self.write(batch).await
    }

    /// Delete a key from the database with custom `WriteOptions`.
    ///
    /// ## Arguments
    /// - `key`: the key to delete
    /// - `options`: the write options to use
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error deleting the key.
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, config::WriteOptions, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.delete_with_options(b"key", &WriteOptions::default()).await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn delete_with_options<K: AsRef<[u8]>>(
        &self,
        key: K,
        options: &WriteOptions,
    ) -> Result<(), SlateDBError> {
        let mut batch = WriteBatch::new();
        batch.delete(key);
        self.write_with_options(batch, options).await
    }

    /// Write a batch of put/delete operations atomically to the database. Batch writes
    /// block other gets and writes until the batch is written to the WAL (or memtable if
    /// WAL is disabled).
    ///
    /// ## Arguments
    /// - `batch`: the batch of put/delete operations to write
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error writing the batch.
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{WriteBatch, Db, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///
    ///     let mut batch = WriteBatch::new();
    ///     batch.put(b"key1", b"value1");
    ///     batch.put(b"key2", b"value2");
    ///     batch.delete(b"key1");
    ///     db.write(batch).await?;
    ///
    ///     Ok(())
    /// }
    /// ```
    pub async fn write(&self, batch: WriteBatch) -> Result<(), SlateDBError> {
        self.write_with_options(batch, &WriteOptions::default())
            .await
    }

    /// Write a batch of put/delete operations atomically to the database. Batch writes
    /// block other gets and writes until the batch is written to the WAL (or memtable if
    /// WAL is disabled).
    ///
    /// ## Arguments
    /// - `batch`: the batch of put/delete operations to write
    /// - `options`: the write options to use
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error writing the batch.
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{WriteBatch, Db, config::WriteOptions, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///
    ///     let mut batch = WriteBatch::new();
    ///     batch.put(b"key1", b"value1");
    ///     batch.put(b"key2", b"value2");
    ///     batch.delete(b"key1");
    ///     db.write_with_options(batch, &WriteOptions::default()).await?;
    ///
    ///     Ok(())
    /// }
    /// ```
    pub async fn write_with_options(
        &self,
        batch: WriteBatch,
        options: &WriteOptions,
    ) -> Result<(), SlateDBError> {
        self.inner.write_with_options(batch, options).await
    }

    /// Flush the database to disk.
    /// If WAL is enabled, flushes the WAL to disk.
    /// If WAL is disabled, flushes the memtables to disk.
    ///
    /// ## Errors
    /// - `SlateDBError`: if there was an error flushing the database
    ///
    /// ## Examples
    ///
    /// ```
    /// use slatedb::{Db, SlateDBError};
    /// use slatedb::object_store::{ObjectStore, memory::InMemory};
    /// use std::sync::Arc;
    ///
    /// #[tokio::main]
    /// async fn main() -> Result<(), SlateDBError> {
    ///     let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
    ///     let db = Db::open("test_db", object_store).await?;
    ///     db.flush().await?;
    ///     Ok(())
    /// }
    /// ```
    pub async fn flush(&self) -> Result<(), SlateDBError> {
        if self.inner.wal_enabled() {
            self.inner.flush_wals().await
        } else {
            self.inner.flush_memtables().await
        }
    }

    pub(crate) async fn await_flush(&self) -> Result<(), SlateDBError> {
        let table = {
            let guard = self.inner.state.read();
            let snapshot = guard.snapshot();
            if self.inner.wal_enabled() {
                snapshot.wal.clone()
            } else {
                snapshot.memtable.clone()
            }
        };
        if table.is_empty() {
            return Ok(());
        }
        table.await_durable().await
    }

    pub fn metrics(&self) -> Arc<StatRegistry> {
        self.inner.stat_registry.clone()
    }
}

#[cfg(test)]
mod tests {
    use std::collections::BTreeMap;
    use std::collections::Bound::Included;
    use std::sync::atomic::Ordering;
    use std::time::Duration;

    use super::*;
    use crate::cached_object_store::stats::{
        OBJECT_STORE_CACHE_PART_ACCESS, OBJECT_STORE_CACHE_PART_HITS,
    };
    use crate::cached_object_store::FsCacheStorage;
    use crate::config::ReadLevel::{Committed, Uncommitted};
    use crate::config::{
        CompactorOptions, ObjectStoreCacheOptions, SizeTieredCompactionSchedulerOptions, Ttl,
    };
    use crate::db_stats::IMMUTABLE_MEMTABLE_FLUSHES;
    use crate::iter::KeyValueIterator;
    use crate::proptest_util::arbitrary;
    use crate::proptest_util::sample;
    use crate::size_tiered_compaction::SizeTieredCompactionSchedulerSupplier;
    use crate::sst_iter::{SstIterator, SstIteratorOptions};
    use crate::test_utils::{assert_iterator, TestClock};
    use crate::types::RowEntry;
    use crate::{proptest_util, test_utils};
    use futures::{future::join_all, StreamExt};
    use object_store::memory::InMemory;
    use object_store::ObjectStore;
    use proptest::test_runner::{TestRng, TestRunner};
    use tokio::runtime::Runtime;
    use tracing::info;

    #[tokio::test]
    async fn test_put_get_delete() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            test_db_options(0, 1024, None),
            object_store,
        )
        .await
        .unwrap();
        let key = b"test_key";
        let value = b"test_value";
        kv_store.put(key, value).await.unwrap();
        kv_store.flush().await.unwrap();

        assert_eq!(
            kv_store.get(key).await.unwrap(),
            Some(Bytes::from_static(value))
        );
        kv_store.delete(key).await.unwrap();
        assert_eq!(None, kv_store.get(key).await.unwrap());
        kv_store.close().await.unwrap();
    }

    #[tokio::test]
    async fn test_get_with_default_ttl_and_read_uncommitted() {
        let clock = Arc::new(TestClock::new());
        let ttl = 100;

        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            test_db_options_with_ttl(0, 1024, None, clock.clone(), Some(ttl)),
            object_store,
        )
        .await
        .unwrap();

        let key = b"test_key";
        let value = b"test_value";

        // insert at t=0
        kv_store.put(key, value).await.unwrap();

        // advance clock to t=99 --> still returned
        clock.ticker.store(99, Ordering::SeqCst);
        assert_eq!(
            Some(Bytes::from_static(value)),
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Uncommitted
                    }
                )
                .await
                .unwrap(),
        );

        // advance clock to t=100 --> no longer returned
        clock.ticker.store(100, Ordering::SeqCst);
        assert_eq!(
            None,
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Uncommitted
                    }
                )
                .await
                .unwrap(),
        );

        kv_store.close().await.unwrap();
    }

    #[tokio::test]
    async fn test_get_with_row_override_ttl_and_read_uncommitted() {
        let clock = Arc::new(TestClock::new());
        let default_ttl = 100;

        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            test_db_options_with_ttl(0, 1024, None, clock.clone(), Some(default_ttl)),
            object_store,
        )
        .await
        .unwrap();

        let key = b"test_key";
        let value = b"test_value";

        // insert at t=0 with row-level override of 50 for ttl
        kv_store
            .put_with_options(
                key,
                value,
                &PutOptions {
                    ttl: Ttl::ExpireAfter(50),
                },
                &WriteOptions::default(),
            )
            .await
            .unwrap();

        // advance clock to t=49 --> still returned
        clock.ticker.store(49, Ordering::SeqCst);
        assert_eq!(
            Some(Bytes::from_static(value)),
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Uncommitted
                    }
                )
                .await
                .unwrap(),
        );

        // advance clock to t=50 --> no longer returned
        clock.ticker.store(50, Ordering::SeqCst);
        assert_eq!(
            None,
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Uncommitted
                    }
                )
                .await
                .unwrap(),
        );

        kv_store.close().await.unwrap();
    }

    #[tokio::test]
    async fn test_get_with_default_ttl_and_read_committed() {
        let clock = Arc::new(TestClock::new());
        let ttl = 100;

        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            test_db_options_with_ttl(0, 1024, None, clock.clone(), Some(ttl)),
            object_store,
        )
        .await
        .unwrap();

        let key = b"test_key";
        let key_other = b"time_advancing_key";
        let value = b"test_value";

        // insert at t=0
        kv_store.put(key, value).await.unwrap();

        // advance clock to t=99 --> still returned
        clock.ticker.store(99, Ordering::SeqCst);
        kv_store.put(key_other, value).await.unwrap(); // fake data to advance clock
        kv_store.flush().await.unwrap();
        assert_eq!(
            Some(Bytes::from_static(value)),
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Committed
                    }
                )
                .await
                .unwrap(),
        );

        // advance clock to t=100 without flushing --> still returned
        clock.ticker.store(100, Ordering::SeqCst);
        assert_eq!(
            Some(Bytes::from_static(value)),
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Committed
                    }
                )
                .await
                .unwrap(),
        );

        // advance durable clock time to t=100 by flushing -- no longer returned
        kv_store.put(key_other, value).await.unwrap(); // fake data to advance clock
        kv_store.flush().await.unwrap();
        assert_eq!(
            None,
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Committed
                    }
                )
                .await
                .unwrap(),
        );

        kv_store.close().await.unwrap();
    }

    #[tokio::test]
    #[cfg(feature = "wal_disable")]
    async fn test_find_with_multiple_repeated_keys() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let mut options = test_db_options(0, 1024 * 1024, None);
        options.wal_enabled = false;
        let db = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            options.clone(),
            object_store,
        )
        .await
        .unwrap();

        // write enough rows with the same key that we yield an L0 SST with multiple blocks
        let mut last_val: String = "foo".to_string();
        for x in 0..4096 {
            let val = format!("val{}", x);
            db.put_with_options(
                b"key",
                val.as_bytes(),
                &PutOptions {
                    ttl: Default::default(),
                },
                &WriteOptions {
                    await_durable: false,
                },
            )
            .await
            .unwrap();
            last_val = val;
            if db.inner.state.write().memtable().size() > (SsTableFormat::default().block_size * 3)
            {
                break;
            }
        }
        assert_eq!(
            Some(Bytes::copy_from_slice(last_val.as_bytes())),
            db.get(b"key").await.unwrap()
        );
        db.flush().await.unwrap();

        let state = db.inner.state.read().snapshot();
        assert_eq!(1, state.state.manifest.core.l0.len());
        let sst = state.state.manifest.core.l0.front().unwrap();
        let index = db.inner.table_store.read_index(sst).await.unwrap();
        assert!(index.borrow().block_meta().len() >= 3);
        assert_eq!(
            Some(Bytes::copy_from_slice(last_val.as_bytes())),
            db.get(b"key").await.unwrap()
        );
        db.close().await.unwrap();
    }

    #[tokio::test]
    async fn test_get_with_row_override_ttl_and_read_committed() {
        let clock = Arc::new(TestClock::new());
        let ttl = 100;

        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            test_db_options_with_ttl(0, 1024, None, clock.clone(), Some(ttl)),
            object_store,
        )
        .await
        .unwrap();

        let key = b"test_key";
        let key_other = b"time_advancing_key";
        let value = b"test_value";

        // insert at t=0 with row-level override of 50 for ttl
        kv_store
            .put_with_options(
                key,
                value,
                &PutOptions {
                    ttl: Ttl::ExpireAfter(50),
                },
                &WriteOptions::default(),
            )
            .await
            .unwrap();

        // advance clock to t=49 --> still returned
        clock.ticker.store(49, Ordering::SeqCst);
        kv_store.put(key_other, value).await.unwrap(); // fake data to advance clock
        kv_store.flush().await.unwrap();
        assert_eq!(
            Some(Bytes::from_static(value)),
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Committed
                    }
                )
                .await
                .unwrap(),
        );

        // advance clock to t=50 without flushing --> still returned
        clock.ticker.store(50, Ordering::SeqCst);
        assert_eq!(
            Some(Bytes::from_static(value)),
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Committed
                    }
                )
                .await
                .unwrap(),
        );

        // advance durable clock time to t=100 by flushing -- no longer returned
        kv_store.put(key_other, value).await.unwrap(); // fake data to advance clock
        kv_store.flush().await.unwrap();
        assert_eq!(
            None,
            kv_store
                .get_with_options(
                    key,
                    &ReadOptions {
                        read_level: Committed
                    }
                )
                .await
                .unwrap(),
        );

        kv_store.close().await.unwrap();
    }

    #[tokio::test]
    async fn test_get_with_object_store_cache_metrics() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let mut opts = test_db_options(0, 1024, None);
        let temp_dir = tempfile::Builder::new()
            .prefix("objstore_cache_test_")
            .tempdir()
            .unwrap();

        opts.object_store_cache_options.root_folder = Some(temp_dir.into_path());
        opts.object_store_cache_options.part_size_bytes = 1024;
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store_with_cache_metrics"),
            opts,
            object_store.clone(),
        )
        .await
        .unwrap();

        let access_count0 = kv_store
            .metrics()
            .lookup(OBJECT_STORE_CACHE_PART_ACCESS)
            .unwrap()
            .get();
        let key = b"test_key";
        let value = b"test_value";
        kv_store.put(key, value).await.unwrap();
        kv_store.flush().await.unwrap();

        let got = kv_store.get(key).await.unwrap();
        let access_count1 = kv_store
            .metrics()
            .lookup(OBJECT_STORE_CACHE_PART_ACCESS)
            .unwrap()
            .get();
        assert_eq!(got, Some(Bytes::from_static(value)));
        assert!(access_count1 > 0);
        assert!(access_count1 >= access_count0);
        assert!(
            kv_store
                .metrics()
                .lookup(OBJECT_STORE_CACHE_PART_HITS)
                .unwrap()
                .get()
                >= 1
        );
    }

    #[tokio::test]
    async fn test_get_with_object_store_cache_stored_files() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let mut opts = test_db_options(0, 1024, None);
        let temp_dir = tempfile::Builder::new()
            .prefix("objstore_cache_test_")
            .tempdir()
            .unwrap();
        let stats_registry = StatRegistry::new();
        let cache_stats = Arc::new(CachedObjectStoreStats::new(&stats_registry));
        let part_size = 1024;
        let cache_storage = Arc::new(FsCacheStorage::new(
            temp_dir.path().to_path_buf(),
            None,
            None,
            cache_stats.clone(),
        ));

        let cached_object_store = CachedObjectStore::new(
            object_store.clone(),
            cache_storage,
            part_size,
            cache_stats.clone(),
        )
        .unwrap();

        opts.object_store_cache_options.root_folder = Some(temp_dir.into_path());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store_with_cache_stored_files"),
            opts,
            cached_object_store.clone(),
        )
        .await
        .unwrap();
        let key = b"test_key";
        let value = b"test_value";
        kv_store.put(key, value).await.unwrap();
        kv_store.flush().await.unwrap();

        assert_eq!(
            cached_object_store
                .list(None)
                .collect::<Vec<_>>()
                .await
                .into_iter()
                .filter_map(Result::ok)
                .map(|meta| meta.location.to_string())
                .collect::<Vec<_>>(),
            vec![
                "tmp/test_kv_store_with_cache_stored_files/manifest/00000000000000000001.manifest"
                    .to_string(),
                "tmp/test_kv_store_with_cache_stored_files/manifest/00000000000000000002.manifest"
                    .to_string(),
                "tmp/test_kv_store_with_cache_stored_files/wal/00000000000000000001.sst"
                    .to_string(),
                "tmp/test_kv_store_with_cache_stored_files/wal/00000000000000000002.sst"
                    .to_string(),
            ],
        );

        // check the files are cached as expected
        let tests = vec![
            (
                "tmp/test_kv_store_with_cache_stored_files/manifest/00000000000000000001.manifest",
                0,
            ),
            (
                "tmp/test_kv_store_with_cache_stored_files/manifest/00000000000000000002.manifest",
                0,
            ),
            (
                "tmp/test_kv_store_with_cache_stored_files/wal/00000000000000000001.sst",
                2,
            ),
            (
                "tmp/test_kv_store_with_cache_stored_files/wal/00000000000000000002.sst",
                0,
            ),
        ];
        for (path, expected) in tests {
            let entry = cached_object_store
                .cache_storage
                .entry(&object_store::path::Path::from(path), part_size);
            assert_eq!(
                entry.cached_parts().await.unwrap().len(),
                expected,
                "{}",
                path
            );
        }
    }

    async fn build_database_from_table(
        table: &BTreeMap<Bytes, Bytes>,
        db_options: DbOptions,
        await_durable: bool,
    ) -> Db {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let db = Db::open_with_opts(Path::from("/tmp/test_kv_store"), db_options, object_store)
            .await
            .unwrap();

        test_utils::seed_database(&db, table, false).await.unwrap();

        if await_durable {
            db.flush().await.unwrap();
        }

        db
    }

    async fn assert_empty_scan(db: &Db, range: BytesRange) {
        let mut iter = db
            .inner
            .scan_with_options(range.clone(), &ScanOptions::default())
            .await
            .unwrap();
        assert_eq!(None, iter.next().await.unwrap());
    }

    #[test]
    fn test_empty_scan_range_returns_empty_iterator() {
        let mut runner = new_proptest_runner(None);
        let table = sample::table(runner.rng(), 1000, 5);

        let runtime = Runtime::new().unwrap();
        let db_options = test_db_options(0, 1024, None);
        let db = runtime.block_on(build_database_from_table(&table, db_options, true));

        runner
            .run(&arbitrary::empty_range(10), |range| {
                runtime.block_on(assert_empty_scan(&db, range));
                Ok(())
            })
            .unwrap();
    }

    async fn assert_records_in_range(
        table: &BTreeMap<Bytes, Bytes>,
        db: &Db,
        scan_options: &ScanOptions,
        range: BytesRange,
    ) {
        let mut iter = db
            .inner
            .scan_with_options(range.clone(), scan_options)
            .await
            .unwrap();
        test_utils::assert_ranged_db_scan(table, range, &mut iter).await;
    }

    #[test]
    fn test_scan_returns_records_in_range() {
        let mut runner = new_proptest_runner(None);
        let table = sample::table(runner.rng(), 1000, 5);

        let runtime = Runtime::new().unwrap();
        let db_options = test_db_options(0, 1024, None);
        let db = runtime.block_on(build_database_from_table(&table, db_options, true));

        runner
            .run(&arbitrary::nonempty_range(10), |range| {
                runtime.block_on(assert_records_in_range(
                    &table,
                    &db,
                    &ScanOptions::default(),
                    range,
                ));
                Ok(())
            })
            .unwrap();
    }

    fn new_proptest_runner(rng_seed: Option<[u8; 32]>) -> TestRunner {
        proptest_util::runner::new(file!(), rng_seed)
    }

    #[test]
    fn test_scan_returns_uncommitted_records_if_read_level_uncommitted() {
        let mut runner = new_proptest_runner(None);
        let table = sample::table(runner.rng(), 1000, 5);

        let runtime = Runtime::new().unwrap();
        let mut db_options = test_db_options(0, 1024, None);
        db_options.flush_interval = Some(Duration::from_secs(5));
        let db = runtime.block_on(build_database_from_table(&table, db_options, false));

        runner
            .run(&arbitrary::nonempty_range(10), |range| {
                let scan_options = ScanOptions {
                    read_level: Uncommitted,
                    ..ScanOptions::default()
                };
                runtime.block_on(assert_records_in_range(&table, &db, &scan_options, range));
                Ok(())
            })
            .unwrap();
    }

    #[test]
    fn test_seek_outside_of_range_returns_invalid_argument() {
        let mut runner = new_proptest_runner(None);
        let table = sample::table(runner.rng(), 1000, 10);

        let runtime = Runtime::new().unwrap();
        let db_options = test_db_options(0, 1024, None);
        let db = runtime.block_on(build_database_from_table(&table, db_options, true));

        runner
            .run(
                &(arbitrary::nonempty_bytes(10), arbitrary::rng()),
                |(arbitrary_key, mut rng)| {
                    runtime.block_on(assert_out_of_bound_seek_returns_invalid_argument(
                        &db,
                        &mut rng,
                        arbitrary_key,
                    ));
                    Ok(())
                },
            )
            .unwrap();

        async fn assert_out_of_bound_seek_returns_invalid_argument(
            db: &Db,
            rng: &mut TestRng,
            arbitrary_key: Bytes,
        ) {
            let mut iter = db
                .scan_with_options(..arbitrary_key.clone(), &ScanOptions::default())
                .await
                .unwrap();

            let lower_bounded_range = BytesRange::from(arbitrary_key.clone()..);
            let value = sample::bytes_in_range(rng, &lower_bounded_range);
            assert!(matches!(
                iter.seek(value).await,
                Err(SlateDBError::InvalidArgument { msg: _ })
            ));

            let mut iter = db
                .scan_with_options(arbitrary_key.clone().., &ScanOptions::default())
                .await
                .unwrap();

            let upper_bounded_range = BytesRange::from(..arbitrary_key.clone());
            let value = sample::bytes_in_range(rng, &upper_bounded_range);
            assert!(matches!(
                iter.seek(value).await,
                Err(SlateDBError::InvalidArgument { msg: _ })
            ));
        }
    }

    #[test]
    fn test_seek_fast_forwards_iterator() {
        let mut runner = new_proptest_runner(None);
        let table = sample::table(runner.rng(), 1000, 10);

        let runtime = Runtime::new().unwrap();
        let db_options = test_db_options(0, 1024, None);
        let db = runtime.block_on(build_database_from_table(&table, db_options, true));

        runner
            .run(
                &(arbitrary::nonempty_range(5), arbitrary::rng()),
                |(range, mut rng)| {
                    runtime.block_on(assert_seek_fast_forwards_iterator(
                        &table, &db, &range, &mut rng,
                    ));
                    Ok(())
                },
            )
            .unwrap();

        async fn assert_seek_fast_forwards_iterator(
            table: &BTreeMap<Bytes, Bytes>,
            db: &Db,
            scan_range: &BytesRange,
            rng: &mut TestRng,
        ) {
            let mut iter = db
                .inner
                .scan_with_options(scan_range.clone(), &ScanOptions::default())
                .await
                .unwrap();

            let seek_key = sample::bytes_in_range(rng, scan_range);
            iter.seek(seek_key.clone()).await.unwrap();

            let seek_range = BytesRange::new(Included(seek_key), scan_range.end_bound().cloned());
            test_utils::assert_ranged_db_scan(table, seek_range, &mut iter).await;
        }
    }

    #[tokio::test]
    async fn test_write_batch() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            test_db_options(0, 1024, None),
            object_store,
        )
        .await
        .unwrap();

        // Create a new WriteBatch
        let mut batch = WriteBatch::new();
        batch.put(b"key1", b"value1");
        batch.put(b"key2", b"value2");
        batch.delete(b"key1");

        // Write the batch
        kv_store.write(batch).await.expect("write batch failed");

        // Read back keys
        assert_eq!(kv_store.get(b"key1").await.unwrap(), None);
        assert_eq!(
            kv_store.get(b"key2").await.unwrap(),
            Some(Bytes::from_static(b"value2"))
        );

        kv_store.close().await.unwrap();
    }

    #[cfg(feature = "wal_disable")]
    #[tokio::test]
    async fn test_write_batch_without_wal() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        // Use a very small l0 size to force flushes so await is notified
        let mut options = test_db_options(0, 8, None);

        // Disable WAL
        options.wal_enabled = false;

        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store_without_wal"),
            options,
            object_store,
        )
        .await
        .unwrap();

        // Create a new WriteBatch
        let mut batch = WriteBatch::new();
        batch.put(b"key1", b"value1");
        batch.put(b"key2", b"value2");
        batch.delete(b"key1");

        // Write the batch
        kv_store.write(batch).await.expect("write batch failed");

        // Read back keys
        assert_eq!(kv_store.get(b"key1").await.unwrap(), None);
        assert_eq!(kv_store.get(b"key2").await.unwrap(), Some("value2".into()));

        kv_store.close().await.unwrap();
    }

    #[tokio::test]
    async fn test_write_batch_with_empty_key() {
        let mut batch = WriteBatch::new();
        let result = std::panic::catch_unwind(move || {
            batch.put(b"", b"value");
        });
        assert!(
            result.is_err(),
            "Expected panic when using empty key in put operation"
        );

        let mut batch = WriteBatch::new();
        let result = std::panic::catch_unwind(move || {
            batch.delete(b"");
        });
        assert!(
            result.is_err(),
            "Expected panic when using empty key in delete operation"
        );
    }

    /// Test that batch writes are atomic. Test does the following:
    ///
    /// - A set of, say 100 keys, 1-100
    /// - Two tasks writing, one writing value to be same key, and other
    ///   writing it to be key*2.
    /// - We wait for both to complete.
    /// - Assert that either all values are same as key, or all values key*2.
    /// - Repeat above loop few times.
    ///
    /// _Note: This test is non-deterministic because it depends on the async
    /// runtime to schedule the tasks in a way that the writes are concurrent._
    #[tokio::test]
    async fn test_concurrent_batch_writes_consistency() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Arc::new(
            Db::open_with_opts(
                Path::from("/tmp/test_concurrent_kv_store"),
                test_db_options(
                    0,
                    1024,
                    // Enable compactor to prevent l0 from filling up and
                    // applying backpressure indefinitely.
                    Some(CompactorOptions {
                        poll_interval: Duration::from_millis(100),
                        max_sst_size: 256,
                        compaction_scheduler: Arc::new(SizeTieredCompactionSchedulerSupplier::new(
                            SizeTieredCompactionSchedulerOptions::default(),
                        )),
                        max_concurrent_compactions: 1,
                        compaction_runtime: None,
                    }),
                ),
                object_store.clone(),
            )
            .await
            .unwrap(),
        );

        const NUM_KEYS: usize = 100;
        const NUM_ROUNDS: usize = 20;

        for _ in 0..NUM_ROUNDS {
            // Write two tasks that write to the same keys
            let task1 = {
                let store = kv_store.clone();
                tokio::spawn(async move {
                    let mut batch = WriteBatch::new();
                    for key in 1..=NUM_KEYS {
                        batch.put(key.to_be_bytes(), key.to_be_bytes());
                    }
                    store.write(batch).await.expect("write batch failed");
                })
            };

            let task2 = {
                let store = kv_store.clone();
                tokio::spawn(async move {
                    let mut batch = WriteBatch::new();
                    for key in 1..=NUM_KEYS {
                        let value = (key * 2).to_be_bytes();
                        batch.put(key.to_be_bytes(), value);
                    }
                    store.write(batch).await.expect("write batch failed");
                })
            };

            // Wait for both tasks to complete
            join_all(vec![task1, task2]).await;

            // Ensure consistency: all values must be either key or key * 2
            let mut all_key = true;
            let mut all_key2 = true;

            for key in 1..=NUM_KEYS {
                let value = kv_store.get(key.to_be_bytes()).await.unwrap();
                let value = value.expect("Value should exist");

                if value.as_ref() != key.to_be_bytes() {
                    all_key = false;
                }
                if value.as_ref() != (key * 2).to_be_bytes() {
                    all_key2 = false;
                }
            }

            // Assert that the result is consistent: either all key or all key * 2
            assert!(
                all_key || all_key2,
                "Inconsistent state: not all values match either key or key * 2"
            );
        }

        kv_store.close().await.unwrap();
    }

    #[tokio::test]
    #[cfg(feature = "wal_disable")]
    async fn test_disable_wal_after_wal_enabled() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        // open a db and write a wal entry
        let options = test_db_options(0, 32, None);
        let db = Db::open_with_opts(path.clone(), options, object_store.clone())
            .await
            .unwrap();
        db.put(&[b'a'; 4], &[b'j'; 4]).await.unwrap();
        db.put(&[b'b'; 4], &[b'k'; 4]).await.unwrap();
        db.close().await.unwrap();

        // open a db with wal disabled and write a memtable
        let mut options = test_db_options(0, 32, None);
        options.wal_enabled = false;
        let db = Db::open_with_opts(path.clone(), options.clone(), object_store.clone())
            .await
            .unwrap();
        db.delete_with_options(
            &[b'b'; 4],
            &WriteOptions {
                await_durable: false,
            },
        )
        .await
        .unwrap();
        db.put(&[b'a'; 4], &[b'z'; 64]).await.unwrap();
        db.close().await.unwrap();

        // ensure we don't overwrite the values we just put on a reload
        let db = Db::open_with_opts(path.clone(), options.clone(), object_store.clone())
            .await
            .unwrap();
        let val = db.get(&[b'a'; 4]).await.unwrap();
        assert_eq!(val.unwrap(), Bytes::copy_from_slice(&[b'z'; 64]));
        let val = db.get(&[b'b'; 4]).await.unwrap();
        assert!(val.is_none());
    }

    #[cfg(feature = "wal_disable")]
    #[tokio::test]
    async fn test_wal_disabled() {
        use crate::{test_utils::assert_iterator, types::RowEntry};

        let clock = Arc::new(TestClock::new());
        let mut options = test_db_options_with_clock(0, 128, None, clock.clone());
        options.wal_enabled = false;
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let sst_format = SsTableFormat::default();
        let table_store = Arc::new(TableStore::new(
            object_store.clone(),
            sst_format,
            path.clone(),
            None,
        ));
        let db = Db::open_with_opts(path.clone(), options, object_store.clone())
            .await
            .unwrap();
        let manifest_store = Arc::new(ManifestStore::new(&path, object_store.clone()));
        let mut stored_manifest = StoredManifest::load(manifest_store.clone()).await.unwrap();
        let write_options = WriteOptions {
            await_durable: false,
        };

        db.put_with_options(
            &[b'a'; 32],
            &[b'j'; 32],
            &PutOptions::default(),
            &write_options,
        )
        .await
        .unwrap();
        db.delete_with_options(&[b'b'; 31], &write_options)
            .await
            .unwrap();

        // ensure the memtable's size is greater than l0_sst_size_bytes, or
        // the memtable will not be flushed to l0, and the test will hang
        // at this put_with_options call.
        let write_options = WriteOptions {
            await_durable: true,
        };
        clock.ticker.store(10, Ordering::SeqCst);
        db.put_with_options(
            &[b'c'; 32],
            &[b'l'; 32],
            &PutOptions::default(),
            &write_options,
        )
        .await
        .unwrap();

        let state = wait_for_manifest_condition(
            &mut stored_manifest,
            |s| !s.l0.is_empty(),
            Duration::from_secs(30),
        )
        .await;
        assert_eq!(state.l0.len(), 1);

        let l0 = state.l0.front().unwrap();
        let mut iter =
            SstIterator::new_borrowed(.., l0, table_store.clone(), SstIteratorOptions::default())
                .await
                .unwrap();
        assert_iterator(
            &mut iter,
            vec![
                RowEntry::new_value(&[b'a'; 32], &[b'j'; 32], 1).with_create_ts(0),
                RowEntry::new_tombstone(&[b'b'; 31], 2).with_create_ts(0),
                RowEntry::new_value(&[b'c'; 32], &[b'l'; 32], 3).with_create_ts(10),
            ],
        )
        .await;
    }

    #[tokio::test]
    async fn test_put_flushes_memtable() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let kv_store = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();

        let manifest_store = Arc::new(ManifestStore::new(&path, object_store.clone()));
        let mut stored_manifest = StoredManifest::load(manifest_store.clone()).await.unwrap();
        let sst_format = SsTableFormat {
            min_filter_keys: 10,
            ..SsTableFormat::default()
        };
        let table_store = Arc::new(TableStore::new(
            object_store.clone(),
            sst_format,
            path.clone(),
            None,
        ));

        // Write data a few times such that each loop results in a memtable flush
        let mut last_compacted = 0;
        for i in 0..3 {
            let key = [b'a' + i; 16];
            let value = [b'b' + i; 50];
            kv_store.put(&key, &value).await.unwrap();
            let key = [b'j' + i; 16];
            let value = [b'k' + i; 50];
            kv_store.put(&key, &value).await.unwrap();
            let db_state = wait_for_manifest_condition(
                &mut stored_manifest,
                |s| s.last_compacted_wal_sst_id > last_compacted,
                Duration::from_secs(30),
            )
            .await;

            // 1 empty wal at startup + 2 wal per iteration.
            assert_eq!(db_state.last_compacted_wal_sst_id, 1 + (i as u64) * 2 + 2);
            last_compacted = db_state.last_compacted_wal_sst_id
        }

        let manifest = stored_manifest.refresh().await.unwrap();
        let l0 = &manifest.core.l0;
        assert_eq!(l0.len(), 3);
        let sst_iter_options = SstIteratorOptions::default();

        for i in 0u8..3u8 {
            let sst1 = l0.get(2 - i as usize).unwrap();
            let mut iter =
                SstIterator::new_borrowed(.., sst1, table_store.clone(), sst_iter_options)
                    .await
                    .unwrap();
            let kv = iter.next().await.unwrap().unwrap();
            assert_eq!(kv.key.as_ref(), [b'a' + i; 16]);
            assert_eq!(kv.value.as_ref(), [b'b' + i; 50]);
            let kv = iter.next().await.unwrap().unwrap();
            assert_eq!(kv.key.as_ref(), [b'j' + i; 16]);
            assert_eq!(kv.value.as_ref(), [b'k' + i; 50]);
            let kv = iter.next().await.unwrap();
            assert!(kv.is_none());
        }
        assert!(
            kv_store
                .metrics()
                .lookup(IMMUTABLE_MEMTABLE_FLUSHES)
                .unwrap()
                .get()
                > 0
        );
    }

    // 2 threads so we can can wait on the write_with_options (main) thread
    // while the write_batch (background) thread is blocked on writing the
    // WAL SST.
    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_apply_wal_memory_backpressure() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let mut options = test_db_options(0, 1, None);
        options.max_unflushed_bytes = 1;
        let db = Db::open_with_fp_registry(
            path.clone(),
            options,
            object_store.clone(),
            fp_registry.clone(),
        )
        .await
        .unwrap();

        let write_opts = WriteOptions {
            await_durable: false,
        };

        // Block WAL flush
        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "pause").unwrap();

        // 1 imm_wal in memory
        db.put_with_options(b"key1", b"val1", &PutOptions::default(), &write_opts)
            .await
            .unwrap();

        let snapshot = db.inner.state.read().snapshot();

        // Unblock WAL flush so runtime shuts down nicely even if we have a failure
        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "off").unwrap();

        // WAL should pile up in memory since it can't be flushed
        assert_eq!(snapshot.state.imm_wal.len(), 1);
    }

    #[tokio::test]
    async fn test_apply_backpressure_to_memtable_flush() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let mut options = test_db_options(0, 1, None);
        options.l0_max_ssts = 4;
        let db = Db::open_with_opts(Path::from("/tmp/test_kv_store"), options, object_store)
            .await
            .unwrap();
        db.put(b"key1", b"val1").await.unwrap();
        db.put(b"key2", b"val2").await.unwrap();
        db.put(b"key3", b"val3").await.unwrap();
        db.put(b"key4", b"val4").await.unwrap();
        db.put(b"key5", b"val5").await.unwrap();

        db.flush().await.unwrap();

        let snapshot = db.inner.state.read().snapshot();
        assert_eq!(snapshot.state.imm_memtable.len(), 1);
    }

    #[tokio::test]
    async fn test_put_empty_value() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            test_db_options(0, 1024, None),
            object_store,
        )
        .await
        .unwrap();
        let key = b"test_key";
        let value = b"";
        kv_store.put(key, value).await.unwrap();
        kv_store.flush().await.unwrap();

        assert_eq!(
            kv_store.get(key).await.unwrap(),
            Some(Bytes::from_static(value))
        );
    }

    #[tokio::test]
    async fn test_flush_while_iterating() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let kv_store = Db::open_with_opts(
            Path::from("/tmp/test_kv_store"),
            test_db_options(0, 1024, None),
            object_store,
        )
        .await
        .unwrap();

        let memtable = {
            let mut lock = kv_store.inner.state.write();
            lock.wal()
                .put(RowEntry::new_value(b"abc1111", b"value1111", 1));
            lock.wal()
                .put(RowEntry::new_value(b"abc2222", b"value2222", 2));
            lock.wal()
                .put(RowEntry::new_value(b"abc3333", b"value3333", 3));
            lock.wal().table().clone()
        };

        let mut iter = memtable.iter();
        let kv = iter.next().await.unwrap().unwrap();
        assert_eq!(kv.key, b"abc1111".as_slice());

        kv_store.flush().await.unwrap();

        let kv = iter.next().await.unwrap().unwrap();
        assert_eq!(kv.key, b"abc2222".as_slice());

        let kv = iter.next().await.unwrap().unwrap();
        assert_eq!(kv.key, b"abc3333".as_slice());
    }

    #[tokio::test]
    async fn test_basic_restore() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let mut next_wal_id = 1;
        let kv_store = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();
        // increment wal id for the empty wal
        next_wal_id += 1;

        // do a few writes that will result in l0 flushes
        let l0_count: u64 = 3;
        for i in 0..l0_count {
            kv_store
                .put(&[b'a' + i as u8; 16], &[b'b' + i as u8; 48])
                .await
                .unwrap();
            kv_store
                .put(&[b'j' + i as u8; 16], &[b'k' + i as u8; 48])
                .await
                .unwrap();
            next_wal_id += 2;
        }

        // write some smaller keys so that we populate wal without flushing to l0
        let sst_count: u64 = 5;
        for i in 0..sst_count {
            kv_store
                .put(&i.to_be_bytes(), &i.to_be_bytes())
                .await
                .unwrap();
            kv_store.flush().await.unwrap();
            next_wal_id += 1;
        }

        kv_store.close().await.unwrap();

        // recover and validate that sst files are loaded on recovery.
        let kv_store_restored = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();
        // increment wal id for the empty wal
        next_wal_id += 1;

        for i in 0..l0_count {
            let val = kv_store_restored.get([b'a' + i as u8; 16]).await.unwrap();
            assert_eq!(val, Some(Bytes::copy_from_slice(&[b'b' + i as u8; 48])));
            let val = kv_store_restored.get([b'j' + i as u8; 16]).await.unwrap();
            assert_eq!(val, Some(Bytes::copy_from_slice(&[b'k' + i as u8; 48])));
        }
        for i in 0..sst_count {
            let val = kv_store_restored.get(i.to_be_bytes()).await.unwrap();
            assert_eq!(val, Some(Bytes::copy_from_slice(&i.to_be_bytes())));
        }
        kv_store_restored.close().await.unwrap();

        // validate that the manifest file exists.
        let manifest_store = Arc::new(ManifestStore::new(&path, object_store.clone()));
        let stored_manifest = StoredManifest::load(manifest_store).await.unwrap();
        let db_state = stored_manifest.db_state();
        assert_eq!(db_state.next_wal_sst_id, next_wal_id);
    }

    #[tokio::test]
    #[allow(clippy::await_holding_lock)]
    async fn test_restore_seq_number() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let db = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();

        db.put(b"key1", b"val1").await.unwrap();
        db.put(b"key2", b"val2").await.unwrap();
        db.put(b"key3", b"val3").await.unwrap();
        db.flush().await.unwrap();
        db.close().await.unwrap();

        let db_restored = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();

        let mut state = db_restored.inner.state.write();
        let memtable = state.memtable();
        let mut iter = memtable.table().iter();
        assert_iterator(
            &mut iter,
            vec![
                RowEntry::new_value(b"key1", b"val1", 1).with_create_ts(0),
                RowEntry::new_value(b"key2", b"val2", 2).with_create_ts(0),
                RowEntry::new_value(b"key3", b"val3", 3).with_create_ts(0),
            ],
        )
        .await;
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 4)]
    async fn test_should_read_uncommitted_data_if_read_level_uncommitted() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let kv_store = Db::open_with_fp_registry(
            path.clone(),
            test_db_options(0, 1024, None),
            object_store.clone(),
            fp_registry.clone(),
        )
        .await
        .unwrap();

        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "pause").unwrap();
        kv_store
            .put_with_options(
                "foo".as_bytes(),
                "bar".as_bytes(),
                &PutOptions::default(),
                &WriteOptions {
                    await_durable: false,
                },
            )
            .await
            .unwrap();

        // Validate uncommitted read
        let val = kv_store
            .get_with_options(
                "foo".as_bytes(),
                &ReadOptions {
                    read_level: Uncommitted,
                },
            )
            .await
            .unwrap();
        assert_eq!(val, Some("bar".into()));

        // Validate committed read should still return None
        let val = kv_store.get("foo".as_bytes()).await.unwrap();
        assert_eq!(val, None);

        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "off").unwrap();
        kv_store.close().await.unwrap();
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 4)]
    async fn test_should_read_only_committed_data() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let kv_store = Db::open_with_fp_registry(
            path.clone(),
            test_db_options(0, 1024, None),
            object_store.clone(),
            fp_registry.clone(),
        )
        .await
        .unwrap();

        kv_store
            .put("foo".as_bytes(), "bar".as_bytes())
            .await
            .unwrap();
        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "pause").unwrap();
        kv_store
            .put_with_options(
                "foo".as_bytes(),
                "bla".as_bytes(),
                &PutOptions::default(),
                &WriteOptions {
                    await_durable: false,
                },
            )
            .await
            .unwrap();

        let val = kv_store.get("foo".as_bytes()).await.unwrap();
        assert_eq!(val, Some("bar".into()));
        let val = kv_store
            .get_with_options(
                "foo".as_bytes(),
                &ReadOptions {
                    read_level: Uncommitted,
                },
            )
            .await
            .unwrap();
        assert_eq!(val, Some("bla".into()));

        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "off").unwrap();
        kv_store.close().await.unwrap();
    }

    #[tokio::test]
    async fn test_should_delete_without_awaiting_flush() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let kv_store = Db::open_with_fp_registry(
            path.clone(),
            test_db_options(0, 1024, None),
            object_store.clone(),
            fp_registry.clone(),
        )
        .await
        .unwrap();

        kv_store
            .put("foo".as_bytes(), "bar".as_bytes())
            .await
            .unwrap();
        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "pause").unwrap();
        kv_store
            .delete_with_options(
                "foo".as_bytes(),
                &WriteOptions {
                    await_durable: false,
                },
            )
            .await
            .unwrap();

        let val = kv_store.get("foo".as_bytes()).await.unwrap();
        assert_eq!(val, Some("bar".into()));
        let val = kv_store
            .get_with_options(
                "foo".as_bytes(),
                &ReadOptions {
                    read_level: Uncommitted,
                },
            )
            .await
            .unwrap();
        assert_eq!(val, None);

        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "off").unwrap();
        kv_store.close().await.unwrap();
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_should_recover_imm_from_wal() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        fail_parallel::cfg(fp_registry.clone(), "write-compacted-sst-io-error", "pause").unwrap();

        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let mut next_wal_id = 1;
        let db = Db::open_with_fp_registry(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
            fp_registry.clone(),
        )
        .await
        .unwrap();
        next_wal_id += 1;

        // write a few keys that will result in memtable flushes
        let key1 = [b'a'; 32];
        let value1 = [b'b'; 96];
        db.put(key1, value1).await.unwrap();
        next_wal_id += 1;
        let key2 = [b'c'; 32];
        let value2 = [b'd'; 96];
        db.put(key2, value2).await.unwrap();
        next_wal_id += 1;

        let reader = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();

        // increment wal id for the empty wal
        next_wal_id += 1;

        // verify that we reload imm
        let snapshot = reader.inner.state.read().snapshot();
        assert_eq!(snapshot.state.imm_memtable.len(), 2);

        // one empty wal and two wals for the puts
        assert_eq!(
            snapshot.state.imm_memtable.front().unwrap().last_wal_id(),
            1 + 2
        );
        assert_eq!(snapshot.state.imm_memtable.get(1).unwrap().last_wal_id(), 2);
        assert_eq!(snapshot.state.core().next_wal_sst_id, next_wal_id);
        assert_eq!(
            reader.get(key1).await.unwrap(),
            Some(Bytes::copy_from_slice(&value1))
        );
        assert_eq!(
            reader.get(key2).await.unwrap(),
            Some(Bytes::copy_from_slice(&value2))
        );

        fail_parallel::cfg(fp_registry.clone(), "write-compacted-sst-io-error", "off").unwrap();
        db.close().await.unwrap();
        reader.close().await.unwrap();
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_should_recover_imm_from_wal_after_flush_error() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        fail_parallel::cfg(
            fp_registry.clone(),
            "write-compacted-sst-io-error",
            "return",
        )
        .unwrap();

        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let db = Db::open_with_fp_registry(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
            fp_registry.clone(),
        )
        .await
        .unwrap();

        // write a few keys that will result in memtable flushes
        let key1 = [b'a'; 32];
        let value1 = [b'b'; 96];
        let result = db.put(&key1, &value1).await;
        assert!(result.is_ok(), "Failed to write key1");

        let flush_result = db.inner.flush_immutable_memtables().await;
        assert!(flush_result.is_err());
        db.close().await.unwrap();

        // pause write-compacted-sst-io-error to prevent immutable tables
        // from being flushed, so we can snapshot the state when there is
        // an immutable table to verify its contents.
        fail_parallel::cfg(fp_registry.clone(), "write-compacted-sst-io-error", "pause").unwrap();

        // reload the db
        let db = Db::open_with_fp_registry(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
            fp_registry.clone(),
        )
        .await
        .unwrap();

        // verify that we reload imm
        let snapshot = db.inner.state.read().snapshot();

        // resume write-compacted-sst-io-error since we got a snapshot and
        // want to let the test finish.
        fail_parallel::cfg(fp_registry.clone(), "write-compacted-sst-io-error", "off").unwrap();

        assert_eq!(snapshot.state.imm_memtable.len(), 1);

        // one empty wal and one wal for the first put
        assert_eq!(
            snapshot.state.imm_memtable.front().unwrap().last_wal_id(),
            1 + 1
        );
        assert!(snapshot.state.imm_memtable.get(1).is_none());

        assert_eq!(snapshot.state.core().next_wal_sst_id, 4);
        assert_eq!(
            db.get(key1).await.unwrap(),
            Some(Bytes::copy_from_slice(&value1))
        );
    }

    #[tokio::test]
    async fn test_should_fail_write_if_wal_flush_task_panics() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let db = Arc::new(
            Db::open_with_fp_registry(
                path.clone(),
                test_db_options(0, 128, None),
                object_store.clone(),
                fp_registry.clone(),
            )
            .await
            .unwrap(),
        );

        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "panic").unwrap();
        let result = db.put(b"foo", b"bar").await;
        assert!(matches!(result, Err(SlateDBError::BackgroundTaskPanic(_))));
    }

    #[tokio::test]
    async fn test_wal_id_last_seen_should_exist_even_if_wal_write_fails() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let db = Arc::new(
            Db::open_with_fp_registry(
                path.clone(),
                test_db_options(0, 128, None),
                object_store.clone(),
                fp_registry.clone(),
            )
            .await
            .unwrap(),
        );

        fail_parallel::cfg(fp_registry.clone(), "write-wal-sst-io-error", "panic").unwrap();

        // Trigger a WAL write, which should not advance the manifest WAL ID
        let result = db.put(b"foo", b"bar").await;
        assert!(matches!(result, Err(SlateDBError::BackgroundTaskPanic(_))));

        // Close, which flushes the latest manifest to the object store
        db.close().await.unwrap();

        let manifest_store = ManifestStore::new(&path, object_store.clone());
        let table_store = Arc::new(TableStore::new(
            object_store.clone(),
            SsTableFormat::default(),
            path.clone(),
            None,
        ));

        // Get the next WAL SST ID based on what's currently in the object store
        let next_wal_sst_id = table_store.next_wal_sst_id(0).await.unwrap();

        // Get the latest manifest
        let (_, manifest) = manifest_store.read_latest_manifest().await.unwrap();

        // The manifest's next_wal_sst_id, which uses `wal_id_last_seen + 1`, should
        // be the same as the next WAL SST ID based on what's currently in the object store
        assert_eq!(manifest.core.next_wal_sst_id, next_wal_sst_id);
    }

    async fn do_test_should_read_compacted_db(options: DbOptions) {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let db = Db::open_with_opts(path.clone(), options, object_store.clone())
            .await
            .unwrap();
        let ms = ManifestStore::new(&path, object_store.clone());
        let mut sm = StoredManifest::load(Arc::new(ms)).await.unwrap();

        // write enough to fill up a few l0 SSTs
        for i in 0..4 {
            db.put(&[b'a' + i; 32], &[1u8 + i; 32]).await.unwrap();
            db.put(&[b'm' + i; 32], &[13u8 + i; 32]).await.unwrap();
        }
        // wait for compactor to compact them
        wait_for_manifest_condition(
            &mut sm,
            |s| s.l0_last_compacted.is_some() && s.l0.is_empty(),
            Duration::from_secs(10),
        )
        .await;
        info!(
            "1 l0: {} {}",
            db.inner.state.read().state().core().l0.len(),
            db.inner.state.read().state().core().compacted.len()
        );
        // write more l0s and wait for compaction
        for i in 0..4 {
            db.put(&[b'f' + i; 32], &[6u8 + i; 32]).await.unwrap();
            db.put(&[b's' + i; 32], &[19u8 + i; 32]).await.unwrap();
        }
        wait_for_manifest_condition(
            &mut sm,
            |s| s.l0_last_compacted.is_some() && s.l0.is_empty(),
            Duration::from_secs(10),
        )
        .await;
        info!(
            "2 l0: {} {}",
            db.inner.state.read().state().core().l0.len(),
            db.inner.state.read().state().core().compacted.len()
        );
        // write another l0
        db.put(&[b'a'; 32], &[128u8; 32]).await.unwrap();
        db.put(&[b'm'; 32], &[129u8; 32]).await.unwrap();

        let val = db.get([b'a'; 32]).await.unwrap();
        assert_eq!(val, Some(Bytes::copy_from_slice(&[128u8; 32])));
        let val = db.get([b'm'; 32]).await.unwrap();
        assert_eq!(val, Some(Bytes::copy_from_slice(&[129u8; 32])));
        for i in 1..4 {
            info!(
                "3 l0: {} {}",
                db.inner.state.read().state().core().l0.len(),
                db.inner.state.read().state().core().compacted.len()
            );
            let val = db.get([b'a' + i; 32]).await.unwrap();
            assert_eq!(val, Some(Bytes::copy_from_slice(&[1u8 + i; 32])));
            let val = db.get([b'm' + i; 32]).await.unwrap();
            assert_eq!(val, Some(Bytes::copy_from_slice(&[13u8 + i; 32])));
        }
        for i in 0..4 {
            let val = db.get([b'f' + i; 32]).await.unwrap();
            assert_eq!(val, Some(Bytes::copy_from_slice(&[6u8 + i; 32])));
            let val = db.get([b's' + i; 32]).await.unwrap();
            assert_eq!(val, Some(Bytes::copy_from_slice(&[19u8 + i; 32])));
        }
        let neg_lookup = db.get(b"abc").await;
        assert!(neg_lookup.unwrap().is_none());
    }

    #[tokio::test]
    async fn test_should_read_from_compacted_db() {
        do_test_should_read_compacted_db(test_db_options(
            0,
            127,
            Some(CompactorOptions {
                poll_interval: Duration::from_millis(100),
                max_sst_size: 256,
                compaction_scheduler: Arc::new(SizeTieredCompactionSchedulerSupplier::new(
                    SizeTieredCompactionSchedulerOptions::default(),
                )),
                max_concurrent_compactions: 1,
                compaction_runtime: None,
            }),
        ))
        .await;
    }

    #[tokio::test]
    async fn test_should_read_from_compacted_db_no_filters() {
        do_test_should_read_compacted_db(test_db_options(
            u32::MAX,
            127,
            Some(CompactorOptions {
                poll_interval: Duration::from_millis(100),
                max_sst_size: 256,
                compaction_scheduler: Arc::new(SizeTieredCompactionSchedulerSupplier::new(
                    SizeTieredCompactionSchedulerOptions::default(),
                )),
                max_concurrent_compactions: 1,
                compaction_runtime: None,
            }),
        ))
        .await
    }

    #[tokio::test]
    async fn test_db_open_should_write_empty_wal() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        // assert that open db writes an empty wal.
        let db = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();
        assert_eq!(db.inner.state.read().state().core().next_wal_sst_id, 2);
        db.put(b"1", b"1").await.unwrap();
        // assert that second open writes another empty wal.
        let db = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();
        assert_eq!(db.inner.state.read().state().core().next_wal_sst_id, 4);
    }

    #[tokio::test]
    async fn test_empty_wal_should_fence_old_writer() {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");

        async fn do_put(db: &Db, key: &[u8], val: &[u8]) -> Result<(), SlateDBError> {
            db.put_with_options(
                key,
                val,
                &PutOptions::default(),
                &WriteOptions {
                    await_durable: true,
                },
            )
            .await
        }

        // open db1 and assert that it can write.
        let db1 = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();
        do_put(&db1, b"1", b"1").await.unwrap();

        // open db2, causing it to write an empty wal and fence db1.
        let db2 = Db::open_with_opts(
            path.clone(),
            test_db_options(0, 128, None),
            object_store.clone(),
        )
        .await
        .unwrap();

        // assert that db1 can no longer write.
        let err = do_put(&db1, b"1", b"1").await;
        assert!(matches!(err, Err(SlateDBError::Fenced)));

        do_put(&db2, b"2", b"2").await.unwrap();
        assert_eq!(db2.inner.state.read().state().core().next_wal_sst_id, 5);
    }

    #[tokio::test]
    async fn test_invalid_clock_progression() {
        // Given:
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");

        let clock = Arc::new(TestClock::new());
        let db = Db::open_with_opts(
            path.clone(),
            DbOptions {
                clock: clock.clone(),
                ..test_db_options(0, 128, None)
            },
            object_store.clone(),
        )
        .await
        .unwrap();

        // When:
        // put with time = 10
        clock.ticker.store(10, Ordering::SeqCst);
        db.put(b"1", b"1").await.unwrap();

        // Then:
        // put with time goes backwards, should fail
        clock.ticker.store(5, Ordering::SeqCst);
        match db.put(b"1", b"1").await {
            Ok(_) => panic!("expected an error on inserting backwards time"),
            Err(e) => assert!(
                e.to_string().contains("Last tick: 10, Next tick: 5"),
                "{}",
                e.to_string()
            ),
        }
    }

    #[tokio::test]
    async fn test_invalid_clock_progression_across_db_instances() {
        // Given:
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");

        let clock = Arc::new(TestClock::new());
        let db = Db::open_with_opts(
            path.clone(),
            DbOptions {
                clock: clock.clone(),
                ..test_db_options(0, 128, None)
            },
            object_store.clone(),
        )
        .await
        .unwrap();

        // When:
        // put with time = 10
        clock.ticker.store(10, Ordering::SeqCst);
        db.put(b"1", b"1").await.unwrap();
        db.flush().await.unwrap();

        let db2 = Db::open_with_opts(
            path.clone(),
            DbOptions {
                clock: clock.clone(),
                ..test_db_options(0, 128, None)
            },
            object_store.clone(),
        )
        .await
        .unwrap();
        clock.ticker.store(5, Ordering::SeqCst);
        match db2.put(b"1", b"1").await {
            Ok(_) => panic!("expected an error on inserting backwards time"),
            Err(e) => assert!(
                e.to_string().contains("Last tick: 10, Next tick: 5"),
                "{}",
                e.to_string()
            ),
        }
    }

    #[tokio::test]
    #[cfg(feature = "wal_disable")]
    async fn should_flush_all_memtables_when_wal_disabled() {
        // Given:
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");

        let db_options = DbOptions {
            wal_enabled: false,
            flush_interval: Some(Duration::from_secs(10)),
            ..DbOptions::default()
        };

        let db = Db::open_with_opts(path.clone(), db_options.clone(), Arc::clone(&object_store))
            .await
            .unwrap();

        let mut rng = proptest_util::rng::new_test_rng(None);
        let table = sample::table(&mut rng, 1000, 5);
        test_utils::seed_database(&db, &table, false).await.unwrap();
        db.flush().await.unwrap();

        // When: reopen the database without closing the old instance
        let reopened_db =
            Db::open_with_opts(path.clone(), db_options.clone(), Arc::clone(&object_store))
                .await
                .unwrap();

        // Then:
        assert_records_in_range(
            &table,
            &reopened_db,
            &ScanOptions::default(),
            BytesRange::from(..),
        )
        .await
    }

    #[tokio::test]
    async fn test_recover_clock_tick_from_wal() {
        let clock = Arc::new(TestClock::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");

        let db = Db::open_with_opts(
            path.clone(),
            test_db_options_with_clock(0, 1024, None, clock.clone()),
            Arc::clone(&object_store),
        )
        .await
        .unwrap();

        clock.ticker.store(10, Ordering::SeqCst);
        db.put(&[b'a'; 4], &[b'j'; 8])
            .await
            .expect("write batch failed");
        clock.ticker.store(11, Ordering::SeqCst);
        db.put(&[b'b'; 4], &[b'k'; 8])
            .await
            .expect("write batch failed");

        // close the db to flush the manifest
        db.close().await.unwrap();

        // check the last_l0_clock_tick persisted in the manifest, it should be
        // i64::MIN because no WAL SST has yet made its way into L0
        let manifest_store = Arc::new(ManifestStore::new(&path, object_store.clone()));
        let stored_manifest = StoredManifest::load(manifest_store).await.unwrap();
        let db_state = stored_manifest.db_state();
        let last_clock_tick = db_state.last_l0_clock_tick;
        assert_eq!(last_clock_tick, i64::MIN);

        let clock = Arc::new(TestClock::new());
        let db = Db::open_with_opts(
            path.clone(),
            test_db_options_with_clock(0, 1024, None, clock.clone()),
            Arc::clone(&object_store),
        )
        .await
        .unwrap();

        assert_eq!(db.inner.mono_clock.last_tick.load(Ordering::SeqCst), 11);
    }

    #[tokio::test]
    async fn test_should_update_manifest_clock_tick_on_l0_flush() {
        let clock = Arc::new(TestClock::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");

        let db = Db::open_with_opts(
            path.clone(),
            test_db_options_with_clock(0, 32, None, clock.clone()),
            Arc::clone(&object_store),
        )
        .await
        .unwrap();

        // this will exceed the l0_sst_size_bytes, meaning a clean shutdown
        // will update the manifest
        clock.ticker.store(10, Ordering::SeqCst);
        db.put(&[b'a'; 4], &[b'j'; 8])
            .await
            .expect("write batch failed");
        clock.ticker.store(11, Ordering::SeqCst);
        db.put(&[b'b'; 4], &[b'k'; 8])
            .await
            .expect("write batch failed");

        // close the db to flush the manifest
        db.flush().await.unwrap();
        db.close().await.unwrap();

        // check the last_clock_tick persisted in the manifest, it should be
        // i64::MIN because no WAL SST has yet made its way into L0
        let manifest_store = Arc::new(ManifestStore::new(&path, object_store.clone()));
        let stored_manifest = StoredManifest::load(manifest_store).await.unwrap();
        let db_state = stored_manifest.db_state();
        let last_clock_tick = db_state.last_l0_clock_tick;
        assert_eq!(last_clock_tick, 11);
    }

    #[tokio::test]
    #[cfg(feature = "wal_disable")]
    async fn test_recover_clock_tick_from_manifest() {
        let clock = Arc::new(TestClock::new());
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = Path::from("/tmp/test_kv_store");
        let mut options = test_db_options_with_clock(0, 32, None, clock.clone());
        options.wal_enabled = false;

        let db = Db::open_with_opts(path.clone(), options, Arc::clone(&object_store))
            .await
            .unwrap();

        clock.ticker.store(10, Ordering::SeqCst);
        db.put(&[b'a'; 4], &[b'j'; 28])
            .await
            .expect("write batch failed");
        clock.ticker.store(11, Ordering::SeqCst);
        db.put(&[b'b'; 4], &[b'k'; 28])
            .await
            .expect("write batch failed");

        // close the db to flush the manifest
        db.flush().await.unwrap();
        db.close().await.unwrap();

        let clock = Arc::new(TestClock::new());
        let mut options = test_db_options_with_clock(0, 32, None, clock.clone());
        options.wal_enabled = false;
        let db = Db::open_with_opts(path.clone(), options, Arc::clone(&object_store))
            .await
            .unwrap();

        assert_eq!(db.inner.mono_clock.last_tick.load(Ordering::SeqCst), 11);
    }

    #[test]
    fn test_write_option_defaults() {
        // This is a regression test for a bug where the defaults for WriteOptions were not being
        // set correctly due to visibility issues.
        let write_options = WriteOptions::default();
        assert!(write_options.await_durable);
    }

    async fn wait_for_manifest_condition(
        sm: &mut StoredManifest,
        cond: impl Fn(&CoreDbState) -> bool,
        timeout: Duration,
    ) -> CoreDbState {
        let start = std::time::Instant::now();
        while start.elapsed() < timeout {
            let manifest = sm.refresh().await.unwrap();
            if cond(&manifest.core) {
                return manifest.core.clone();
            }
            tokio::time::sleep(Duration::from_millis(10)).await;
        }
        panic!("manifest condition took longer than timeout")
    }

    fn test_db_options(
        min_filter_keys: u32,
        l0_sst_size_bytes: usize,
        compactor_options: Option<CompactorOptions>,
    ) -> DbOptions {
        test_db_options_with_clock(
            min_filter_keys,
            l0_sst_size_bytes,
            compactor_options,
            Arc::new(TestClock::new()),
        )
    }

    fn test_db_options_with_clock(
        min_filter_keys: u32,
        l0_sst_size_bytes: usize,
        compactor_options: Option<CompactorOptions>,
        clock: Arc<TestClock>,
    ) -> DbOptions {
        test_db_options_with_ttl(
            min_filter_keys,
            l0_sst_size_bytes,
            compactor_options,
            clock,
            None,
        )
    }

    fn test_db_options_with_ttl(
        min_filter_keys: u32,
        l0_sst_size_bytes: usize,
        compactor_options: Option<CompactorOptions>,
        clock: Arc<TestClock>,
        ttl: Option<u64>,
    ) -> DbOptions {
        DbOptions {
            flush_interval: Some(Duration::from_millis(100)),
            #[cfg(feature = "wal_disable")]
            wal_enabled: true,
            manifest_poll_interval: Duration::from_millis(100),
            max_unflushed_bytes: 134_217_728,
            l0_max_ssts: 8,
            min_filter_keys,
            filter_bits_per_key: 10,
            l0_sst_size_bytes,
            compactor_options,
            compression_codec: None,
            object_store_cache_options: ObjectStoreCacheOptions::default(),
            block_cache: None,
            garbage_collector_options: None,
            clock,
            default_ttl: ttl,
        }
    }
}