1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258
3259
3260
3261
3262
3263
3264
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274
3275
3276
3277
3278
3279
3280
3281
3282
3283
3284
3285
3286
3287
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311
3312
3313
3314
3315
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363
3364
3365
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375
3376
3377
3378
3379
3380
3381
3382
3383
3384
3385
3386
3387
3388
3389
3390
3391
3392
3393
3394
3395
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405
3406
3407
3408
3409
3410
3411
3412
3413
3414
3415
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
3426
3427
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442
3443
3444
3445
3446
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458
3459
3460
3461
3462
3463
3464
3465
3466
3467
3468
3469
3470
3471
3472
3473
3474
3475
3476
3477
3478
3479
3480
3481
3482
3483
3484
3485
3486
3487
3488
3489
3490
3491
3492
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516
3517
3518
3519
3520
3521
3522
3523
3524
3525
3526
3527
3528
3529
3530
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548
3549
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599
3600
3601
3602
3603
3604
3605
3606
3607
3608
3609
3610
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636
3637
3638
3639
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651
3652
3653
3654
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691
3692
3693
3694
3695
3696
3697
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711
3712
3713
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745
3746
3747
3748
3749
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
3767
3768
3769
3770
3771
3772
3773
3774
3775
3776
3777
3778
3779
3780
3781
3782
3783
3784
3785
3786
3787
3788
3789
3790
3791
3792
3793
3794
3795
3796
3797
3798
3799
3800
3801
3802
3803
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818
3819
3820
3821
3822
3823
3824
3825
3826
3827
3828
3829
3830
3831
3832
3833
3834
3835
3836
3837
3838
3839
3840
3841
3842
3843
3844
3845
3846
3847
3848
3849
3850
3851
3852
3853
3854
3855
3856
3857
3858
3859
3860
3861
3862
3863
3864
3865
3866
3867
3868
3869
3870
3871
3872
3873
3874
3875
3876
3877
3878
3879
3880
3881
3882
3883
3884
3885
3886
3887
3888
3889
3890
3891
3892
3893
3894
3895
3896
3897
3898
3899
3900
3901
3902
3903
3904
3905
3906
3907
3908
3909
3910
3911
3912
3913
3914
3915
3916
3917
3918
3919
3920
3921
3922
3923
3924
3925
3926
3927
3928
3929
3930
3931
3932
3933
3934
3935
3936
3937
3938
3939
3940
3941
3942
3943
3944
3945
3946
3947
3948
3949
3950
3951
3952
3953
3954
3955
3956
3957
3958
3959
3960
3961
3962
3963
3964
3965
3966
3967
3968
3969
3970
3971
3972
3973
3974
3975
3976
3977
3978
3979
3980
3981
3982
3983
3984
3985
3986
3987
3988
3989
3990
3991
3992
3993
3994
3995
3996
3997
3998
3999
4000
4001
4002
4003
4004
4005
4006
4007
4008
4009
4010
4011
4012
4013
4014
4015
4016
4017
4018
4019
4020
4021
4022
4023
4024
4025
4026
4027
4028
4029
4030
4031
4032
4033
4034
4035
4036
4037
4038
4039
4040
4041
4042
4043
4044
4045
4046
4047
4048
4049
4050
4051
4052
4053
4054
4055
4056
4057
4058
4059
4060
4061
4062
4063
4064
4065
4066
4067
4068
4069
4070
4071
4072
4073
4074
4075
4076
4077
4078
4079
4080
4081
4082
4083
4084
4085
4086
4087
4088
4089
4090
4091
4092
4093
4094
4095
4096
4097
4098
4099
4100
4101
4102
4103
4104
4105
4106
4107
4108
4109
4110
4111
4112
4113
4114
4115
4116
4117
4118
4119
4120
4121
4122
4123
4124
4125
4126
4127
4128
4129
4130
4131
4132
4133
4134
4135
4136
4137
4138
4139
4140
4141
4142
4143
4144
4145
4146
4147
4148
4149
4150
4151
4152
4153
4154
4155
4156
4157
4158
4159
4160
4161
4162
4163
4164
4165
4166
4167
4168
4169
4170
4171
4172
4173
4174
4175
4176
4177
4178
4179
4180
4181
4182
4183
4184
4185
4186
4187
4188
4189
4190
4191
4192
4193
4194
4195
4196
4197
4198
4199
4200
4201
4202
4203
4204
4205
4206
4207
4208
4209
4210
4211
4212
4213
4214
4215
4216
4217
4218
4219
4220
4221
4222
4223
4224
4225
4226
4227
4228
4229
4230
4231
4232
4233
4234
4235
4236
4237
4238
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248
4249
4250
4251
4252
4253
4254
4255
4256
4257
4258
4259
4260
4261
4262
4263
4264
4265
4266
4267
4268
4269
4270
4271
4272
4273
4274
4275
4276
4277
4278
4279
4280
4281
4282
4283
4284
4285
4286
4287
4288
4289
4290
4291
4292
4293
4294
4295
4296
4297
4298
4299
4300
4301
4302
4303
4304
4305
4306
4307
4308
4309
4310
4311
4312
4313
4314
4315
4316
4317
4318
4319
4320
4321
4322
4323
4324
4325
4326
4327
4328
4329
4330
4331
4332
4333
4334
4335
4336
4337
4338
4339
4340
4341
4342
4343
4344
4345
4346
4347
4348
4349
4350
4351
4352
4353
4354
4355
4356
4357
4358
4359
4360
4361
4362
4363
4364
4365
4366
4367
4368
4369
4370
4371
4372
4373
4374
4375
4376
4377
4378
4379
4380
4381
4382
4383
4384
4385
4386
4387
4388
4389
4390
4391
4392
4393
4394
4395
4396
4397
4398
4399
4400
4401
4402
4403
4404
4405
4406
4407
4408
4409
4410
4411
4412
4413
4414
4415
4416
4417
4418
4419
4420
4421
4422
4423
4424
4425
4426
4427
4428
4429
4430
4431
4432
4433
4434
4435
4436
4437
4438
4439
4440
4441
4442
4443
4444
4445
4446
4447
4448
4449
4450
4451
4452
4453
4454
4455
4456
4457
4458
4459
4460
4461
4462
4463
4464
4465
4466
4467
4468
4469
4470
4471
4472
4473
4474
4475
4476
4477
4478
4479
4480
4481
4482
4483
4484
4485
4486
4487
4488
4489
4490
4491
4492
4493
4494
4495
4496
4497
4498
4499
4500
4501
4502
4503
4504
4505
4506
4507
4508
4509
4510
4511
4512
4513
4514
4515
4516
4517
4518
4519
4520
4521
4522
4523
4524
4525
4526
4527
4528
4529
4530
4531
4532
4533
4534
4535
4536
4537
4538
4539
4540
4541
4542
4543
4544
4545
4546
4547
4548
4549
4550
4551
4552
4553
4554
4555
4556
4557
4558
4559
4560
4561
4562
4563
4564
4565
4566
4567
4568
4569
4570
4571
4572
4573
4574
4575
4576
4577
4578
4579
4580
4581
4582
4583
4584
4585
4586
4587
4588
4589
4590
4591
4592
4593
4594
4595
4596
4597
4598
4599
4600
4601
4602
4603
4604
4605
4606
4607
4608
4609
4610
4611
4612
4613
4614
4615
4616
4617
4618
4619
4620
4621
4622
4623
4624
4625
4626
4627
4628
4629
4630
4631
4632
4633
4634
4635
4636
4637
4638
4639
4640
4641
4642
4643
4644
4645
4646
4647
4648
4649
4650
4651
4652
4653
4654
4655
4656
4657
4658
4659
4660
4661
4662
4663
4664
4665
4666
4667
4668
4669
4670
4671
4672
4673
4674
4675
4676
4677
4678
4679
4680
4681
4682
4683
4684
4685
4686
4687
4688
4689
4690
4691
4692
4693
4694
4695
4696
4697
4698
4699
4700
4701
4702
4703
4704
4705
4706
4707
4708
4709
4710
4711
4712
4713
4714
4715
4716
4717
4718
4719
4720
4721
4722
4723
4724
4725
4726
4727
4728
4729
4730
4731
4732
4733
4734
4735
4736
4737
4738
4739
4740
4741
4742
4743
4744
4745
4746
4747
4748
4749
4750
4751
4752
4753
4754
4755
4756
4757
4758
4759
4760
4761
4762
4763
4764
4765
4766
4767
4768
4769
4770
4771
4772
4773
4774
4775
4776
4777
4778
4779
4780
4781
4782
4783
4784
4785
4786
4787
4788
4789
4790
4791
4792
4793
4794
4795
4796
4797
4798
4799
4800
4801
4802
4803
4804
4805
4806
4807
4808
4809
4810
4811
4812
4813
4814
4815
4816
4817
4818
4819
4820
4821
4822
4823
4824
4825
4826
4827
4828
4829
4830
4831
4832
4833
4834
4835
4836
4837
4838
4839
4840
4841
4842
4843
4844
4845
4846
4847
4848
4849
4850
4851
4852
4853
4854
4855
4856
4857
4858
4859
4860
4861
4862
4863
4864
4865
4866
4867
4868
4869
4870
4871
4872
4873
4874
4875
4876
4877
4878
4879
4880
4881
4882
4883
4884
4885
4886
4887
4888
4889
4890
4891
4892
4893
4894
4895
4896
4897
4898
4899
4900
4901
4902
4903
4904
4905
4906
4907
4908
4909
4910
4911
4912
4913
4914
4915
4916
4917
4918
4919
4920
4921
4922
4923
4924
4925
4926
4927
4928
4929
4930
4931
4932
4933
4934
4935
4936
4937
4938
4939
4940
4941
4942
4943
4944
4945
4946
4947
4948
4949
4950
4951
4952
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors
#![allow(clippy::print_stderr)]
//! Write path for MemWAL.
//!
//! This module contains all components for the write path:
//! - [`ShardWriter`] - Main writer interface for a single shard
//! - [`MemTable`] - In-memory table storing Arrow RecordBatches
//! - [`WalFlusher`] - Write-ahead log buffer for durability (Arrow IPC format)
//! - [`IndexStore`] - In-memory index management
//! - [`MemTableFlusher`] - Flush MemTable to storage as single Lance file
use std::collections::VecDeque;
use std::fmt::Debug;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::{Arc, RwLock as StdRwLock};
use std::time::{Duration, Instant};
use arrow_array::RecordBatch;
use arrow_schema::Schema as ArrowSchema;
use async_trait::async_trait;
use lance_core::datatypes::Schema;
use lance_core::{Error, Result};
use lance_index::mem_wal::ShardManifest;
use lance_io::object_store::ObjectStore;
use log::{debug, error, info, warn};
use object_store::path::Path;
use tokio::sync::{RwLock, mpsc};
use tokio::task::JoinHandle;
use tokio::time::{Interval, interval_at};
use tokio_util::sync::CancellationToken;
use tracing::instrument;
use uuid::Uuid;
pub use super::index::{
BTreeIndexConfig, BTreeMemIndex, FtsIndexConfig, HnswIndexConfig, IndexStore, MemIndexConfig,
};
pub use super::memtable::CacheConfig;
pub use super::memtable::MemTable;
pub use super::memtable::batch_store::{BatchStore, StoreFull, StoredBatch};
pub use super::memtable::flush::MemTableFlusher;
pub use super::memtable::scanner::MemTableScanner;
pub use super::util::{WatchableOnceCell, WatchableOnceCellReader};
pub use super::wal::{WalEntry, WalEntryData, WalFlushResult, WalFlusher};
use super::memtable::flush::TriggerMemTableFlush;
use super::wal::{
TriggerWalFlush, WalAppender, WalFlushSource, WalOnlyState, WalTailer, empty_flush_result,
};
use super::manifest::ShardManifestStore;
// ============================================================================
// Configuration
// ============================================================================
/// Configuration for a shard writer.
#[derive(Debug, Clone)]
pub struct ShardWriterConfig {
/// Unique identifier for this shard (UUID v4).
pub shard_id: Uuid,
/// Shard spec ID this shard was created with.
/// A value of 0 indicates a manually-created shard not governed by any spec.
pub shard_spec_id: u32,
/// Whether to wait for WAL flush before returning from writes.
///
/// When true (durable writes):
/// - Each write waits for WAL persistence before returning
/// - Guarantees no data loss on crash
/// - Higher latency due to object storage writes
///
/// When false (non-durable writes):
/// - Writes return immediately after buffering in memory
/// - Potential data loss if process crashes before flush
/// - Lower latency, batched S3 operations
pub durable_write: bool,
/// Whether to update indexes synchronously on each write.
///
/// When true:
/// - Newly written data is immediately searchable via indexes
/// - Higher latency due to index update overhead
///
/// When false:
/// - Index updates are deferred
/// - New data may not appear in index-accelerated queries immediately
pub sync_indexed_write: bool,
/// Maximum WAL buffer size in bytes before triggering a flush.
///
/// This is a soft threshold - write batches are atomic and won't be split.
/// WAL flushes when buffer exceeds this size OR when `max_wal_flush_interval` elapses.
/// Default: 10MB
pub max_wal_buffer_size: usize,
/// Time-based WAL flush interval.
///
/// WAL buffer will be flushed after this duration even if size threshold
/// hasn't been reached. This ensures bounded data loss window in non-durable mode
/// and prevents accumulating too much data before flushing to object storage.
/// Default: 100ms
pub max_wal_flush_interval: Option<Duration>,
/// Maximum MemTable size in bytes before triggering a flush to storage.
///
/// MemTable size is checked every `max_wal_flush_interval` (during WAL flush ticks).
/// Default: 256MB
pub max_memtable_size: usize,
/// Maximum number of rows in a MemTable.
///
/// Used to pre-allocate the in-memory HNSW graph and vector storage
/// capacity. When the memtable reaches capacity, it will be flushed.
/// Default: 100,000 rows
pub max_memtable_rows: usize,
/// Maximum number of batches in a MemTable.
///
/// Used to pre-allocate batch storage. When this limit is reached,
/// memtable will be flushed. Sized for typical ML workloads with
/// 1024-dim vectors (~82KB per 20-row batch).
/// Default: 8,000 batches
pub max_memtable_batches: usize,
/// Batch size for parallel HEAD requests when scanning for manifest versions.
///
/// Higher values scan faster but use more parallel requests.
/// Default: 2
pub manifest_scan_batch_size: usize,
/// Maximum unflushed bytes before applying backpressure.
///
/// When total unflushed data (active memtable + frozen memtables) exceeds this,
/// new writes will block until some data is flushed to storage.
/// This prevents unbounded memory growth during write spikes.
///
/// Default: 1GB
pub max_unflushed_memtable_bytes: usize,
/// Interval for logging warnings when writes are blocked by backpressure.
///
/// When a write is blocked waiting for WAL flush, memtable flush, or index
/// updates to complete, a warning is logged after this duration. The write
/// will continue waiting indefinitely (it never fails due to backpressure),
/// but warnings are logged at this interval to help diagnose slow flushes.
///
/// Default: 30 seconds
pub backpressure_log_interval: Duration,
/// Maximum rows to buffer before flushing to async indexes.
///
/// Only applies when `sync_indexed_write` is false. Larger values enable
/// better vectorization but increase memory usage and latency before data
/// becomes searchable.
///
/// Default: 10,000 rows
pub async_index_buffer_rows: usize,
/// Maximum time to buffer before flushing to async indexes.
///
/// Only applies when `sync_indexed_write` is false. Ensures bounded latency
/// for data to become searchable even during low write throughput.
///
/// Default: 1 second
pub async_index_interval: Duration,
/// Interval for periodic stats logging.
///
/// Stats (write throughput, backpressure events, memtable size) are logged
/// at this interval. Set to None to disable periodic stats logging.
///
/// Default: 60 seconds
pub stats_log_interval: Option<Duration>,
/// Whether to maintain an in-memory MemTable on top of the WAL.
///
/// When `true` (default), the writer maintains an in-memory `MemTable`,
/// optionally with indexes, and asynchronously flushes frozen MemTables
/// to Lance files alongside the WAL.
///
/// When `false`, the writer skips the MemTable layer entirely:
/// - No MemTable / BatchStore / IndexStore is allocated.
/// - `index_configs` must be empty (validated at open time).
/// - No MemTable freezing or Lance file flushing happens.
/// - `max_unflushed_memtable_bytes` is reused as the backpressure
/// budget for the WAL-only pending-batch queue: `put` blocks while
/// the queue's estimated bytes meet or exceed this threshold.
/// - The async batched WAL pipeline still runs, driven by the same
/// `max_wal_buffer_size`, `max_wal_flush_interval`, and
/// `durable_write` settings as MemTable mode.
///
/// MemTable-tied tunables (`max_memtable_size`, `max_memtable_rows`,
/// `max_memtable_batches`, `sync_indexed_write`, `async_index_buffer_rows`,
/// `async_index_interval`) are ignored when `enable_memtable == false`.
///
/// For raw single-entry synchronous atomic appends with no buffering and
/// no background tasks, use `WalAppender` directly — it is a strictly
/// lower-level primitive.
pub enable_memtable: bool,
}
impl Default for ShardWriterConfig {
fn default() -> Self {
Self {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: true,
sync_indexed_write: true,
max_wal_buffer_size: 10 * 1024 * 1024, // 10MB
max_wal_flush_interval: Some(Duration::from_millis(100)), // 100ms
max_memtable_size: 256 * 1024 * 1024, // 256MB
max_memtable_rows: 100_000, // 100k rows
max_memtable_batches: 8_000, // 8k batches
manifest_scan_batch_size: 2,
max_unflushed_memtable_bytes: 1024 * 1024 * 1024, // 1GB
backpressure_log_interval: Duration::from_secs(30),
async_index_buffer_rows: 10_000,
async_index_interval: Duration::from_secs(1),
stats_log_interval: Some(Duration::from_secs(60)), // 1 minute
enable_memtable: true,
}
}
}
impl ShardWriterConfig {
/// Create a new configuration with the given shard ID.
pub fn new(shard_id: Uuid) -> Self {
Self {
shard_id,
..Default::default()
}
}
/// Set the shard spec ID.
pub fn with_shard_spec_id(mut self, spec_id: u32) -> Self {
self.shard_spec_id = spec_id;
self
}
/// Set durable writes mode.
pub fn with_durable_write(mut self, durable: bool) -> Self {
self.durable_write = durable;
self
}
/// Set indexed writes mode.
pub fn with_sync_indexed_write(mut self, indexed: bool) -> Self {
self.sync_indexed_write = indexed;
self
}
/// Set maximum WAL buffer size.
pub fn with_max_wal_buffer_size(mut self, size: usize) -> Self {
self.max_wal_buffer_size = size;
self
}
/// Set maximum flush interval.
pub fn with_max_wal_flush_interval(mut self, interval: Duration) -> Self {
self.max_wal_flush_interval = Some(interval);
self
}
/// Set maximum MemTable size.
pub fn with_max_memtable_size(mut self, size: usize) -> Self {
self.max_memtable_size = size;
self
}
/// Set maximum MemTable rows for index pre-allocation.
pub fn with_max_memtable_rows(mut self, rows: usize) -> Self {
self.max_memtable_rows = rows;
self
}
/// Set maximum MemTable batches for batch store pre-allocation.
pub fn with_max_memtable_batches(mut self, batches: usize) -> Self {
self.max_memtable_batches = batches;
self
}
/// Set manifest scan batch size.
pub fn with_manifest_scan_batch_size(mut self, size: usize) -> Self {
self.manifest_scan_batch_size = size;
self
}
/// Set maximum unflushed bytes for backpressure.
pub fn with_max_unflushed_memtable_bytes(mut self, size: usize) -> Self {
self.max_unflushed_memtable_bytes = size;
self
}
/// Set backpressure log interval.
pub fn with_backpressure_log_interval(mut self, interval: Duration) -> Self {
self.backpressure_log_interval = interval;
self
}
/// Set async index buffer rows.
pub fn with_async_index_buffer_rows(mut self, rows: usize) -> Self {
self.async_index_buffer_rows = rows;
self
}
/// Set async index interval.
pub fn with_async_index_interval(mut self, interval: Duration) -> Self {
self.async_index_interval = interval;
self
}
/// Set stats logging interval. Use None to disable periodic stats logging.
pub fn with_stats_log_interval(mut self, interval: Option<Duration>) -> Self {
self.stats_log_interval = interval;
self
}
/// Toggle the in-memory MemTable layer. See `enable_memtable` for the
/// full WAL-only-mode contract. Defaults to `true`.
pub fn with_enable_memtable(mut self, enable: bool) -> Self {
self.enable_memtable = enable;
self
}
}
// ============================================================================
// Background Task Infrastructure
// ============================================================================
/// Factory function for creating ticker messages.
type MessageFactory<T> = Box<dyn Fn() -> T + Send + Sync>;
/// Handler trait for processing messages in a background task.
#[async_trait]
pub trait MessageHandler<T: Send + Debug + 'static>: Send {
/// Define periodic tickers that generate messages.
fn tickers(&mut self) -> Vec<(Duration, MessageFactory<T>)> {
vec![]
}
/// Handle a single message.
async fn handle(&mut self, message: T) -> Result<()>;
/// Cleanup on shutdown.
async fn cleanup(&mut self, _shutdown_ok: bool) -> Result<()> {
Ok(())
}
}
/// Dispatcher that runs the event loop for a single message handler.
struct TaskDispatcher<T: Send + Debug> {
handler: Box<dyn MessageHandler<T>>,
rx: mpsc::UnboundedReceiver<T>,
cancellation_token: CancellationToken,
name: String,
}
impl<T: Send + Debug + 'static> TaskDispatcher<T> {
async fn run(mut self) -> Result<()> {
let tickers = self.handler.tickers();
let mut ticker_intervals: Vec<(Interval, MessageFactory<T>)> = tickers
.into_iter()
.map(|(duration, factory)| {
let interval = interval_at(tokio::time::Instant::now() + duration, duration);
(interval, factory)
})
.collect();
// A single failing message must not bring down the dispatcher:
// the WAL flusher and MemTable flusher both run on this loop,
// and dropping their channels deadlocks all subsequent puts
// (and panics any task waiting on the corresponding watch).
// Log and keep draining; the worst case for a transient flush
// failure is replay from the WAL on next open.
let result = loop {
if ticker_intervals.is_empty() {
tokio::select! {
biased;
_ = self.cancellation_token.cancelled() => {
debug!("Task '{}' received cancellation", self.name);
break Ok(());
}
msg = self.rx.recv() => {
match msg {
Some(message) => {
if let Err(e) = self.handler.handle(message).await {
error!("Task '{}' error handling message: {}", self.name, e);
}
}
None => {
debug!("Task '{}' channel closed", self.name);
break Ok(());
}
}
}
}
} else {
let first_ticker = ticker_intervals.first_mut().unwrap();
let first_interval = &mut first_ticker.0;
tokio::select! {
biased;
_ = self.cancellation_token.cancelled() => {
debug!("Task '{}' received cancellation", self.name);
break Ok(());
}
_ = first_interval.tick() => {
let message = (ticker_intervals[0].1)();
if let Err(e) = self.handler.handle(message).await {
error!("Task '{}' error handling ticker message: {}", self.name, e);
}
}
msg = self.rx.recv() => {
match msg {
Some(message) => {
if let Err(e) = self.handler.handle(message).await {
error!("Task '{}' error handling message: {}", self.name, e);
}
}
None => {
debug!("Task '{}' channel closed", self.name);
break Ok(());
}
}
}
}
}
};
let cleanup_ok = result.is_ok();
self.handler.cleanup(cleanup_ok).await?;
info!("Task dispatcher '{}' stopped", self.name);
result
}
}
/// Executor that manages multiple background tasks.
pub struct TaskExecutor {
tasks: StdRwLock<Vec<(String, JoinHandle<Result<()>>)>>,
cancellation_token: CancellationToken,
}
impl TaskExecutor {
pub fn new() -> Self {
Self {
tasks: StdRwLock::new(Vec::new()),
cancellation_token: CancellationToken::new(),
}
}
pub fn add_handler<T: Send + Debug + 'static>(
&self,
name: String,
handler: Box<dyn MessageHandler<T>>,
rx: mpsc::UnboundedReceiver<T>,
) -> Result<()> {
let dispatcher = TaskDispatcher {
handler,
rx,
cancellation_token: self.cancellation_token.clone(),
name: name.clone(),
};
let handle = tokio::spawn(async move { dispatcher.run().await });
self.tasks.write().unwrap().push((name, handle));
Ok(())
}
pub async fn shutdown_all(&self) -> Result<()> {
info!("Shutting down all tasks");
self.cancellation_token.cancel();
let tasks = std::mem::take(&mut *self.tasks.write().unwrap());
for (name, handle) in tasks {
match handle.await {
Ok(Ok(())) => debug!("Task '{}' completed successfully", name),
Ok(Err(e)) => warn!("Task '{}' completed with error: {}", name, e),
Err(e) => error!("Task '{}' panicked: {}", name, e),
}
}
Ok(())
}
}
impl Default for TaskExecutor {
fn default() -> Self {
Self::new()
}
}
// ============================================================================
// Durability and Backpressure Types
// ============================================================================
/// Result of a durability notification.
///
/// This is a simple enum that can be cloned, unlike `Result<(), Error>`.
#[derive(Clone, Debug, PartialEq, Eq)]
pub enum DurabilityResult {
/// Write is now durable.
Durable,
/// Write failed with an error message.
Failed(String),
}
impl DurabilityResult {
/// Create a successful durability result.
pub fn ok() -> Self {
Self::Durable
}
/// Create a failed durability result.
pub fn err(msg: impl Into<String>) -> Self {
Self::Failed(msg.into())
}
/// Check if the result is durable.
pub fn is_ok(&self) -> bool {
matches!(self, Self::Durable)
}
/// Convert to a Result.
pub fn into_result(self) -> Result<()> {
match self {
Self::Durable => Ok(()),
Self::Failed(msg) => Err(Error::io(msg)),
}
}
}
/// Type alias for durability watchers.
pub type DurabilityWatcher = WatchableOnceCellReader<DurabilityResult>;
/// Type alias for durability cells.
pub type DurabilityCell = WatchableOnceCell<DurabilityResult>;
/// Statistics for backpressure monitoring.
#[derive(Debug, Default)]
pub struct BackpressureStats {
/// Total number of times backpressure was applied.
total_count: AtomicU64,
/// Total time spent waiting on backpressure (in milliseconds).
total_wait_ms: AtomicU64,
}
impl BackpressureStats {
/// Create new backpressure stats.
pub fn new() -> Self {
Self::default()
}
/// Record a backpressure event.
pub fn record(&self, wait_ms: u64) {
self.total_count.fetch_add(1, Ordering::Relaxed);
self.total_wait_ms.fetch_add(wait_ms, Ordering::Relaxed);
}
/// Get the total backpressure count.
pub fn count(&self) -> u64 {
self.total_count.load(Ordering::Relaxed)
}
/// Get the total time spent waiting on backpressure.
pub fn total_wait_ms(&self) -> u64 {
self.total_wait_ms.load(Ordering::Relaxed)
}
/// Get a snapshot of all stats.
pub fn snapshot(&self) -> BackpressureStatsSnapshot {
BackpressureStatsSnapshot {
total_count: self.total_count.load(Ordering::Relaxed),
total_wait_ms: self.total_wait_ms.load(Ordering::Relaxed),
}
}
}
/// Snapshot of backpressure statistics.
#[derive(Debug, Clone, Default)]
pub struct BackpressureStatsSnapshot {
/// Total number of times backpressure was applied.
pub total_count: u64,
/// Total time spent waiting on backpressure (in milliseconds).
pub total_wait_ms: u64,
}
/// Backpressure controller for managing write flow.
pub struct BackpressureController {
/// Configuration.
config: ShardWriterConfig,
/// Stats for monitoring.
stats: Arc<BackpressureStats>,
}
impl BackpressureController {
/// Create a new backpressure controller.
pub fn new(config: ShardWriterConfig) -> Self {
Self {
config,
stats: Arc::new(BackpressureStats::new()),
}
}
/// Get backpressure stats.
pub fn stats(&self) -> &Arc<BackpressureStats> {
&self.stats
}
/// Check and apply backpressure if needed.
///
/// This method blocks if the system is under memory pressure, waiting for
/// frozen memtables to be flushed to storage until under threshold.
///
/// Backpressure is applied when:
/// - `unflushed_memtable_bytes` >= `max_unflushed_memtable_bytes`
///
/// # Arguments
/// - `get_state`: Closure that returns current (unflushed_memtable_bytes, oldest_memtable_watcher)
///
/// The closure is called in a loop to get fresh state after each wait.
pub async fn maybe_apply_backpressure<F>(&self, mut get_state: F) -> Result<()>
where
F: FnMut() -> (usize, Option<DurabilityWatcher>),
{
let start = std::time::Instant::now();
let mut iteration = 0u32;
loop {
let (unflushed_memtable_bytes, oldest_watcher) = get_state();
// Check if under threshold
if unflushed_memtable_bytes < self.config.max_unflushed_memtable_bytes {
if iteration > 0 {
let wait_ms = start.elapsed().as_millis() as u64;
self.stats.record(wait_ms);
}
return Ok(());
}
iteration += 1;
debug!(
"Backpressure triggered: unflushed_bytes={}, max={}, iteration={}",
unflushed_memtable_bytes, self.config.max_unflushed_memtable_bytes, iteration
);
// Wait for oldest memtable to flush
if let Some(mut mem_watcher) = oldest_watcher {
tokio::select! {
_ = mem_watcher.await_value() => {}
_ = tokio::time::sleep(self.config.backpressure_log_interval) => {
warn!(
"Backpressure wait timeout, continuing to wait: unflushed_bytes={}, interval={}s, iteration={}",
unflushed_memtable_bytes,
self.config.backpressure_log_interval.as_secs(),
iteration
);
}
}
} else {
// No watcher available - sleep briefly to avoid busy loop
tokio::time::sleep(std::time::Duration::from_millis(10)).await;
}
}
}
}
/// Result of a write operation.
#[derive(Debug)]
pub struct WriteResult {
/// Range of batch positions [start, end) for inserted batches.
/// For a single batch, this is [pos, pos+1).
pub batch_positions: std::ops::Range<usize>,
}
/// ShardWriter state shared across tasks.
struct WriterState {
memtable: MemTable,
last_flushed_wal_entry_position: u64,
/// Total size of frozen memtables (for backpressure).
frozen_memtable_bytes: usize,
/// Flush watchers for frozen memtables (for backpressure).
frozen_flush_watchers: VecDeque<(usize, DurabilityWatcher)>,
/// Sealed-but-undrained memtables, kept queryable so a concurrent reader
/// sees no hole between `freeze_memtable` and the flush task's manifest
/// commit. Pushed in `freeze_memtable`; removed by generation in
/// `flush_memtable` on commit success only (retained on failure until a
/// later flush or WAL replay on reopen).
frozen_memtables: VecDeque<Arc<MemTable>>,
/// Flag to prevent duplicate memtable flush requests.
flush_requested: bool,
/// Counter for WAL flush threshold crossings.
wal_flush_trigger_count: usize,
/// Last time a WAL flush was triggered (for time-based flush).
last_wal_flush_trigger_time: u64,
}
/// Capture a point-in-time scan handle to one in-memory memtable (active
/// or frozen — same shape). Shared by `active_memtable_ref` and
/// `in_memory_memtable_refs` so both stamp identical fields.
fn in_memory_ref(mt: &MemTable) -> crate::dataset::mem_wal::scanner::InMemoryMemTableRef {
crate::dataset::mem_wal::scanner::InMemoryMemTableRef {
batch_store: mt.batch_store(),
index_store: mt
.indexes_arc()
.unwrap_or_else(|| Arc::new(IndexStore::new())),
schema: mt.schema().clone(),
generation: mt.generation(),
}
}
fn start_time() -> std::time::Instant {
use std::sync::OnceLock;
static START: OnceLock<std::time::Instant> = OnceLock::new();
*START.get_or_init(std::time::Instant::now)
}
fn now_millis() -> u64 {
start_time().elapsed().as_millis() as u64
}
/// Replay WAL entries written after the last successfully-flushed generation
/// into the freshly-built MemTable. Updates any in-memory indexes attached to
/// the MemTable so replayed rows are immediately searchable.
///
/// Returns the WAL position the next `WalAppender::append` should use — i.e.
/// one past the highest replayed position, or the original start position if
/// the loop found nothing (we proved that position is empty by getting
/// `None` back from the tailer). The caller can pass this directly to
/// `WalAppender::seed_next_position` unconditionally so the first post-open
/// append skips its own discovery probe.
///
/// Aborts with an error if any replayed entry's `writer_epoch` is strictly
/// greater than `our_epoch` — that indicates a successor writer claimed the
/// shard between our `claim_epoch` and this replay, fencing us.
async fn replay_memtable_from_wal(
object_store: Arc<ObjectStore>,
base_path: Path,
shard_id: Uuid,
our_epoch: u64,
manifest: &ShardManifest,
memtable: &mut MemTable,
) -> Result<u64> {
// WAL positions are 1-based (see `FIRST_WAL_ENTRY_POSITION`), so a
// cursor of 0 means "no flush has ever stamped this shard" and replay
// starts at position 1. After flushing position N the cursor holds N
// and replay starts at N+1. The arithmetic collapses to a single
// saturating_add(1) in both cases — we deliberately do not consult
// `flushed_generations` here, since an external compactor may
// legitimately drain that vector back to empty after merging its
// contents into the base table.
let start_position = manifest.replay_after_wal_entry_position.saturating_add(1);
// The MemTable is always freshly built before this function runs, so
// any existing BatchStore entries can only have come from this replay
// pass. We index everything in `[0, batch_count)` at the end.
debug_assert_eq!(memtable.batch_count(), 0);
let tailer = WalTailer::new(object_store, base_path, shard_id);
let mut position = start_position;
loop {
match tailer.read_entry(position).await? {
// The first NotFound proves the WAL tip is at `position`, which
// is the next write position to hand back.
None => break,
Some(entry) => {
if entry.writer_epoch > our_epoch {
return Err(Error::io(format!(
"WAL replay aborted: entry at position {} has writer_epoch {} > our claimed epoch {} for shard {} (writer was fenced during open)",
position, entry.writer_epoch, our_epoch, shard_id
)));
}
if !entry.batches.is_empty() {
memtable.insert_batches_only(entry.batches).await?;
}
position = position.checked_add(1).ok_or_else(|| {
Error::io(format!(
"WAL position overflow during replay for shard {}",
shard_id
))
})?;
}
}
}
// Update in-memory indexes with the replayed batches so readers see them
// through the index path (matching what would have happened on the
// pre-crash writer's WAL flush). Indexes from the previous writer don't
// persist; this rebuilds them from the WAL.
if let Some(indexes) = memtable.indexes_arc() {
let batches_after = memtable.batch_count();
if batches_after > 0 {
let store = memtable.batch_store();
let mut stored: Vec<StoredBatch> = Vec::with_capacity(batches_after);
for pos in 0..batches_after {
if let Some(s) = store.get(pos) {
stored.push(s.clone());
}
}
tokio::task::spawn_blocking(move || indexes.insert_batches_parallel(&stored))
.await
.map_err(|e| {
Error::internal(format!("WAL replay index update task panicked: {}", e))
})??;
}
}
Ok(position)
}
/// Shared state for writer operations.
struct SharedWriterState {
state: Arc<RwLock<WriterState>>,
wal_flusher: Arc<WalFlusher>,
wal_flush_tx: mpsc::UnboundedSender<TriggerWalFlush>,
memtable_flush_tx: mpsc::UnboundedSender<TriggerMemTableFlush>,
config: ShardWriterConfig,
schema: Arc<ArrowSchema>,
pk_field_ids: Vec<i32>,
max_memtable_batches: usize,
max_memtable_rows: usize,
index_configs: Vec<MemIndexConfig>,
}
impl SharedWriterState {
#[allow(clippy::too_many_arguments)]
fn new(
state: Arc<RwLock<WriterState>>,
wal_flusher: Arc<WalFlusher>,
wal_flush_tx: mpsc::UnboundedSender<TriggerWalFlush>,
memtable_flush_tx: mpsc::UnboundedSender<TriggerMemTableFlush>,
config: ShardWriterConfig,
schema: Arc<ArrowSchema>,
pk_field_ids: Vec<i32>,
max_memtable_batches: usize,
max_memtable_rows: usize,
index_configs: Vec<MemIndexConfig>,
) -> Self {
Self {
state,
wal_flusher,
wal_flush_tx,
memtable_flush_tx,
config,
schema,
pk_field_ids,
max_memtable_batches,
max_memtable_rows,
index_configs,
}
}
/// Freeze the current memtable and send it to the flush handler.
///
/// Takes `&mut WriterState` directly since caller already holds the lock.
fn freeze_memtable(&self, state: &mut WriterState) -> Result<u64> {
let pending_wal_range = state.memtable.batch_store().pending_wal_flush_range();
let last_wal_entry_position = state.last_flushed_wal_entry_position;
let old_batch_store = state.memtable.batch_store();
let old_indexes = state.memtable.indexes_arc();
let next_generation = state.memtable.generation() + 1;
let mut new_memtable = MemTable::with_capacity(
self.schema.clone(),
next_generation,
self.pk_field_ids.clone(),
CacheConfig::default(),
self.max_memtable_batches,
)?;
if !self.index_configs.is_empty() {
let indexes = Arc::new(IndexStore::from_configs(
&self.index_configs,
self.max_memtable_rows,
self.max_memtable_batches,
)?);
new_memtable.set_indexes_arc(indexes);
}
let mut old_memtable = std::mem::replace(&mut state.memtable, new_memtable);
old_memtable.freeze(last_wal_entry_position);
let _memtable_flush_watcher = old_memtable.create_memtable_flush_completion();
if pending_wal_range.is_some() {
let completion_cell: WatchableOnceCell<std::result::Result<WalFlushResult, String>> =
WatchableOnceCell::new();
let completion_reader = completion_cell.reader();
old_memtable.set_wal_flush_completion(completion_reader);
let end_batch_position = old_batch_store.len();
self.wal_flusher.trigger_flush(
WalFlushSource::BatchStore {
batch_store: old_batch_store,
indexes: old_indexes,
},
end_batch_position,
Some(completion_cell),
)?;
}
let frozen_size = old_memtable.estimated_size();
state.frozen_memtable_bytes += frozen_size;
let flush_watcher = old_memtable
.get_memtable_flush_watcher()
.expect("Flush watcher should exist after create_memtable_flush_completion");
state
.frozen_flush_watchers
.push_back((frozen_size, flush_watcher));
let frozen_memtable = Arc::new(old_memtable);
// Keep this generation queryable until its manifest commit lands
// (dropped in `flush_memtable`, success only). Arc refcount, not a
// copy — the flush task holds it alive for the whole drain anyway.
state.frozen_memtables.push_back(frozen_memtable.clone());
debug!(
"Frozen memtable generation {}, pending_count = {}",
next_generation - 1,
state.frozen_flush_watchers.len()
);
let _ = self.memtable_flush_tx.send(TriggerMemTableFlush {
memtable: frozen_memtable,
done: None,
});
Ok(next_generation)
}
/// Track batch for WAL durability.
fn track_batch_for_wal(&self, batch_position: usize) -> super::wal::BatchDurableWatcher {
self.wal_flusher.track_batch(batch_position)
}
/// Check if memtable flush is needed and trigger if so.
///
/// Takes `&mut WriterState` directly since caller already holds the lock.
fn maybe_trigger_memtable_flush(&self, state: &mut WriterState) -> Result<()> {
if state.flush_requested {
return Ok(());
}
let should_flush = state.memtable.estimated_size() >= self.config.max_memtable_size
|| state.memtable.is_batch_store_full();
if should_flush {
state.flush_requested = true;
self.freeze_memtable(state)?;
state.flush_requested = false;
}
Ok(())
}
/// Check if WAL flush is needed and trigger if so.
///
/// Takes `&mut WriterState` directly since caller already holds the lock.
fn maybe_trigger_wal_flush(&self, state: &mut WriterState) {
let threshold = self.config.max_wal_buffer_size;
let batch_count = state.memtable.batch_count();
let total_bytes = state.memtable.estimated_size();
let batch_store = state.memtable.batch_store();
let indexes = state.memtable.indexes_arc();
// Check if there are any unflushed batches
let has_pending = batch_store.pending_wal_flush_count() > 0;
// Check time-based trigger first
let time_trigger = if let Some(interval) = self.config.max_wal_flush_interval {
let interval_millis = interval.as_millis() as u64;
let last_trigger = state.last_wal_flush_trigger_time;
let now = now_millis();
// If last_trigger is 0, this is the first write - start the timer but don't flush
if last_trigger == 0 {
state.last_wal_flush_trigger_time = now;
None
} else {
let elapsed = now.saturating_sub(last_trigger);
if elapsed >= interval_millis && has_pending {
state.last_wal_flush_trigger_time = now;
Some(now)
} else {
None
}
}
} else {
None
};
// If time trigger fired, send a flush message
if time_trigger.is_some() {
let _ = self.wal_flush_tx.send(TriggerWalFlush {
source: WalFlushSource::BatchStore {
batch_store,
indexes,
},
end_batch_position: batch_count,
done: None,
});
return;
}
// Check size-based trigger
if threshold == 0 {
return;
}
// Calculate how many thresholds have been crossed (1 at 10MB, 2 at 20MB, etc.)
let thresholds_crossed = total_bytes / threshold;
// Trigger flush for each unclaimed threshold crossing
while state.wal_flush_trigger_count < thresholds_crossed {
state.wal_flush_trigger_count += 1;
// Update last trigger time so time-based trigger doesn't fire immediately after
state.last_wal_flush_trigger_time = now_millis();
// Trigger WAL flush with captured batch range
let _ = self.wal_flush_tx.send(TriggerWalFlush {
source: WalFlushSource::BatchStore {
batch_store: batch_store.clone(),
indexes: indexes.clone(),
},
end_batch_position: batch_count,
done: None,
});
}
}
}
impl SharedWriterState {
fn unflushed_memtable_bytes(&self) -> usize {
// Total unflushed bytes = active memtable + all frozen memtables
self.state
.try_read()
.ok()
.map(|s| {
let active = s.memtable.estimated_size();
active + s.frozen_memtable_bytes
})
.unwrap_or(0)
}
fn oldest_memtable_watcher(&self) -> Option<DurabilityWatcher> {
// Return a watcher for the oldest frozen memtable's flush completion.
// If no frozen memtables, return the active memtable's watcher since it will
// eventually be frozen and flushed.
self.state.try_read().ok().and_then(|s| {
// First try frozen memtable watchers
s.frozen_flush_watchers
.front()
.map(|(_, watcher)| watcher.clone())
// If no frozen memtables, use active memtable's watcher
.or_else(|| s.memtable.get_memtable_flush_watcher())
})
}
}
/// Trigger-tracking state for WAL-only mode (no MemTable).
///
/// MemTable mode keeps these counters inside `WriterState`. WAL-only mode
/// has no `WriterState`, so the same counters live here, with one
/// adjustment: instead of a monotonic threshold-crossing count (which
/// can't decrement when the pending queue drains), we record the
/// `pending_bytes` value at the time of the last size-trigger. A trigger
/// fires whenever `pending_bytes` has grown by at least one
/// `max_wal_buffer_size` since `last_trigger_pending_bytes`. When the
/// pending queue is drained, `pending_bytes` drops below the recorded
/// baseline, and the next push detects this via `pending_bytes < baseline`
/// and resets the baseline.
#[derive(Debug, Default)]
struct WalOnlyTriggerState {
/// `pending_bytes` value recorded the last time a size-based trigger
/// fired. Resets to 0 when `pending_bytes` drops below it (drain
/// happened since the last trigger).
last_trigger_pending_bytes: usize,
/// Last time a WAL flush was triggered (for time-based trigger).
last_wal_flush_trigger_time: u64,
}
/// Per-mode state for `ShardWriter`.
enum WriterMode {
/// Default mode: MemTable + indexes + Lance file flushing on top of
/// the WAL.
MemTable {
state: Arc<RwLock<WriterState>>,
writer_state: Arc<SharedWriterState>,
backpressure: BackpressureController,
},
/// WAL-only mode: drainable pending-batch queue + WAL pipeline. No
/// MemTable, no indexes, no Lance file flushing.
WalOnly {
state: Arc<WalOnlyState>,
wal_flush_tx: mpsc::UnboundedSender<TriggerWalFlush>,
trigger: StdRwLock<WalOnlyTriggerState>,
backpressure: BackpressureController,
},
}
/// Main writer for a MemWAL shard.
pub struct ShardWriter {
config: ShardWriterConfig,
epoch: u64,
wal_flusher: Arc<WalFlusher>,
task_executor: Arc<TaskExecutor>,
manifest_store: Arc<ShardManifestStore>,
stats: SharedWriteStats,
mode: WriterMode,
}
impl ShardWriter {
/// Open or create a ShardWriter.
///
/// The `base_path` should come from `ObjectStore::from_uri()` to ensure
/// WAL files are written inside the dataset directory.
#[instrument(name = "sw_open", level = "info", skip_all, fields(shard_id = %config.shard_id, index_count = index_configs.len()))]
pub async fn open(
object_store: Arc<ObjectStore>,
base_path: Path,
base_uri: impl Into<String>,
config: ShardWriterConfig,
schema: Arc<ArrowSchema>,
index_configs: Vec<MemIndexConfig>,
) -> Result<Self> {
if !config.enable_memtable && !index_configs.is_empty() {
return Err(Error::invalid_input(
"indexes require enable_memtable = true; \
WAL-only mode does not maintain in-memory indexes",
));
}
let base_uri = base_uri.into();
let shard_id = config.shard_id;
let manifest_store = Arc::new(ShardManifestStore::new(
object_store.clone(),
&base_path,
shard_id,
config.manifest_scan_batch_size,
));
// Claim the shard (epoch-based fencing) — done once, then shared
// with the WalAppender via `with_claimed_epoch`.
let (epoch, manifest) = manifest_store.claim_epoch(config.shard_spec_id).await?;
info!(
"Opened ShardWriter for shard {} (epoch {}, generation {}, enable_memtable {})",
shard_id, epoch, manifest.current_generation, config.enable_memtable
);
// Create WAL appender (owns object store, epoch, and position
// state). Seed the appender's stats hint from the higher of the
// manifest's two cursors: `replay_after_wal_entry_position` is
// updated authoritatively at every MemTable flush, while
// `wal_entry_position_last_seen` is a best-effort hint that may
// lag behind. Either can lead the other depending on which was
// updated last, so take the max (then +1) to get the most
// accurate post-recovery cursor for `wal_stats()`.
let position_hint_seed = manifest
.wal_entry_position_last_seen
.max(manifest.replay_after_wal_entry_position)
.saturating_add(1);
let wal_appender = Arc::new(WalAppender::with_claimed_epoch(
object_store.clone(),
base_path.clone(),
shard_id,
manifest_store.clone(),
epoch,
position_hint_seed,
));
// Create WAL flusher backed by the shared appender.
let mut wal_flusher = WalFlusher::new(wal_appender);
let (wal_flush_tx, wal_flush_rx) = mpsc::unbounded_channel();
wal_flusher.set_flush_channel(wal_flush_tx.clone());
let wal_flusher = Arc::new(wal_flusher);
let stats = new_shared_stats();
let task_executor = Arc::new(TaskExecutor::new());
let mode = if config.enable_memtable {
Self::open_memtable_mode(
&config,
&schema,
&manifest,
&index_configs,
wal_flusher.clone(),
wal_flush_tx,
wal_flush_rx,
object_store.clone(),
base_path,
base_uri,
shard_id,
epoch,
manifest_store.clone(),
stats.clone(),
&task_executor,
)
.await?
} else {
Self::open_wal_only_mode(
&config,
wal_flusher.clone(),
wal_flush_tx,
wal_flush_rx,
stats.clone(),
&task_executor,
)?
};
Ok(Self {
config,
epoch,
wal_flusher,
task_executor,
manifest_store,
stats,
mode,
})
}
#[allow(clippy::too_many_arguments)]
async fn open_memtable_mode(
config: &ShardWriterConfig,
schema: &Arc<ArrowSchema>,
manifest: &ShardManifest,
index_configs: &[MemIndexConfig],
wal_flusher: Arc<WalFlusher>,
wal_flush_tx: mpsc::UnboundedSender<TriggerWalFlush>,
wal_flush_rx: mpsc::UnboundedReceiver<TriggerWalFlush>,
object_store: Arc<ObjectStore>,
base_path: Path,
base_uri: String,
shard_id: Uuid,
epoch: u64,
manifest_store: Arc<ShardManifestStore>,
stats: SharedWriteStats,
task_executor: &Arc<TaskExecutor>,
) -> Result<WriterMode> {
// Create MemTable with primary key field IDs from schema
let lance_schema = Schema::try_from(schema.as_ref())?;
let pk_field_ids: Vec<i32> = lance_schema
.unenforced_primary_key()
.iter()
.map(|f| f.id)
.collect();
let mut memtable = MemTable::with_capacity(
schema.clone(),
manifest.current_generation,
pk_field_ids.clone(),
CacheConfig::default(),
config.max_memtable_batches,
)?;
// Create indexes if configured and set them on the MemTable.
if !index_configs.is_empty() {
let indexes = Arc::new(IndexStore::from_configs(
index_configs,
config.max_memtable_rows,
config.max_memtable_batches,
)?);
memtable.set_indexes_arc(indexes);
}
// Replay any WAL entries written after the last successfully-flushed
// generation. Each entry's writer_epoch is checked against ours; an
// entry with a strictly greater epoch indicates a successor writer
// claimed the shard between our `claim_epoch` and replay, so we
// abort the open with a fence error. The replay walked the tailer
// up to the WAL tip, so we hand the discovered next-write position
// straight to the appender — its first append skips the
// discover_next_position probe entirely.
let next_wal_position = replay_memtable_from_wal(
object_store.clone(),
base_path.clone(),
shard_id,
epoch,
manifest,
&mut memtable,
)
.await?;
wal_flusher
.wal_appender()
.seed_next_position(next_wal_position)
.await;
// Seed the writer's covered-WAL cursor from the post-replay tip:
// `next_wal_position` is one past the highest WAL entry we just
// replayed into the active memtable, so everything strictly below
// it is durably reflected in this writer's memtable. We can't
// seed from `manifest.wal_entry_position_last_seen` — that field
// is bumped on every successful tailer read by other readers, so
// it may sit above what's actually covered by any flushed
// generation. Subtracting 1 from a fresh shard's `next_wal_position`
// of `FIRST_WAL_ENTRY_POSITION` (= 1) yields 0, which correctly
// means "no entry covered yet."
let initial_covered_wal_entry_position = next_wal_position.saturating_sub(1);
let state = Arc::new(RwLock::new(WriterState {
memtable,
last_flushed_wal_entry_position: initial_covered_wal_entry_position,
frozen_memtable_bytes: 0,
frozen_flush_watchers: VecDeque::new(),
frozen_memtables: VecDeque::new(),
flush_requested: false,
wal_flush_trigger_count: 0,
last_wal_flush_trigger_time: 0,
}));
let (memtable_flush_tx, memtable_flush_rx) = mpsc::unbounded_channel();
let flusher = Arc::new(MemTableFlusher::new(
object_store,
base_path,
base_uri,
shard_id,
manifest_store,
));
let backpressure = BackpressureController::new(config.clone());
// Background WAL flush handler — parallel WAL I/O + index updates.
let wal_handler =
WalFlushHandler::new(wal_flusher.clone(), Some(state.clone()), stats.clone());
task_executor.add_handler(
"wal_flusher".to_string(),
Box::new(wal_handler),
wal_flush_rx,
)?;
// Background MemTable flush handler — frozen memtable to Lance file.
let memtable_handler = MemTableFlushHandler::new(state.clone(), flusher, epoch, stats);
task_executor.add_handler(
"memtable_flusher".to_string(),
Box::new(memtable_handler),
memtable_flush_rx,
)?;
// Shared state used by `put()` to dispatch trigger checks.
let writer_state = Arc::new(SharedWriterState::new(
state.clone(),
wal_flusher,
wal_flush_tx,
memtable_flush_tx,
config.clone(),
schema.clone(),
pk_field_ids,
config.max_memtable_batches,
config.max_memtable_rows,
index_configs.to_vec(),
));
Ok(WriterMode::MemTable {
state,
writer_state,
backpressure,
})
}
fn open_wal_only_mode(
config: &ShardWriterConfig,
wal_flusher: Arc<WalFlusher>,
wal_flush_tx: mpsc::UnboundedSender<TriggerWalFlush>,
wal_flush_rx: mpsc::UnboundedReceiver<TriggerWalFlush>,
stats: SharedWriteStats,
task_executor: &Arc<TaskExecutor>,
) -> Result<WriterMode> {
// Background WAL flush handler — no MemTable state to consult, so
// pass `None` for the frozen-vs-active detection.
let wal_handler = WalFlushHandler::new(wal_flusher, None, stats);
task_executor.add_handler(
"wal_flusher".to_string(),
Box::new(wal_handler),
wal_flush_rx,
)?;
// Reuse `BackpressureController` (which is keyed off
// `max_unflushed_memtable_bytes`) as the WAL-only backpressure
// budget. WAL-only callers feed it `WalOnlyState::estimated_size()`.
// Keeps the config knob meaningful in WAL-only mode and prevents
// the pending queue from growing unbounded under non-durable writes.
let backpressure = BackpressureController::new(config.clone());
Ok(WriterMode::WalOnly {
state: Arc::new(WalOnlyState::default()),
wal_flush_tx,
trigger: StdRwLock::new(WalOnlyTriggerState::default()),
backpressure,
})
}
/// Write record batches to the shard.
///
/// All batches are inserted atomically with a single lock acquisition.
/// This is more efficient than calling put() multiple times for Arrow IPC
/// streams that contain multiple batches.
///
/// # Arguments
///
/// * `batches` - The record batches to write
///
/// # Returns
///
/// A WriteResult with batch position range and optional durability watcher.
///
/// # Note
///
/// Fencing is detected lazily during WAL flush via atomic writes.
/// If another writer has taken over, the WAL flush will fail with
/// `AlreadyExists`, indicating this writer has been fenced.
#[instrument(name = "sw_put", level = "info", skip_all, fields(batch_count = batches.len(), shard_id = %self.config.shard_id))]
pub async fn put(&self, batches: Vec<RecordBatch>) -> Result<WriteResult> {
if batches.is_empty() {
return Err(Error::invalid_input("Cannot write empty batch list"));
}
for (i, batch) in batches.iter().enumerate() {
if batch.num_rows() == 0 {
return Err(Error::invalid_input(format!("Batch {} is empty", i)));
}
}
match &self.mode {
WriterMode::MemTable {
state,
writer_state,
backpressure,
} => {
self.put_memtable(batches, state, writer_state, backpressure)
.await
}
WriterMode::WalOnly {
state,
wal_flush_tx,
trigger,
backpressure,
} => {
self.put_wal_only(batches, state, wal_flush_tx, trigger, backpressure)
.await
}
}
}
async fn put_memtable(
&self,
batches: Vec<RecordBatch>,
state_lock: &Arc<RwLock<WriterState>>,
writer_state: &Arc<SharedWriterState>,
backpressure: &BackpressureController,
) -> Result<WriteResult> {
// Apply backpressure if needed (before acquiring main lock)
backpressure
.maybe_apply_backpressure(|| {
(
writer_state.unflushed_memtable_bytes(),
writer_state.oldest_memtable_watcher(),
)
})
.await?;
let start = std::time::Instant::now();
// Acquire write lock for entire operation (atomic approach)
let (batch_positions, mut durable_watcher, batch_store, indexes) = {
let mut state = state_lock.write().await;
// 1. Insert all batches into memtable atomically
let results = state.memtable.insert_batches_only(batches).await?;
// Get batch position range
let start_pos = results.first().map(|(pos, _, _)| *pos).unwrap_or(0);
let end_pos = results.last().map(|(pos, _, _)| pos + 1).unwrap_or(0);
let batch_positions = start_pos..end_pos;
// 2. Track last batch for WAL durability
let durable_watcher = writer_state.track_batch_for_wal(end_pos.saturating_sub(1));
// 3. Check if WAL flush should be triggered
writer_state.maybe_trigger_wal_flush(&mut state);
// 4. Check if memtable flush is needed
if let Err(e) = writer_state.maybe_trigger_memtable_flush(&mut state) {
warn!("Failed to trigger memtable flush: {}", e);
}
// Get batch_store and indexes while we have the lock (for durable_write case)
let batch_store = state.memtable.batch_store();
let indexes = state.memtable.indexes_arc();
(batch_positions, durable_watcher, batch_store, indexes)
}; // Lock released here
self.stats.record_put(start.elapsed());
// Wait for durability if configured (outside the lock)
if self.config.durable_write {
self.wal_flusher.trigger_flush(
WalFlushSource::BatchStore {
batch_store,
indexes,
},
batch_positions.end,
None,
)?;
durable_watcher.wait().await?;
}
Ok(WriteResult { batch_positions })
}
async fn put_wal_only(
&self,
batches: Vec<RecordBatch>,
state: &Arc<WalOnlyState>,
wal_flush_tx: &mpsc::UnboundedSender<TriggerWalFlush>,
trigger: &StdRwLock<WalOnlyTriggerState>,
backpressure: &BackpressureController,
) -> Result<WriteResult> {
// Apply backpressure against the pending queue before pushing. The
// budget reuses `max_unflushed_memtable_bytes` since WAL-only mode
// shares the same "in-memory bytes waiting for durable storage"
// shape as MemTable mode. WAL-only mode has no per-frozen-MemTable
// watcher, so the backpressure loop falls back to its short sleep.
backpressure
.maybe_apply_backpressure(|| (state.estimated_size(), None))
.await?;
let start = std::time::Instant::now();
// Push batches into the pending queue and capture the assigned
// [start, end) range. `next_batch_position` is monotonic across the
// writer's lifetime; positions are not BatchStore indices but they
// are used the same way for durability tracking.
let batch_positions = state.push(batches);
// Time- and size-based triggers, mirroring MemTable mode but reading
// pending bytes from `WalOnlyState` instead of an active MemTable.
// Only fires for non-durable writes; durable writes go through the
// explicit done-cell path below so flush errors (e.g., fence) reach
// the caller.
if !self.config.durable_write {
let target_position = batch_positions.end;
let pending_bytes = state.estimated_size();
self.maybe_trigger_wal_flush_wal_only(
state,
wal_flush_tx,
trigger,
target_position,
pending_bytes,
);
}
self.stats.record_put(start.elapsed());
// For durable writes, trigger an immediate flush and wait for the
// done cell. Using the done cell instead of the durability watermark
// watcher ensures flush errors (e.g., the WalAppender returning a
// fence error) propagate back to `put` instead of hanging.
if self.config.durable_write {
let done = WatchableOnceCell::new();
let reader = done.reader();
self.wal_flusher.trigger_flush(
WalFlushSource::WalOnly {
state: state.clone(),
},
batch_positions.end,
Some(done),
)?;
let mut reader = reader;
match reader.await_value().await {
Some(Ok(_)) => {}
Some(Err(msg)) => return Err(Error::io(msg)),
None => {
return Err(Error::io(
"WAL flush handler exited before reporting durability",
));
}
}
}
Ok(WriteResult { batch_positions })
}
/// WAL-only-mode size+time trigger. Mirrors `SharedWriterState::maybe_trigger_wal_flush`
/// but reads its inputs from `WalOnlyState` (pending queue) instead of
/// the active MemTable.
fn maybe_trigger_wal_flush_wal_only(
&self,
state: &Arc<WalOnlyState>,
wal_flush_tx: &mpsc::UnboundedSender<TriggerWalFlush>,
trigger: &StdRwLock<WalOnlyTriggerState>,
end_batch_position: usize,
pending_bytes: usize,
) {
let threshold = self.config.max_wal_buffer_size;
let has_pending = state.batch_count() > 0;
let mut t = trigger.write().unwrap();
// Time-based trigger.
if let Some(interval) = self.config.max_wal_flush_interval {
let interval_millis = interval.as_millis() as u64;
let now = now_millis();
if t.last_wal_flush_trigger_time == 0 {
t.last_wal_flush_trigger_time = now;
} else {
let elapsed = now.saturating_sub(t.last_wal_flush_trigger_time);
if elapsed >= interval_millis && has_pending {
t.last_wal_flush_trigger_time = now;
let _ = wal_flush_tx.send(TriggerWalFlush {
source: WalFlushSource::WalOnly {
state: state.clone(),
},
end_batch_position,
done: None,
});
return;
}
}
}
if threshold == 0 {
return;
}
// Size-based trigger: fire one trigger per `max_wal_buffer_size`
// crossed since the last time we triggered. If the pending queue
// shrank below the recorded baseline (a drain happened), reset the
// baseline first so the next crossing fires correctly.
if pending_bytes < t.last_trigger_pending_bytes {
t.last_trigger_pending_bytes = 0;
}
while pending_bytes >= t.last_trigger_pending_bytes + threshold {
t.last_trigger_pending_bytes += threshold;
t.last_wal_flush_trigger_time = now_millis();
let _ = wal_flush_tx.send(TriggerWalFlush {
source: WalFlushSource::WalOnly {
state: state.clone(),
},
end_batch_position,
done: None,
});
}
}
/// Get a snapshot of current write statistics.
pub fn stats(&self) -> WriteStatsSnapshot {
self.stats.snapshot()
}
/// Get the shared stats handle (for external monitoring).
pub fn stats_handle(&self) -> SharedWriteStats {
self.stats.clone()
}
/// Get the current shard manifest.
pub async fn manifest(&self) -> Result<Option<ShardManifest>> {
self.manifest_store.read_latest().await
}
/// Get the writer's epoch.
pub fn epoch(&self) -> u64 {
self.epoch
}
/// Get the shard ID.
pub fn shard_id(&self) -> Uuid {
self.config.shard_id
}
/// Return `Err` if a successor writer has claimed a higher epoch.
pub async fn check_fenced(&self) -> Result<()> {
self.manifest_store.check_fenced(self.epoch).await
}
/// Get current MemTable statistics. Returns an error in WAL-only mode
/// (no MemTable exists).
pub async fn memtable_stats(&self) -> Result<MemTableStats> {
let state_lock = self.memtable_state_lock()?;
let state = state_lock.read().await;
let batch_store = state.memtable.batch_store();
let pending_wal = batch_store.pending_wal_flush_stats();
Ok(MemTableStats {
row_count: state.memtable.row_count(),
batch_count: state.memtable.batch_count(),
estimated_size: state.memtable.estimated_size(),
generation: state.memtable.generation(),
max_buffered_batch_position: batch_store.max_buffered_batch_position(),
max_flushed_batch_position: batch_store.max_flushed_batch_position(),
pending_wal_start_batch_position: pending_wal.start_batch_position,
pending_wal_end_batch_position: pending_wal.end_batch_position,
pending_wal_batch_count: pending_wal.batch_count,
pending_wal_row_count: pending_wal.row_count,
pending_wal_estimated_bytes: pending_wal.estimated_bytes,
})
}
/// Create a scanner for querying the current MemTable data.
///
/// The scanner provides read access to all data currently in the MemTable,
/// with optional filtering, projection, and index support.
///
/// The scanner captures the current `max_visible_batch_position` from the
/// `IndexStore` at construction time to ensure consistent visibility.
///
/// Returns an error in WAL-only mode.
pub async fn scan(&self) -> Result<MemTableScanner> {
let state_lock = self.memtable_state_lock()?;
let state = state_lock.read().await;
Ok(state.memtable.scan())
}
/// A handle to just the active memtable, for unified LSM scanning.
/// Prefer [`Self::in_memory_memtable_refs`] on the read path — it also
/// carries frozen-awaiting-flush generations.
///
/// Returns an error in WAL-only mode.
pub async fn active_memtable_ref(
&self,
) -> Result<crate::dataset::mem_wal::scanner::InMemoryMemTableRef> {
let state_lock = self.memtable_state_lock()?;
let state = state_lock.read().await;
Ok(in_memory_ref(&state.memtable))
}
/// The active memtable plus every frozen-awaiting-flush memtable,
/// captured atomically under one `state.read()` (no torn freeze).
/// Mirrors `WriterState { memtable, frozen_memtables }`. The WAL read
/// path uses this instead of [`Self::active_memtable_ref`] so a
/// concurrent reader sees no hole while a flush drains.
///
/// Returns an error in WAL-only mode.
pub async fn in_memory_memtable_refs(
&self,
) -> Result<crate::dataset::mem_wal::scanner::InMemoryMemTables> {
let state_lock = self.memtable_state_lock()?;
let state = state_lock.read().await;
Ok(crate::dataset::mem_wal::scanner::InMemoryMemTables {
active: in_memory_ref(&state.memtable),
frozen: state
.frozen_memtables
.iter()
.map(|m| in_memory_ref(m))
.collect(),
})
}
/// Returns the MemTable-mode state lock or a clear invalid_input error
/// when running in WAL-only mode.
fn memtable_state_lock(&self) -> Result<&Arc<RwLock<WriterState>>> {
match &self.mode {
WriterMode::MemTable { state, .. } => Ok(state),
WriterMode::WalOnly { .. } => Err(Error::invalid_input(
"MemTable accessor not available when enable_memtable = false (WAL-only mode)",
)),
}
}
/// Get WAL statistics.
pub fn wal_stats(&self) -> WalStats {
WalStats {
next_wal_entry_position: self.wal_flusher.next_wal_entry_position(),
}
}
/// Seal the active memtable so it's queued for L0 flush. No-op when
/// the active memtable is empty. Errors in WAL-only mode or if this
/// writer has been fenced by a successor. Pair with
/// [`Self::wait_for_flush_drain`] to wait for the queued flush.
///
/// Beyond test setup where deterministic flush points are required,
/// this is the primary lever for callers that need to drive flushes
/// out-of-band from the size/interval triggers — for example, to cap
/// resident memtable bytes across many shards in a multi-table WAL
/// writer process, or to drain the WAL ahead of a format change so
/// the next epoch starts with no replayable entries from the old
/// layout.
#[instrument(name = "sw_force_seal_active", level = "info", skip_all, fields(shard_id = %self.config.shard_id, epoch = self.epoch))]
pub async fn force_seal_active(&self) -> Result<()> {
match &self.mode {
WriterMode::MemTable {
state,
writer_state,
..
} => {
self.check_fenced().await?;
let mut state = state.write().await;
if state.memtable.batch_count() == 0 {
return Ok(());
}
writer_state.freeze_memtable(&mut state)?;
Ok(())
}
WriterMode::WalOnly { .. } => Err(Error::invalid_input(
"force_seal_active not available in WAL-only mode (no MemTable)",
)),
}
}
/// Block until every frozen memtable in the L0 flush queue has
/// landed and been recorded in the manifest. Does not wait on the
/// active memtable — call [`Self::force_seal_active`] first if you
/// want everything-on-disk. Errors in WAL-only mode, or if any
/// awaited flush reports `DurabilityResult::Failed`.
///
/// Useful in tests for deterministic post-flush assertions, and in
/// production wherever a caller needs a synchronous fence after
/// [`Self::force_seal_active`] — e.g. trimming memtable residency
/// across shards in a multi-table WAL writer, or ensuring the WAL
/// is fully drained to Lance storage before rolling to a new
/// format/epoch.
#[instrument(name = "sw_wait_for_flush_drain", level = "info", skip_all, fields(shard_id = %self.config.shard_id, epoch = self.epoch))]
pub async fn wait_for_flush_drain(&self) -> Result<()> {
let state_lock = match &self.mode {
WriterMode::MemTable { state, .. } => state,
WriterMode::WalOnly { .. } => {
return Err(Error::invalid_input(
"wait_for_flush_drain not available in WAL-only mode (no MemTable)",
));
}
};
loop {
let watchers: Vec<DurabilityWatcher> = {
let st = state_lock.read().await;
st.frozen_flush_watchers
.iter()
.map(|(_, w)| w.clone())
.collect()
};
if watchers.is_empty() {
return Ok(());
}
for mut w in watchers {
match w.await_value().await {
Some(durability) => durability.into_result()?,
None => {
return Err(Error::io(
"MemTable flush handler exited before reporting completion",
));
}
}
}
}
}
/// Close the writer gracefully.
///
/// Flushes pending data and shuts down background tasks.
#[instrument(name = "sw_close", level = "info", skip_all, fields(shard_id = %self.config.shard_id, epoch = self.epoch))]
pub async fn close(self) -> Result<()> {
info!("Closing ShardWriter for shard {}", self.config.shard_id);
match &self.mode {
WriterMode::MemTable {
state,
writer_state,
..
} => {
// Send final WAL flush message and wait for completion
let st = state.read().await;
let batch_store = st.memtable.batch_store();
let indexes = st.memtable.indexes_arc();
let batch_count = st.memtable.batch_count();
drop(st);
if batch_count > 0 {
let done = WatchableOnceCell::new();
let reader = done.reader();
if writer_state
.wal_flush_tx
.send(TriggerWalFlush {
source: WalFlushSource::BatchStore {
batch_store,
indexes,
},
end_batch_position: batch_count,
done: Some(done),
})
.is_ok()
{
let mut reader = reader;
let _ = reader.await_value().await;
}
}
// Freeze the active memtable (if any rows) so it joins the
// pending-flush queue, then wait for every frozen
// memtable's flush to complete. Without this, close() left
// any rows that hadn't crossed the size/batch trigger
// sitting in memory at shutdown — they were durable in the
// WAL but never produced a Lance fragment, which is
// surprising for callers who reasonably expect close() to
// make all data fully durable in the LSM sense (and which
// makes flush-cost benchmarks impossible to measure).
//
// Propagate any freeze error: at close time the caller
// has explicitly asked for full durability, so silently
// dropping a freeze failure would lose data without any
// signal. If freeze fails, surface the error rather than
// continuing on to drain only the pre-existing frozen
// memtables (whose flushes can still be waited on, but
// the caller now knows the close was incomplete).
let watchers: Vec<_> = {
let mut st = state.write().await;
if st.memtable.row_count() > 0 {
writer_state.freeze_memtable(&mut st)?;
}
st.frozen_flush_watchers
.iter()
.map(|(_, w)| w.clone())
.collect()
};
for mut w in watchers {
let _ = w.await_value().await;
}
}
WriterMode::WalOnly {
state,
wal_flush_tx,
trigger: _,
backpressure: _,
} => {
// Drain any pending batches via a final flush; wait for completion.
let pending = state.batch_count();
let end_position = state.next_batch_position();
if pending > 0 {
let done = WatchableOnceCell::new();
let reader = done.reader();
if wal_flush_tx
.send(TriggerWalFlush {
source: WalFlushSource::WalOnly {
state: state.clone(),
},
end_batch_position: end_position,
done: Some(done),
})
.is_ok()
{
let mut reader = reader;
let _ = reader.await_value().await;
}
}
}
}
// Shutdown background tasks
self.task_executor.shutdown_all().await?;
info!("ShardWriter closed for shard {}", self.config.shard_id);
Ok(())
}
}
/// MemTable statistics.
#[derive(Debug, Clone)]
pub struct MemTableStats {
pub row_count: usize,
pub batch_count: usize,
pub estimated_size: usize,
pub generation: u64,
pub max_buffered_batch_position: Option<usize>,
pub max_flushed_batch_position: Option<usize>,
pub pending_wal_start_batch_position: Option<usize>,
pub pending_wal_end_batch_position: Option<usize>,
pub pending_wal_batch_count: usize,
pub pending_wal_row_count: usize,
pub pending_wal_estimated_bytes: usize,
}
/// WAL statistics.
#[derive(Debug, Clone)]
pub struct WalStats {
/// Next WAL entry position to be used.
pub next_wal_entry_position: u64,
}
/// Background handler for WAL flush operations.
///
/// This handler does parallel WAL I/O + index updates during flush.
/// Indexes are passed through the TriggerWalFlush message.
struct WalFlushHandler {
wal_flusher: Arc<WalFlusher>,
/// MemTable-mode writer state, used to detect "frozen vs active" flushes
/// via Arc::ptr_eq on the active batch_store. `None` when running in
/// WAL-only mode (no MemTable, no frozen-vs-active distinction).
memtable_state: Option<Arc<RwLock<WriterState>>>,
stats: SharedWriteStats,
}
impl WalFlushHandler {
fn new(
wal_flusher: Arc<WalFlusher>,
memtable_state: Option<Arc<RwLock<WriterState>>>,
stats: SharedWriteStats,
) -> Self {
Self {
wal_flusher,
memtable_state,
stats,
}
}
}
#[async_trait]
impl MessageHandler<TriggerWalFlush> for WalFlushHandler {
async fn handle(&mut self, message: TriggerWalFlush) -> Result<()> {
let TriggerWalFlush {
source,
end_batch_position,
done,
} = message;
let result = self.do_flush(source, end_batch_position).await;
// Propagate the just-appended WAL entry position back into the
// writer state so a subsequent MemTable freeze can stamp the
// correct `covered_wal_entry_position` into the manifest. Without
// this, `replay_after_wal_entry_position` stays at 0 and replay
// re-reads already-flushed entries after restart.
//
// Always update state before signalling the completion cell so any
// waiter that reads state immediately after the cell fires sees
// the new position.
if let (Ok(flush_result), Some(state_lock)) = (&result, &self.memtable_state)
&& let Some(entry) = &flush_result.entry
{
let mut state = state_lock.write().await;
state.last_flushed_wal_entry_position =
state.last_flushed_wal_entry_position.max(entry.position);
}
// Notify completion if requested
if let Some(cell) = done {
cell.write(result.map_err(|e| e.to_string()));
}
Ok(())
}
}
impl WalFlushHandler {
/// Unified flush method for both active and frozen memtables and for
/// WAL-only mode.
///
/// For BatchStore sources, detects frozen vs active flush by comparing
/// the passed batch_store with the current active memtable's
/// batch_store. If different, it's a frozen memtable flush.
#[instrument(
name = "wal_do_flush",
level = "debug",
skip_all,
fields(end_batch_position)
)]
async fn do_flush(
&self,
source: WalFlushSource,
end_batch_position: usize,
) -> Result<WalFlushResult> {
let start = Instant::now();
// Whether this flush actually updates any in-memory indexes — only
// a BatchStore source carrying a non-empty `IndexStore` does. Used
// to gate the `record_index_update` stat so WAL-only flushes don't
// pollute the index-update counters.
let has_indexes = matches!(
&source,
WalFlushSource::BatchStore {
indexes: Some(_),
..
}
);
// Early-out for BatchStore sources where the watermark already
// covers the requested end position. Detection of "frozen flush"
// requires the active memtable's batch_store; WAL-only handlers
// don't have one (`memtable_state` is `None`) and never receive a
// BatchStore source, so the early-out simplifies to the watermark
// comparison.
if let WalFlushSource::BatchStore { batch_store, .. } = &source {
let max_flushed = batch_store.max_flushed_batch_position();
let flushed_up_to = max_flushed.map(|p| p + 1).unwrap_or(0);
let is_frozen_flush = if let Some(state_lock) = &self.memtable_state {
let state = state_lock.read().await;
!Arc::ptr_eq(batch_store, &state.memtable.batch_store())
} else {
false
};
if !is_frozen_flush && flushed_up_to >= end_batch_position {
return Ok(empty_flush_result());
}
}
// Delegate the actual flush to the WalFlusher.
let flush_result = self.wal_flusher.flush(&source, end_batch_position).await?;
let batches_flushed = flush_result
.entry
.as_ref()
.map(|e| e.num_batches)
.unwrap_or(0);
if batches_flushed > 0 {
self.stats
.record_wal_flush(start.elapsed(), flush_result.wal_bytes);
self.stats.record_wal_io(flush_result.wal_io_duration);
if has_indexes {
self.stats.record_index_update(
flush_result.index_update_duration,
flush_result.rows_indexed,
);
}
}
Ok(flush_result)
}
}
/// Background handler for MemTable flush operations.
///
/// This handler receives frozen memtables directly via messages and flushes them to Lance storage.
/// Freezing is done by the writer (via SharedWriterState::freeze_memtable) to ensure
/// immediate memtable switching, so writes can continue on the new memtable while this
/// handler flushes in the background.
struct MemTableFlushHandler {
state: Arc<RwLock<WriterState>>,
flusher: Arc<MemTableFlusher>,
epoch: u64,
stats: SharedWriteStats,
}
impl MemTableFlushHandler {
fn new(
state: Arc<RwLock<WriterState>>,
flusher: Arc<MemTableFlusher>,
epoch: u64,
stats: SharedWriteStats,
) -> Self {
Self {
state,
flusher,
epoch,
stats,
}
}
}
#[async_trait]
impl MessageHandler<TriggerMemTableFlush> for MemTableFlushHandler {
async fn handle(&mut self, message: TriggerMemTableFlush) -> Result<()> {
let TriggerMemTableFlush { memtable, done } = message;
let result = self.flush_memtable(memtable).await;
if let Some(tx) = done {
// Send result through the channel - caller is waiting for it
let _ = tx.send(result);
} else {
// No done channel, propagate errors
result?;
}
Ok(())
}
}
impl MemTableFlushHandler {
/// Flush the given frozen memtable to Lance storage.
///
/// This method waits for the WAL flush to complete (sent at freeze time),
/// then flushes to Lance storage. The WAL flush is already queued by
/// freeze_memtable to ensure strict ordering of WAL entries.
///
/// Whether the flush succeeds or fails, the memtable's flush-completion
/// watcher is always signaled and the backpressure queue is always drained
/// for this memtable. Otherwise `wait_for_flush_drain` would observe a
/// dropped watch channel and return `Err` instead of the actual outcome.
#[instrument(name = "mt_flush", level = "info", skip_all, fields(generation = memtable.generation(), row_count = memtable.row_count()))]
async fn flush_memtable(
&mut self,
memtable: Arc<MemTable>,
) -> Result<super::memtable::flush::FlushResult> {
let start = Instant::now();
let memtable_size = memtable.estimated_size();
let flush_result = async {
// Step 1: Wait for WAL flush completion (already queued at freeze time).
// The TriggerWalFlush message was sent by freeze_memtable to ensure
// strict ordering of WAL entries. If the freeze didn't trigger a
// flush (no pending WAL range), there's no completion cell and the
// memtable was already WAL-flushed by an earlier put.
let wal_flushed_position =
if let Some(mut completion_reader) = memtable.take_wal_flush_completion() {
match completion_reader.await_value().await {
Some(Ok(flush_result)) => flush_result.entry.map(|e| e.position),
Some(Err(e)) => return Err(Error::io(format!("WAL flush failed: {}", e))),
None => {
return Err(Error::io(
"WAL flush handler exited before reporting completion",
));
}
}
} else {
None
};
// Step 2: Flush the memtable to Lance storage. The covered WAL
// entry position is either the one we just appended (per-memtable,
// from the completion cell — authoritative even when concurrent
// flushes have raced ahead in `state.last_flushed_wal_entry_position`)
// or, when no flush was triggered at freeze time, the memtable's
// frozen-at marker captured at freeze. Stamping this into the
// manifest is what lets replay-on-reopen skip entries this
// generation covers.
let covered_wal_entry_position = wal_flushed_position
.or_else(|| memtable.frozen_at_wal_entry_position())
.unwrap_or(0);
self.flusher
.flush(&memtable, self.epoch, covered_wal_entry_position)
.await
}
.await;
// Step 3: Always signal completion (with the outcome) and drain
// backpressure state for this memtable, even on failure.
let durability = match &flush_result {
Ok(_) => DurabilityResult::ok(),
Err(e) => DurabilityResult::err(e.to_string()),
};
memtable.signal_memtable_flush_complete(durability);
{
let mut state = self.state.write().await;
// Backpressure drain: unconditional so `wait_for_flush_drain`
// sees the watcher's error signal, not a dropped channel.
if let Some((_size, _watcher)) = state.frozen_flush_watchers.pop_front() {
state.frozen_memtable_bytes =
state.frozen_memtable_bytes.saturating_sub(memtable_size);
}
// Drop the queryable handle ONLY on commit success. On failure
// keep it: rows must stay in the read union until a later flush
// or WAL replay, else a transient flush error reopens the hole.
// Keyed by generation, so non-FIFO completion is fine.
if flush_result.is_ok() {
let flushed_generation = memtable.generation();
state
.frozen_memtables
.retain(|m| m.generation() != flushed_generation);
}
}
let result = flush_result?;
self.stats
.record_memtable_flush(start.elapsed(), result.rows_flushed);
info!(
"Flushed frozen memtable generation {} ({} rows in {:?})",
result.generation.generation,
result.rows_flushed,
start.elapsed()
);
Ok(result)
}
}
// ============================================================================
// Write Statistics
// ============================================================================
/// Write performance statistics.
///
/// All fields use atomic operations for thread-safe updates.
/// Use `snapshot()` to get a consistent view of all stats.
#[derive(Debug, Default)]
pub struct WriteStats {
// Put operation stats
put_count: AtomicU64,
put_time_nanos: AtomicU64,
// WAL flush stats (total time = max(wal_io, index_update) due to parallel execution)
wal_flush_count: AtomicU64,
wal_flush_time_nanos: AtomicU64,
wal_flush_bytes: AtomicU64,
// WAL flush sub-component stats (for diagnosing bottlenecks)
wal_io_time_nanos: AtomicU64,
wal_io_count: AtomicU64,
index_update_time_nanos: AtomicU64,
index_update_count: AtomicU64,
index_update_rows: AtomicU64,
// MemTable flush stats
memtable_flush_count: AtomicU64,
memtable_flush_time_nanos: AtomicU64,
memtable_flush_rows: AtomicU64,
}
/// Snapshot of write statistics at a point in time.
#[derive(Debug, Clone)]
pub struct WriteStatsSnapshot {
pub put_count: u64,
pub put_time: Duration,
pub wal_flush_count: u64,
pub wal_flush_time: Duration,
pub wal_flush_bytes: u64,
// WAL flush sub-component stats
pub wal_io_time: Duration,
pub wal_io_count: u64,
pub index_update_time: Duration,
pub index_update_count: u64,
pub index_update_rows: u64,
pub memtable_flush_count: u64,
pub memtable_flush_time: Duration,
pub memtable_flush_rows: u64,
}
impl WriteStats {
/// Create a new stats collector.
pub fn new() -> Self {
Self::default()
}
/// Record a put operation.
pub fn record_put(&self, duration: Duration) {
self.put_count.fetch_add(1, Ordering::Relaxed);
self.put_time_nanos
.fetch_add(duration.as_nanos() as u64, Ordering::Relaxed);
}
/// Record a WAL flush operation (total time including parallel I/O and index).
pub fn record_wal_flush(&self, duration: Duration, bytes: usize) {
self.wal_flush_count.fetch_add(1, Ordering::Relaxed);
self.wal_flush_time_nanos
.fetch_add(duration.as_nanos() as u64, Ordering::Relaxed);
self.wal_flush_bytes
.fetch_add(bytes as u64, Ordering::Relaxed);
}
/// Record WAL I/O duration (sub-component of WAL flush).
pub fn record_wal_io(&self, duration: Duration) {
self.wal_io_count.fetch_add(1, Ordering::Relaxed);
self.wal_io_time_nanos
.fetch_add(duration.as_nanos() as u64, Ordering::Relaxed);
}
/// Record index update duration (sub-component of WAL flush).
pub fn record_index_update(&self, duration: Duration, rows: usize) {
self.index_update_count.fetch_add(1, Ordering::Relaxed);
self.index_update_time_nanos
.fetch_add(duration.as_nanos() as u64, Ordering::Relaxed);
self.index_update_rows
.fetch_add(rows as u64, Ordering::Relaxed);
}
/// Record a MemTable flush operation.
pub fn record_memtable_flush(&self, duration: Duration, rows: usize) {
self.memtable_flush_count.fetch_add(1, Ordering::Relaxed);
self.memtable_flush_time_nanos
.fetch_add(duration.as_nanos() as u64, Ordering::Relaxed);
self.memtable_flush_rows
.fetch_add(rows as u64, Ordering::Relaxed);
}
/// Get a snapshot of current statistics.
pub fn snapshot(&self) -> WriteStatsSnapshot {
WriteStatsSnapshot {
put_count: self.put_count.load(Ordering::Relaxed),
put_time: Duration::from_nanos(self.put_time_nanos.load(Ordering::Relaxed)),
wal_flush_count: self.wal_flush_count.load(Ordering::Relaxed),
wal_flush_time: Duration::from_nanos(self.wal_flush_time_nanos.load(Ordering::Relaxed)),
wal_flush_bytes: self.wal_flush_bytes.load(Ordering::Relaxed),
wal_io_time: Duration::from_nanos(self.wal_io_time_nanos.load(Ordering::Relaxed)),
wal_io_count: self.wal_io_count.load(Ordering::Relaxed),
index_update_time: Duration::from_nanos(
self.index_update_time_nanos.load(Ordering::Relaxed),
),
index_update_count: self.index_update_count.load(Ordering::Relaxed),
index_update_rows: self.index_update_rows.load(Ordering::Relaxed),
memtable_flush_count: self.memtable_flush_count.load(Ordering::Relaxed),
memtable_flush_time: Duration::from_nanos(
self.memtable_flush_time_nanos.load(Ordering::Relaxed),
),
memtable_flush_rows: self.memtable_flush_rows.load(Ordering::Relaxed),
}
}
/// Reset all statistics.
pub fn reset(&self) {
self.put_count.store(0, Ordering::Relaxed);
self.put_time_nanos.store(0, Ordering::Relaxed);
self.wal_flush_count.store(0, Ordering::Relaxed);
self.wal_flush_time_nanos.store(0, Ordering::Relaxed);
self.wal_flush_bytes.store(0, Ordering::Relaxed);
self.wal_io_time_nanos.store(0, Ordering::Relaxed);
self.wal_io_count.store(0, Ordering::Relaxed);
self.index_update_time_nanos.store(0, Ordering::Relaxed);
self.index_update_count.store(0, Ordering::Relaxed);
self.index_update_rows.store(0, Ordering::Relaxed);
self.memtable_flush_count.store(0, Ordering::Relaxed);
self.memtable_flush_time_nanos.store(0, Ordering::Relaxed);
self.memtable_flush_rows.store(0, Ordering::Relaxed);
}
}
impl WriteStatsSnapshot {
/// Get average put latency.
pub fn avg_put_latency(&self) -> Option<Duration> {
if self.put_count > 0 {
Some(self.put_time / self.put_count as u32)
} else {
None
}
}
/// Get put throughput (puts per second based on time spent in puts).
pub fn put_throughput(&self) -> f64 {
if self.put_time.as_secs_f64() > 0.0 {
self.put_count as f64 / self.put_time.as_secs_f64()
} else {
0.0
}
}
/// Get average WAL flush latency.
pub fn avg_wal_flush_latency(&self) -> Option<Duration> {
if self.wal_flush_count > 0 {
Some(self.wal_flush_time / self.wal_flush_count as u32)
} else {
None
}
}
/// Get average WAL flush size in bytes.
pub fn avg_wal_flush_bytes(&self) -> Option<u64> {
if self.wal_flush_count > 0 {
Some(self.wal_flush_bytes / self.wal_flush_count)
} else {
None
}
}
/// Get WAL write throughput (bytes per second based on WAL flush time).
pub fn wal_throughput_bytes(&self) -> f64 {
if self.wal_flush_time.as_secs_f64() > 0.0 {
self.wal_flush_bytes as f64 / self.wal_flush_time.as_secs_f64()
} else {
0.0
}
}
/// Get average WAL I/O latency.
pub fn avg_wal_io_latency(&self) -> Option<Duration> {
if self.wal_io_count > 0 {
Some(self.wal_io_time / self.wal_io_count as u32)
} else {
None
}
}
/// Get average index update latency.
pub fn avg_index_update_latency(&self) -> Option<Duration> {
if self.index_update_count > 0 {
Some(self.index_update_time / self.index_update_count as u32)
} else {
None
}
}
/// Get average rows per index update.
pub fn avg_index_update_rows(&self) -> Option<u64> {
if self.index_update_count > 0 {
Some(self.index_update_rows / self.index_update_count)
} else {
None
}
}
/// Get average MemTable flush latency.
pub fn avg_memtable_flush_latency(&self) -> Option<Duration> {
if self.memtable_flush_count > 0 {
Some(self.memtable_flush_time / self.memtable_flush_count as u32)
} else {
None
}
}
/// Get average MemTable flush size in rows.
pub fn avg_memtable_flush_rows(&self) -> Option<u64> {
if self.memtable_flush_count > 0 {
Some(self.memtable_flush_rows / self.memtable_flush_count)
} else {
None
}
}
/// Log stats summary using tracing (for structured telemetry).
pub fn log_summary(&self, prefix: &str) {
tracing::info!(
prefix = prefix,
put_count = self.put_count,
put_throughput = self.put_throughput(),
put_avg_latency_us = self.avg_put_latency().unwrap_or_default().as_micros() as u64,
wal_flush_count = self.wal_flush_count,
wal_flush_bytes = self.wal_flush_bytes,
wal_avg_latency_us =
self.avg_wal_flush_latency().unwrap_or_default().as_micros() as u64,
memtable_flush_count = self.memtable_flush_count,
memtable_flush_rows = self.memtable_flush_rows,
memtable_avg_latency_us = self
.avg_memtable_flush_latency()
.unwrap_or_default()
.as_micros() as u64,
"MemWAL stats summary"
);
}
/// Log detailed WAL flush breakdown (WAL I/O vs index update) using tracing.
pub fn log_wal_breakdown(&self, prefix: &str) {
if self.wal_flush_count > 0 {
tracing::info!(
prefix = prefix,
wal_total_latency_us =
self.avg_wal_flush_latency().unwrap_or_default().as_micros() as u64,
wal_io_latency_us =
self.avg_wal_io_latency().unwrap_or_default().as_micros() as u64,
index_update_latency_us = self
.avg_index_update_latency()
.unwrap_or_default()
.as_micros() as u64,
index_update_rows = self.index_update_rows,
"MemWAL WAL flush breakdown"
);
}
}
}
/// Shared stats handle for use across components.
pub type SharedWriteStats = Arc<WriteStats>;
/// Create a new shared stats collector.
pub fn new_shared_stats() -> SharedWriteStats {
Arc::new(WriteStats::new())
}
#[cfg(test)]
mod tests {
use super::*;
use arrow_array::{Int32Array, StringArray};
use arrow_schema::{DataType, Field};
use tempfile::TempDir;
async fn create_local_store() -> (Arc<ObjectStore>, Path, String, TempDir) {
let temp_dir = tempfile::tempdir().unwrap();
let uri = format!("file://{}", temp_dir.path().display());
let (store, path) = ObjectStore::from_uri(&uri).await.unwrap();
(store, path, uri, temp_dir)
}
fn create_test_schema() -> Arc<ArrowSchema> {
Arc::new(ArrowSchema::new(vec![
Field::new("id", DataType::Int32, false),
Field::new("name", DataType::Utf8, true),
]))
}
fn create_test_batch(schema: &ArrowSchema, start_id: i32, num_rows: usize) -> RecordBatch {
RecordBatch::try_new(
Arc::new(schema.clone()),
vec![
Arc::new(Int32Array::from_iter_values(
start_id..start_id + num_rows as i32,
)),
Arc::new(StringArray::from_iter_values(
(0..num_rows).map(|i| format!("name_{}", start_id as usize + i)),
)),
],
)
.unwrap()
}
#[tokio::test]
async fn test_shard_writer_basic_write() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: false,
sync_indexed_write: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: 64 * 1024 * 1024,
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(
store,
base_path,
base_uri,
config.clone(),
schema.clone(),
vec![],
)
.await
.unwrap();
// Write a batch
let batch = create_test_batch(&schema, 0, 10);
let result = writer.put(vec![batch]).await.unwrap();
assert_eq!(result.batch_positions, 0..1);
// Check stats
let stats = writer.memtable_stats().await.unwrap();
assert_eq!(stats.row_count, 10);
assert_eq!(stats.batch_count, 1);
// Close writer
writer.close().await.unwrap();
}
#[tokio::test]
async fn test_shard_writer_multiple_writes() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: false,
sync_indexed_write: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: 64 * 1024 * 1024,
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(store, base_path, base_uri, config, schema.clone(), vec![])
.await
.unwrap();
// Write multiple batches in a single put call
let batches: Vec<_> = (0..5)
.map(|i| create_test_batch(&schema, i * 10, 10))
.collect();
let result = writer.put(batches).await.unwrap();
assert_eq!(result.batch_positions, 0..5);
let stats = writer.memtable_stats().await.unwrap();
assert_eq!(stats.row_count, 50);
assert_eq!(stats.batch_count, 5);
writer.close().await.unwrap();
}
#[tokio::test]
async fn test_shard_writer_with_indexes() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: false,
sync_indexed_write: true,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: 64 * 1024 * 1024,
manifest_scan_batch_size: 2,
..Default::default()
};
let index_configs = vec![MemIndexConfig::BTree(BTreeIndexConfig {
name: "id_idx".to_string(),
field_id: 0,
column: "id".to_string(),
})];
let writer = ShardWriter::open(
store,
base_path,
base_uri,
config,
schema.clone(),
index_configs,
)
.await
.unwrap();
// Write a batch
let batch = create_test_batch(&schema, 0, 10);
writer.put(vec![batch]).await.unwrap();
let stats = writer.memtable_stats().await.unwrap();
assert_eq!(stats.row_count, 10);
writer.close().await.unwrap();
}
/// Test memtable auto-flush triggered by size threshold.
#[tokio::test]
async fn test_shard_writer_auto_flush_by_size() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
// Use a small memtable size to trigger auto-flush
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: false,
sync_indexed_write: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: 1024, // Very small - will trigger flush quickly
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(store, base_path, base_uri, config, schema.clone(), vec![])
.await
.unwrap();
let initial_gen = writer.memtable_stats().await.unwrap().generation;
// Write batches until auto-flush triggers
for i in 0..20 {
let batch = create_test_batch(&schema, i * 10, 10);
writer.put(vec![batch]).await.unwrap();
}
// Give time for background flush to process
tokio::time::sleep(tokio::time::Duration::from_millis(100)).await;
// Check that generation increased (indicating flush happened)
let stats = writer.memtable_stats().await.unwrap();
assert!(
stats.generation > initial_gen,
"Generation should increment after auto-flush"
);
writer.close().await.unwrap();
}
/// Regression for #6713: a single failing `handle()` must not kill
/// the dispatcher. Earlier the loop would `break Err(e)` on the
/// first message error, dropping the rx side and stranding
/// subsequent senders. The flusher tasks need to survive transient
/// errors so the writer keeps making forward progress.
#[tokio::test]
async fn test_task_dispatcher_survives_handle_error() {
use std::sync::atomic::{AtomicUsize, Ordering};
struct FlakyHandler {
call_count: Arc<AtomicUsize>,
}
#[async_trait]
impl MessageHandler<u32> for FlakyHandler {
async fn handle(&mut self, message: u32) -> Result<()> {
let n = self.call_count.fetch_add(1, Ordering::SeqCst);
if n == 0 {
Err(Error::io("first message intentionally fails"))
} else {
let _ = message;
Ok(())
}
}
}
let executor = TaskExecutor::new();
let call_count = Arc::new(AtomicUsize::new(0));
let (tx, rx) = mpsc::unbounded_channel::<u32>();
executor
.add_handler(
"flaky".to_string(),
Box::new(FlakyHandler {
call_count: call_count.clone(),
}),
rx,
)
.unwrap();
// Send three messages: the first errors, the next two should
// still be delivered to the (still-alive) handler.
tx.send(1).unwrap();
tx.send(2).unwrap();
tx.send(3).unwrap();
for _ in 0..50 {
if call_count.load(Ordering::SeqCst) >= 3 {
break;
}
tokio::time::sleep(tokio::time::Duration::from_millis(20)).await;
}
assert!(
call_count.load(Ordering::SeqCst) >= 3,
"dispatcher exited after the first error; only {} message(s) were handled",
call_count.load(Ordering::SeqCst)
);
executor.shutdown_all().await.ok();
}
/// Same as the local-fs test but against memory:// — closer to S3
/// semantics (conditional PUT, list-prefix consistency).
#[tokio::test]
async fn test_shard_writer_auto_flush_repeatedly_memory_store() {
let base_uri = "memory:///bench_test_flush";
let (store, base_path) = ObjectStore::from_uri(base_uri).await.unwrap();
let base_uri = base_uri.to_string();
let schema = create_test_schema();
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: true,
sync_indexed_write: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: 64,
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(store, base_path, base_uri, config, schema.clone(), vec![])
.await
.unwrap();
let initial_gen = writer.memtable_stats().await.unwrap().generation;
for i in 0..1000 {
let batch = create_test_batch(&schema, i * 10, 10);
writer.put(vec![batch]).await.unwrap();
}
tokio::time::sleep(tokio::time::Duration::from_secs(2)).await;
let stats = writer.memtable_stats().await.unwrap();
assert!(
stats.generation >= initial_gen + 50,
"expected many flushes; generation went {} → {}",
initial_gen,
stats.generation
);
writer.close().await.unwrap();
}
/// Regression for #6713: with durable_write=true and a memtable
/// size threshold that fires frequently, the memtable flush task
/// hit "Dataset already exists: …_gen_1" once the second flush
/// started.
#[tokio::test]
async fn test_shard_writer_auto_flush_repeatedly_stress() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: true,
sync_indexed_write: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
// Tiny size threshold — every batch crosses it.
max_memtable_size: 64,
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(store, base_path, base_uri, config, schema.clone(), vec![])
.await
.unwrap();
let initial_gen = writer.memtable_stats().await.unwrap().generation;
// Every put crosses the size threshold, so each one queues a
// freeze. We want to catch any bug where two flushes collide on
// path/generation. Drive 1000 puts so we get ≥ 100 flushes —
// enough rope for the bug to show up.
for i in 0..1000 {
let batch = create_test_batch(&schema, i * 10, 10);
writer.put(vec![batch]).await.unwrap();
}
tokio::time::sleep(tokio::time::Duration::from_secs(2)).await;
let stats = writer.memtable_stats().await.unwrap();
assert!(
stats.generation >= initial_gen + 50,
"expected many successful auto-flushes; generation went {} → {}",
initial_gen,
stats.generation
);
writer.close().await.unwrap();
}
/// Regression: `close()` must flush the active memtable (not just
/// drain the WAL). Earlier, with a `max_memtable_size` set well
/// above the workload, no auto-flush would fire and `close()`
/// would return without producing a Lance fragment — data was
/// durable in the WAL but no LSM-level generation existed,
/// surprising callers and making flush-cost benchmarks impossible.
///
/// The test verifies, end-to-end:
/// 1. close() returns Ok (a freeze/flush error must propagate, not
/// be silently dropped).
/// 2. The persisted shard manifest's `current_generation` has
/// advanced past the initial generation — direct evidence that
/// a MemTable flush + manifest commit happened during close()
/// rather than the active memtable being dropped on the floor.
/// (Verifying replay's post-flush behavior is tangled with
/// independent replay logic and is exercised by the dedicated
/// `test_memtable_replay_*` tests.)
#[tokio::test]
async fn test_close_flushes_active_memtable() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
// Reuse the same shard_id when reopening so we observe state
// produced by the first writer's `close()`.
let shard_id = Uuid::new_v4();
// Huge size threshold so puts never trigger an auto-flush.
let config = ShardWriterConfig {
shard_id,
shard_spec_id: 0,
durable_write: true,
sync_indexed_write: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: usize::MAX,
max_unflushed_memtable_bytes: usize::MAX,
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
config.clone(),
schema.clone(),
vec![],
)
.await
.unwrap();
let initial_gen = writer.memtable_stats().await.unwrap().generation;
for i in 0..50 {
let batch = create_test_batch(&schema, i * 10, 10);
writer.put(vec![batch]).await.unwrap();
}
// Pre-close sanity: no auto-flush should have fired.
let stats_before = writer.memtable_stats().await.unwrap();
assert_eq!(
stats_before.generation, initial_gen,
"no flush should have fired during puts (size threshold is usize::MAX)"
);
assert!(
stats_before.row_count > 0,
"memtable should hold the rows we just inserted"
);
// close() must succeed; any freeze/flush error must propagate.
writer
.close()
.await
.expect("close() must succeed and propagate any freeze/flush error");
// Reopen the same shard and read the persisted manifest. The
// active memtable from the closed writer was frozen + flushed
// inside close(), which must have committed a new manifest
// recording the advanced generation.
let reopened =
ShardWriter::open(store, base_path, base_uri, config, schema.clone(), vec![])
.await
.unwrap();
let manifest = reopened
.manifest()
.await
.unwrap()
.expect("reopened shard must have a persisted manifest");
assert!(
manifest.current_generation > initial_gen,
"expected manifest current_generation to advance past {} after close() flushed the active memtable; got {}",
initial_gen,
manifest.current_generation,
);
reopened.close().await.unwrap();
}
/// Regression: the memtable flush should successfully fire many
/// times in a row. A bug where every flush wrote the same path was
/// caught by lance-format/lance#6713.
#[tokio::test]
async fn test_shard_writer_auto_flush_repeatedly() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
// durable_write=true matches the LSM `merge_insert` defaults and
// is the configuration that surfaced #6713 in the wild.
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: true,
sync_indexed_write: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
// Tiny size threshold so a few batches cross it.
max_memtable_size: 1024,
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(store, base_path, base_uri, config, schema.clone(), vec![])
.await
.unwrap();
let initial_gen = writer.memtable_stats().await.unwrap().generation;
// Drive enough write traffic to trigger several auto-flushes.
// durable_write=true means each put waits for the WAL flush, so
// we don't need explicit yields between puts.
for i in 0..200 {
let batch = create_test_batch(&schema, i * 10, 10);
writer.put(vec![batch]).await.unwrap();
}
// Wait for the background memtable flushes to drain.
tokio::time::sleep(tokio::time::Duration::from_millis(500)).await;
// Generation should have advanced by at least 3 — i.e. we want to
// confirm multiple flushes succeeded back to back, not just one.
let stats = writer.memtable_stats().await.unwrap();
assert!(
stats.generation >= initial_gen + 3,
"expected ≥ 3 successful auto-flushes; generation went {} → {}",
initial_gen,
stats.generation
);
writer.close().await.unwrap();
}
#[tokio::test]
async fn test_no_backpressure_when_under_threshold() {
let config = ShardWriterConfig::default().with_max_unflushed_memtable_bytes(1024 * 1024); // 1MB
let controller = BackpressureController::new(config);
// Should return immediately - well under threshold (100 bytes < 1MB)
controller
.maybe_apply_backpressure(|| (100, None))
.await
.unwrap();
assert_eq!(controller.stats().count(), 0);
}
#[tokio::test]
async fn test_backpressure_loops_until_under_threshold() {
use std::sync::atomic::AtomicUsize;
use std::time::Duration;
let config = ShardWriterConfig::default()
.with_max_unflushed_memtable_bytes(100) // Very low threshold
.with_backpressure_log_interval(Duration::from_millis(50));
let controller = BackpressureController::new(config);
// Simulate: starts at 1000 bytes, drops by 400 each call (simulating memtable flushes)
let call_count = Arc::new(AtomicUsize::new(0));
let call_count_clone = call_count.clone();
controller
.maybe_apply_backpressure(move || {
let count = call_count_clone.fetch_add(1, std::sync::atomic::Ordering::Relaxed);
// 1000 -> 600 -> 200 -> under threshold (need 3 iterations)
let unflushed = 1000usize.saturating_sub(count * 400);
(unflushed, None)
})
.await
.unwrap();
// Should have called get_state 4 times (initial + 3 waits until under 100)
assert_eq!(call_count.load(std::sync::atomic::Ordering::Relaxed), 4);
// Should have recorded backpressure wait time (waited 3 times)
assert_eq!(controller.stats().count(), 1);
}
#[test]
fn test_record_put() {
let stats = WriteStats::new();
stats.record_put(Duration::from_millis(10));
stats.record_put(Duration::from_millis(20));
let snapshot = stats.snapshot();
assert_eq!(snapshot.put_count, 2);
assert_eq!(snapshot.put_time, Duration::from_millis(30));
assert_eq!(snapshot.avg_put_latency(), Some(Duration::from_millis(15)));
}
#[test]
fn test_record_wal_flush() {
let stats = WriteStats::new();
stats.record_wal_flush(Duration::from_millis(100), 1024);
stats.record_wal_flush(Duration::from_millis(200), 2048);
let snapshot = stats.snapshot();
assert_eq!(snapshot.wal_flush_count, 2);
assert_eq!(snapshot.wal_flush_time, Duration::from_millis(300));
assert_eq!(snapshot.wal_flush_bytes, 3072);
assert_eq!(snapshot.avg_wal_flush_bytes(), Some(1536));
}
#[test]
fn test_record_memtable_flush() {
let stats = WriteStats::new();
stats.record_memtable_flush(Duration::from_secs(1), 10000);
let snapshot = stats.snapshot();
assert_eq!(snapshot.memtable_flush_count, 1);
assert_eq!(snapshot.memtable_flush_time, Duration::from_secs(1));
assert_eq!(snapshot.memtable_flush_rows, 10000);
}
#[test]
fn test_stats_reset() {
let stats = WriteStats::new();
stats.record_put(Duration::from_millis(10));
stats.record_wal_flush(Duration::from_millis(100), 1024);
stats.reset();
let snapshot = stats.snapshot();
assert_eq!(snapshot.put_count, 0);
assert_eq!(snapshot.wal_flush_count, 0);
}
// ----- WAL-only mode (`enable_memtable = false`) -----
fn wal_only_config(shard_id: Uuid) -> ShardWriterConfig {
ShardWriterConfig {
shard_id,
shard_spec_id: 0,
durable_write: true,
enable_memtable: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
manifest_scan_batch_size: 2,
..Default::default()
}
}
#[tokio::test]
async fn test_wal_only_durable_round_trip() {
use crate::dataset::mem_wal::wal::WalTailer;
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let shard_id = Uuid::new_v4();
let writer = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri,
wal_only_config(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
// Two durable puts → two WAL entries (durable_write triggers an
// explicit flush per put).
let r1 = writer
.put(vec![create_test_batch(&schema, 0, 4)])
.await
.unwrap();
let r2 = writer
.put(vec![create_test_batch(&schema, 100, 2)])
.await
.unwrap();
assert_eq!(r1.batch_positions, 0..1);
assert_eq!(r2.batch_positions, 1..2);
writer.close().await.unwrap();
// Read back via WalTailer. WAL positions are 1-based, so two
// entries from a fresh shard land at 1 and 2.
let tailer = WalTailer::new(store, base_path, shard_id);
assert_eq!(tailer.first_position().await.unwrap(), 1);
assert_eq!(tailer.next_position().await.unwrap(), 3);
let e0 = tailer.read_entry(1).await.unwrap().unwrap();
let e1 = tailer.read_entry(2).await.unwrap().unwrap();
assert_eq!(e0.batches.len(), 1);
assert_eq!(e0.batches[0].num_rows(), 4);
assert_eq!(e1.batches.len(), 1);
assert_eq!(e1.batches[0].num_rows(), 2);
// Both entries from the same writer should have the same epoch.
assert_eq!(e0.writer_epoch, e1.writer_epoch);
assert!(e0.writer_epoch >= 1);
}
#[tokio::test]
async fn test_wal_only_rejects_index_configs() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let index_configs = vec![MemIndexConfig::BTree(BTreeIndexConfig {
name: "id_idx".to_string(),
field_id: 0,
column: "id".to_string(),
})];
let err = ShardWriter::open(
store,
base_path,
base_uri,
wal_only_config(Uuid::new_v4()),
schema,
index_configs,
)
.await
.err()
.expect("expected invalid_input");
assert!(
err.to_string().contains("indexes require enable_memtable"),
"unexpected error: {err}"
);
}
#[tokio::test]
async fn test_wal_only_rejects_empty_batches() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let writer = ShardWriter::open(
store,
base_path,
base_uri,
wal_only_config(Uuid::new_v4()),
schema.clone(),
vec![],
)
.await
.unwrap();
// Empty list.
let err = writer.put(vec![]).await.err().unwrap();
assert!(err.to_string().contains("empty batch list"));
// Single empty batch.
let zero = arrow_array::RecordBatch::new_empty(schema);
let err = writer.put(vec![zero]).await.err().unwrap();
assert!(err.to_string().contains("Batch 0 is empty"));
writer.close().await.unwrap();
}
#[tokio::test]
async fn test_wal_only_memtable_accessors_error() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let writer = ShardWriter::open(
store,
base_path,
base_uri,
wal_only_config(Uuid::new_v4()),
schema,
vec![],
)
.await
.unwrap();
let err = writer.memtable_stats().await.err().unwrap();
assert!(err.to_string().contains("WAL-only mode"));
let err = writer.scan().await.err().unwrap();
assert!(err.to_string().contains("WAL-only mode"));
let err = writer.active_memtable_ref().await.err().unwrap();
assert!(err.to_string().contains("WAL-only mode"));
let err = writer.in_memory_memtable_refs().await.err().unwrap();
assert!(err.to_string().contains("WAL-only mode"));
writer.close().await.unwrap();
}
#[tokio::test]
async fn test_wal_only_async_batches_multiple_puts() {
use crate::dataset::mem_wal::wal::WalTailer;
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let mut config = wal_only_config(Uuid::new_v4());
let shard_id = config.shard_id;
// Non-durable: puts should accumulate in the pending queue until
// close() drains them with a single WAL entry.
config.durable_write = false;
config.max_wal_flush_interval = None;
config.max_wal_buffer_size = 100 * 1024 * 1024; // never crossed
let writer = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri,
config,
schema.clone(),
vec![],
)
.await
.unwrap();
for i in 0..3 {
writer
.put(vec![create_test_batch(&schema, i * 10, 10)])
.await
.unwrap();
}
writer.close().await.unwrap();
// All three puts should be in a single WAL entry at position 1
// (WAL positions are 1-based).
let tailer = WalTailer::new(store, base_path, shard_id);
assert_eq!(tailer.next_position().await.unwrap(), 2);
let entry = tailer.read_entry(1).await.unwrap().unwrap();
assert_eq!(entry.batches.len(), 3);
for (i, batch) in entry.batches.iter().enumerate() {
assert_eq!(batch.num_rows(), 10, "batch {i}");
}
}
#[tokio::test]
async fn test_wal_only_fencing() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let shard_id = Uuid::new_v4();
let writer_a = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
wal_only_config(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
writer_a
.put(vec![create_test_batch(&schema, 0, 1)])
.await
.unwrap();
// Writer B claims a higher epoch and writes — fences A.
let writer_b = ShardWriter::open(
store,
base_path,
base_uri,
wal_only_config(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
assert!(writer_b.epoch() > writer_a.epoch());
writer_b
.put(vec![create_test_batch(&schema, 1, 1)])
.await
.unwrap();
// A's next durable put must fail with a fence error (the underlying
// WalAppender::append surfaces it via atomic_put).
let err = writer_a
.put(vec![create_test_batch(&schema, 2, 1)])
.await
.expect_err("expected fence error");
assert!(
err.to_string().contains("Writer fenced"),
"unexpected error: {err}"
);
writer_b.close().await.unwrap();
}
#[tokio::test]
async fn test_check_fenced_detects_successor_claim() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let shard_id = Uuid::new_v4();
let writer_a = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
wal_only_config(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
// Not yet fenced.
writer_a.check_fenced().await.unwrap();
// Successor claims a higher epoch.
let writer_b = ShardWriter::open(
store,
base_path,
base_uri,
wal_only_config(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
assert!(writer_b.epoch() > writer_a.epoch());
// A's check_fenced surfaces the fence without needing a put round-trip.
let err = writer_a
.check_fenced()
.await
.expect_err("expected fence error");
assert!(
err.to_string().contains("Writer fenced"),
"unexpected error: {err}"
);
// B is the current writer and is not fenced.
writer_b.check_fenced().await.unwrap();
writer_b.close().await.unwrap();
}
// ----- MemTable replay on open -----
fn memtable_config_with_pk(shard_id: Uuid) -> ShardWriterConfig {
ShardWriterConfig {
shard_id,
shard_spec_id: 0,
durable_write: true,
sync_indexed_write: false,
max_wal_buffer_size: 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: 64 * 1024 * 1024,
manifest_scan_batch_size: 2,
..Default::default()
}
}
fn schema_with_pk() -> Arc<ArrowSchema> {
use arrow_schema::Field;
// Mark `id` as the unenforced primary key.
let pk_meta: std::collections::HashMap<String, String> = [(
"lance-schema:unenforced-primary-key".to_string(),
"1".to_string(),
)]
.into_iter()
.collect();
let id_field = Field::new("id", DataType::Int32, false).with_metadata(pk_meta);
Arc::new(ArrowSchema::new(vec![
id_field,
Field::new("name", DataType::Utf8, true),
]))
}
/// Replay-on-open recovers durable WAL entries that were never flushed
/// to a Lance generation. Setup: writer A durably writes batches, drops
/// without close (so MemTable freeze never runs); writer B reopens and
/// must see A's rows in its MemTable scan.
#[tokio::test]
async fn test_memtable_replay_recovers_unflushed_writes() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = schema_with_pk();
let shard_id = Uuid::new_v4();
// Writer A: write two durable batches, then drop without close.
// The WAL files persist; the in-memory MemTable does not.
{
let writer_a = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
memtable_config_with_pk(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
writer_a
.put(vec![create_test_batch(&schema, 0, 5)])
.await
.unwrap();
writer_a
.put(vec![create_test_batch(&schema, 100, 3)])
.await
.unwrap();
// intentionally drop without close()
}
// Writer B reopens. Replay must rehydrate A's two batches into the
// active MemTable.
let writer_b = ShardWriter::open(
store,
base_path,
base_uri,
memtable_config_with_pk(shard_id),
schema,
vec![],
)
.await
.unwrap();
let stats = writer_b.memtable_stats().await.unwrap();
assert_eq!(
stats.row_count, 8,
"expected replay to insert 5 + 3 = 8 rows, got {}",
stats.row_count
);
assert_eq!(
stats.batch_count, 2,
"expected replay to insert 2 batches, got {}",
stats.batch_count
);
writer_b.close().await.unwrap();
}
/// Replay is a no-op on a fresh shard: the MemTable starts empty.
#[tokio::test]
async fn test_memtable_replay_no_op_on_fresh_shard() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = schema_with_pk();
let shard_id = Uuid::new_v4();
let writer = ShardWriter::open(
store,
base_path,
base_uri,
memtable_config_with_pk(shard_id),
schema,
vec![],
)
.await
.unwrap();
let stats = writer.memtable_stats().await.unwrap();
assert_eq!(stats.row_count, 0);
assert_eq!(stats.batch_count, 0);
writer.close().await.unwrap();
}
/// Regression for the OSS-WAL compactor-drain bug: after a flush
/// records its generation in the manifest and an external compactor
/// later drains `flushed_generations` back to empty (the legitimate
/// outcome of merging the generation into the base table), reopening
/// the writer must not re-replay the already-flushed WAL entry into
/// the active memtable.
///
/// Under the pre-fix logic, replay disambiguated "fresh shard" from
/// "flushed-then-compacted" with `flushed_generations.is_empty()`,
/// which collapsed both cases into start-at-0. With 1-based WAL
/// positions and a default cursor of 0 meaning "no flush stamped",
/// the flush-then-drain sequence leaves `replay_after_wal_entry_position`
/// pinned at the flushed position, so replay correctly starts past it.
#[tokio::test]
async fn test_memtable_replay_skips_entries_after_external_compaction() {
use crate::dataset::mem_wal::ShardManifestStore;
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = schema_with_pk();
let shard_id = Uuid::new_v4();
// Writer A: write 5 rows, close (forces a flush of the active
// memtable). The manifest now records a flushed generation and
// pins `replay_after_wal_entry_position` to the covered WAL entry.
{
let writer_a = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
memtable_config_with_pk(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
writer_a
.put(vec![create_test_batch(&schema, 0, 5)])
.await
.unwrap();
writer_a.close().await.unwrap();
}
// Simulate an external compactor merging the flushed generation
// into the base table: drain `flushed_generations` to empty via a
// direct manifest commit. The cursor stays where the flush put it.
let manifest_store = ShardManifestStore::new(store.clone(), &base_path, shard_id, 2);
let pre = manifest_store.read_latest().await.unwrap().unwrap();
assert!(
!pre.flushed_generations.is_empty(),
"writer A's close() should have stamped a flushed generation"
);
let cursor_at_flush = pre.replay_after_wal_entry_position;
assert!(
cursor_at_flush >= 1,
"expected cursor to land on a 1-based WAL position after flush, got {cursor_at_flush}"
);
// Bump the epoch (claim_epoch) so we can commit_update without
// being fenced; this also mirrors how a compactor process would
// hold its own writer claim.
let (compactor_epoch, _) = manifest_store.claim_epoch(pre.shard_spec_id).await.unwrap();
manifest_store
.commit_update(compactor_epoch, |current| ShardManifest {
version: current.version + 1,
flushed_generations: vec![],
..current.clone()
})
.await
.unwrap();
let post = manifest_store.read_latest().await.unwrap().unwrap();
assert!(
post.flushed_generations.is_empty(),
"compactor drain should have left flushed_generations empty"
);
assert_eq!(
post.replay_after_wal_entry_position, cursor_at_flush,
"compactor must not touch the replay cursor"
);
// Writer B reopens. Pre-fix: replay saw flushed_generations empty,
// restarted at WAL position 0, and re-inserted writer A's rows.
// Post-fix: replay starts at cursor + 1, finds no entry, and the
// memtable stays empty.
let writer_b = ShardWriter::open(
store,
base_path,
base_uri,
memtable_config_with_pk(shard_id),
schema,
vec![],
)
.await
.unwrap();
let stats = writer_b.memtable_stats().await.unwrap();
assert_eq!(
stats.row_count, 0,
"memtable must not re-replay compacted WAL entries; got {} rows",
stats.row_count
);
assert_eq!(stats.batch_count, 0);
writer_b.close().await.unwrap();
}
/// Replay aborts the open with a clear fence error if it encounters a
/// WAL entry written with an epoch strictly greater than ours. Simulate
/// the race where another writer wrote an entry with a higher epoch
/// between our `claim_epoch` and our replay by injecting a high-epoch
/// entry directly via `WalAppender::with_claimed_epoch` (which
/// bypasses `claim_epoch` and so does not bump the manifest).
#[tokio::test]
async fn test_memtable_replay_fenced_aborts_open() {
use crate::dataset::mem_wal::ShardManifestStore;
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = schema_with_pk();
let shard_id = Uuid::new_v4();
// Writer A: write one durable batch (claims epoch 1, writes entry at position 1).
{
let writer_a = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
memtable_config_with_pk(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
writer_a
.put(vec![create_test_batch(&schema, 0, 1)])
.await
.unwrap();
// drop without close
}
// Inject a WAL entry written with epoch 100 — far above whatever
// claim_epoch will hand the next opener. The manifest is not
// updated since we use `with_claimed_epoch` directly.
let manifest_store = Arc::new(ShardManifestStore::new(
store.clone(),
&base_path,
shard_id,
2,
));
let high_epoch_appender = WalAppender::with_claimed_epoch(
store.clone(),
base_path.clone(),
shard_id,
manifest_store,
100,
// hint seed irrelevant; the real position counter is discovered
// lazily on the first append.
0,
);
high_epoch_appender
.append(vec![create_test_batch(&schema, 999, 1)])
.await
.unwrap();
// Writer B opens. claim_epoch returns 2 (manifest's writer_epoch
// was 1 before this open). Replay reads the injected entry, sees
// epoch 100 > 2, and aborts with a fence error.
let result = ShardWriter::open(
store,
base_path,
base_uri,
memtable_config_with_pk(shard_id),
schema,
vec![],
)
.await;
let Err(err) = result else {
panic!("expected open to fail with fence error during replay");
};
let msg = err.to_string();
assert!(
msg.contains("WAL replay aborted") && msg.contains("fenced"),
"unexpected error: {msg}"
);
}
/// Regression: `wal_stats().next_wal_entry_position` must reflect the
/// post-recovery cursor immediately on reopen, not 0 until the first
/// append discovers the tip. Pre-fix the appender's hint was seeded at
/// 0 and only updated after the first successful append, so external
/// monitors saw 0 between open and first put on a shard with prior
/// entries.
#[tokio::test]
async fn test_wal_stats_seeded_from_manifest_on_reopen() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let shard_id = Uuid::new_v4();
// First writer creates a shard, writes one entry, closes.
let writer1 = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
wal_only_config(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
writer1
.put(vec![create_test_batch(&schema, 0, 1)])
.await
.unwrap();
writer1.close().await.unwrap();
// Reopen: stats must reflect the post-recovery cursor immediately,
// before any put has happened on this writer.
let writer2 = ShardWriter::open(
store,
base_path,
base_uri,
wal_only_config(shard_id),
schema,
vec![],
)
.await
.unwrap();
let next = writer2.wal_stats().next_wal_entry_position;
assert!(
next >= 1,
"expected wal_stats to reflect post-recovery cursor (>= 1) on reopen, got {next}"
);
writer2.close().await.unwrap();
}
/// Regression test for the size-based trigger after a drain.
///
/// Earlier the WAL-only size trigger used a monotonic counter
/// (`wal_flush_trigger_count`) which never reset across drains. After
/// the first crossing the counter was >= 1 and `pending_bytes / threshold`
/// could never grow past 1 (because pending_bytes resets on drain), so
/// the size trigger silently stopped firing. This test pushes batches
/// to cross the size threshold multiple times across drains and asserts
/// every crossing produces a WAL entry.
#[tokio::test]
async fn test_wal_only_size_trigger_fires_repeatedly() {
use crate::dataset::mem_wal::wal::WalTailer;
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let mut config = wal_only_config(Uuid::new_v4());
let shard_id = config.shard_id;
// Non-durable so puts don't auto-flush. Time trigger off so only
// the size trigger drives flushes.
config.durable_write = false;
config.max_wal_flush_interval = None;
// Pick a tiny threshold so a single batch crosses it.
config.max_wal_buffer_size = 1;
let writer = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri,
config,
schema.clone(),
vec![],
)
.await
.unwrap();
// Three puts, each large enough to cross the (1-byte) threshold.
// Without the fix, only the first would trigger; the rest would
// sit in the queue until close().
for i in 0..3 {
writer
.put(vec![create_test_batch(&schema, i * 10, 10)])
.await
.unwrap();
// Yield, then sleep, so the background flush handler can
// drain the trigger queue before the next push — otherwise
// multiple pending triggers can coalesce into a single drain.
// 50ms historically failed on slow Windows CI runners; 250ms
// gives a comfortable margin without making the suite slow.
tokio::task::yield_now().await;
tokio::time::sleep(std::time::Duration::from_millis(250)).await;
}
writer.close().await.unwrap();
// Each put should have produced its own WAL entry — three crossings,
// three entries. Without the regression fix, all three batches end
// up in a single entry written by `close()`.
let tailer = WalTailer::new(store, base_path, shard_id);
let next = tailer.next_position().await.unwrap();
assert!(
next >= 3,
"expected at least 3 WAL entries (one per crossing), got next_position = {next}"
);
}
/// Regression test for concurrent durable WAL-only puts on a fenced
/// writer. Earlier `flush_from_wal_only` did a destructive `drain()`
/// before calling `wal_appender.append`. If the append failed (e.g.
/// fence), the drained batches were dropped — the next concurrent put
/// would then see an empty pending queue and spuriously return Ok,
/// hiding the data loss. With the snapshot/commit fix, the failed flush
/// leaves the batches in the queue, and the concurrent put gets a clean
/// fence error too (when its own flush attempts the same WAL position).
#[tokio::test]
async fn test_wal_only_fenced_concurrent_puts_do_not_silently_succeed() {
use std::sync::Arc;
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let shard_id = Uuid::new_v4();
// Writer A claims epoch 1, writes one entry (takes WAL position 1,
// caches its next-position as 2 internally).
let writer_a = Arc::new(
ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
wal_only_config(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap(),
);
writer_a
.put(vec![create_test_batch(&schema, 0, 1)])
.await
.unwrap();
// Writer B claims epoch 2 and writes its own entry (takes WAL
// position 1). A is now fenced: A's next put will attempt WAL
// position 1 (its cached next), collide with B's entry, and
// surface a "Writer fenced" error from `check_fenced`.
let writer_b = ShardWriter::open(
store,
base_path,
base_uri,
wal_only_config(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
writer_b
.put(vec![create_test_batch(&schema, 1, 1)])
.await
.unwrap();
// Two concurrent durable puts on the (now-fenced) writer A. With
// the destructive-drain bug, the first flush would consume both
// pending batches into a failing append; the second flush would
// see an empty queue and return spurious success, silently losing
// the second put's data. With the snapshot/commit fix, the failed
// append leaves both batches in the queue and the second flush
// also fails with the fence error.
let a1 = writer_a.clone();
let a2 = writer_a.clone();
let schema1 = schema.clone();
let schema2 = schema.clone();
let h1 = tokio::spawn(async move { a1.put(vec![create_test_batch(&schema1, 2, 1)]).await });
let h2 = tokio::spawn(async move { a2.put(vec![create_test_batch(&schema2, 3, 1)]).await });
let r1 = h1.await.unwrap();
let r2 = h2.await.unwrap();
assert!(
r1.is_err() && r2.is_err(),
"expected both concurrent puts on a fenced writer to fail, got r1={r1:?} r2={r2:?}",
);
writer_b.close().await.unwrap();
}
#[tokio::test]
async fn test_wal_only_stats_no_memtable_flush() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let writer = ShardWriter::open(
store,
base_path,
base_uri,
wal_only_config(Uuid::new_v4()),
schema.clone(),
vec![],
)
.await
.unwrap();
writer
.put(vec![create_test_batch(&schema, 0, 1)])
.await
.unwrap();
let stats_handle = writer.stats_handle();
writer.close().await.unwrap();
let snapshot = stats_handle.snapshot();
assert!(snapshot.put_count >= 1, "expected at least one put");
assert!(
snapshot.wal_flush_count >= 1,
"expected at least one WAL flush"
);
assert_eq!(
snapshot.memtable_flush_count, 0,
"WAL-only mode must never trigger a memtable flush"
);
assert_eq!(
snapshot.index_update_count, 0,
"WAL-only mode must never trigger an index update"
);
}
#[tokio::test]
async fn test_force_seal_active_and_wait_for_flush_drain() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
// Thresholds high enough that auto-flush won't fire; the seal is
// the only thing that should rotate the memtable.
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: false,
sync_indexed_write: false,
max_wal_buffer_size: 64 * 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: 64 * 1024 * 1024,
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(store, base_path, base_uri, config, schema.clone(), vec![])
.await
.unwrap();
let initial_gen = writer.memtable_stats().await.unwrap().generation;
let flushed_before = writer
.manifest()
.await
.unwrap()
.map(|m| m.flushed_generations.len())
.unwrap_or(0);
writer
.put(vec![create_test_batch(&schema, 0, 10)])
.await
.unwrap();
writer.force_seal_active().await.unwrap();
writer.wait_for_flush_drain().await.unwrap();
let stats = writer.memtable_stats().await.unwrap();
assert_eq!(stats.generation, initial_gen + 1);
assert_eq!(stats.batch_count, 0);
let manifest = writer
.manifest()
.await
.unwrap()
.expect("manifest should exist after flush");
assert_eq!(manifest.flushed_generations.len(), flushed_before + 1);
writer.close().await.unwrap();
}
/// On a successful flush commit the sealed generation is dropped from
/// the queryable set (no leak), and its rows land in the manifest.
#[tokio::test]
async fn test_frozen_dropped_after_successful_flush() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let config = ShardWriterConfig {
shard_id: Uuid::new_v4(),
shard_spec_id: 0,
durable_write: false,
sync_indexed_write: false,
max_wal_buffer_size: 64 * 1024 * 1024,
max_wal_flush_interval: None,
max_memtable_size: 64 * 1024 * 1024,
manifest_scan_batch_size: 2,
..Default::default()
};
let writer = ShardWriter::open(store, base_path, base_uri, config, schema.clone(), vec![])
.await
.unwrap();
let initial_gen = writer.memtable_stats().await.unwrap().generation;
writer
.put(vec![create_test_batch(&schema, 0, 10)])
.await
.unwrap();
writer.force_seal_active().await.unwrap();
writer.wait_for_flush_drain().await.unwrap();
let refs = writer.in_memory_memtable_refs().await.unwrap();
assert!(
refs.frozen.is_empty(),
"frozen handle must be dropped once the flush commit lands"
);
assert_eq!(refs.active.generation, initial_gen + 1);
let manifest = writer.manifest().await.unwrap().expect("manifest exists");
assert!(
manifest
.flushed_generations
.iter()
.any(|g| g.generation == initial_gen),
"flushed generation must be recorded in the manifest"
);
writer.close().await.unwrap();
}
/// Regression: a transient flush failure must NOT reopen the
/// concurrent-read-vs-flush hole. The sealed generation stays in the
/// queryable set (rows intact) until a later flush or WAL replay.
/// Failure is induced deterministically by fencing the writer with a
/// successor before the seal, so the flush's `check_fenced` rejects it.
#[tokio::test]
async fn test_frozen_retained_after_failed_flush() {
let (store, base_path, base_uri, _temp_dir) = create_local_store().await;
let schema = create_test_schema();
let shard_id = Uuid::new_v4();
let writer_a = ShardWriter::open(
store.clone(),
base_path.clone(),
base_uri.clone(),
memtable_config_with_pk(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
let initial_gen = writer_a.memtable_stats().await.unwrap().generation;
writer_a
.put(vec![create_test_batch(&schema, 0, 10)])
.await
.unwrap();
// Successor claims a higher epoch, fencing A.
let writer_b = ShardWriter::open(
store,
base_path,
base_uri,
memtable_config_with_pk(shard_id),
schema.clone(),
vec![],
)
.await
.unwrap();
assert!(writer_b.epoch() > writer_a.epoch());
// `force_seal_active` would reject up-front on a fenced writer;
// freeze directly so the failure surfaces at flush-commit time —
// exactly the freeze/flush race the fix guards.
match &writer_a.mode {
WriterMode::MemTable {
state,
writer_state,
..
} => {
let mut st = state.write().await;
writer_state.freeze_memtable(&mut st).unwrap();
}
WriterMode::WalOnly { .. } => unreachable!("opened in memtable mode"),
}
// The fenced flush fails; the drain surfaces that error.
assert!(
writer_a.wait_for_flush_drain().await.is_err(),
"fenced flush should fail the drain"
);
// The hole did not reopen: the sealed generation is still queryable
// with its rows, alongside the new (empty) active generation.
let refs = writer_a.in_memory_memtable_refs().await.unwrap();
assert_eq!(refs.frozen.len(), 1, "sealed generation must be retained");
assert_eq!(refs.frozen[0].generation, initial_gen);
assert!(
!refs.frozen[0].batch_store.is_empty(),
"retained sealed memtable must still hold its rows"
);
assert_eq!(refs.active.generation, initial_gen + 1);
writer_b.close().await.unwrap();
}
}
#[cfg(test)]
mod shard_writer_tests {
use std::sync::Arc;
use crate::index::DatasetIndexExt;
use arrow_array::{
FixedSizeListArray, Float32Array, Int64Array, RecordBatch, RecordBatchIterator, StringArray,
};
use arrow_schema::{DataType, Field, Schema as ArrowSchema};
use lance_arrow::FixedSizeListArrayExt;
use lance_index::IndexType;
use lance_index::scalar::ScalarIndexParams;
use lance_index::scalar::inverted::InvertedIndexParams;
use lance_index::vector::ivf::IvfBuildParams;
use lance_index::vector::pq::builder::PQBuildParams;
use lance_linalg::distance::MetricType;
use uuid::Uuid;
use crate::dataset::mem_wal::DatasetMemWalExt;
use crate::dataset::{Dataset, WriteParams};
use crate::index::vector::VectorIndexParams;
use super::super::ShardWriterConfig;
fn create_test_schema(vector_dim: i32) -> Arc<ArrowSchema> {
use std::collections::HashMap;
let mut id_metadata = HashMap::new();
id_metadata.insert(
"lance-schema:unenforced-primary-key".to_string(),
"true".to_string(),
);
let id_field = Field::new("id", DataType::Int64, false).with_metadata(id_metadata);
Arc::new(ArrowSchema::new(vec![
id_field,
Field::new(
"vector",
DataType::FixedSizeList(
Arc::new(Field::new("item", DataType::Float32, true)),
vector_dim,
),
true,
),
Field::new("text", DataType::Utf8, true),
]))
}
fn create_append_only_schema(vector_dim: i32) -> Arc<ArrowSchema> {
Arc::new(ArrowSchema::new(vec![
Field::new("id", DataType::Int64, false),
Field::new(
"vector",
DataType::FixedSizeList(
Arc::new(Field::new("item", DataType::Float32, true)),
vector_dim,
),
true,
),
Field::new("text", DataType::Utf8, true),
]))
}
fn create_test_batch(
schema: &ArrowSchema,
start_id: i64,
num_rows: usize,
vector_dim: i32,
) -> RecordBatch {
let vectors: Vec<f32> = (0..num_rows)
.flat_map(|i| {
let seed = (start_id as usize + i) as f32;
(0..vector_dim as usize).map(move |d| (seed * 0.1 + d as f32 * 0.01).sin())
})
.collect();
let vector_array =
FixedSizeListArray::try_new_from_values(Float32Array::from(vectors), vector_dim)
.unwrap();
let texts: Vec<String> = (0..num_rows)
.map(|i| format!("Sample text for row {}", start_id as usize + i))
.collect();
RecordBatch::try_new(
Arc::new(schema.clone()),
vec![
Arc::new(Int64Array::from_iter_values(
start_id..start_id + num_rows as i64,
)),
Arc::new(vector_array),
Arc::new(StringArray::from_iter_values(texts)),
],
)
.unwrap()
}
#[tokio::test]
async fn test_initialize_mem_wal_records_writer_config_defaults() {
let vector_dim = 128;
let schema = create_test_schema(vector_dim);
let uri = format!("memory://test_writer_config_defaults_{}", Uuid::new_v4());
let initial_batch = create_test_batch(&schema, 0, 100, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
let writer_config = ShardWriterConfig::default()
.with_durable_write(false)
.with_max_memtable_size(8 * 1024 * 1024);
dataset
.initialize_mem_wal()
.writer_config_defaults(writer_config)
.add_writer_config_default("custom_knob", "custom_value")
.execute()
.await
.expect("Failed to initialize MemWAL");
// Defaults must survive the manifest round-trip so all writers share them.
let details = dataset
.mem_wal_index_details()
.await
.expect("Failed to read MemWAL index details")
.expect("MemWAL index details should exist");
let defaults = &details.writer_config_defaults;
// ShardWriterConfig tunables are recorded under their field names.
assert_eq!(
defaults.get("durable_write").map(String::as_str),
Some("false")
);
assert_eq!(
defaults.get("max_memtable_size").map(String::as_str),
Some("8388608")
);
// Duration knobs are recorded in milliseconds with a `_ms` suffix.
assert_eq!(
defaults
.get("max_wal_flush_interval_ms")
.map(String::as_str),
Some("100")
);
// Every tunable field is present.
assert!(defaults.contains_key("sync_indexed_write"));
assert!(defaults.contains_key("enable_memtable"));
assert!(defaults.contains_key("async_index_interval_ms"));
// add_writer_config_default records arbitrary keys.
assert_eq!(
defaults.get("custom_knob").map(String::as_str),
Some("custom_value")
);
// Shard identity is not a configuration default.
assert!(!defaults.contains_key("shard_id"));
assert!(!defaults.contains_key("shard_spec_id"));
}
#[tokio::test]
async fn test_initialize_mem_wal_bucket_sharding() {
let vector_dim = 128;
let schema = create_test_schema(vector_dim);
let uri = format!("memory://test_bucket_sharding_{}", Uuid::new_v4());
let initial_batch = create_test_batch(&schema, 0, 100, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
// num_buckets out of range is rejected.
let result = dataset
.initialize_mem_wal()
.bucket_sharding("id", 0)
.execute()
.await;
assert!(result.is_err(), "num_buckets = 0 should be rejected");
dataset
.initialize_mem_wal()
.bucket_sharding("text", 8)
.execute()
.await
.expect("Failed to initialize MemWAL");
let details = dataset
.mem_wal_index_details()
.await
.expect("Failed to read MemWAL index details")
.expect("MemWAL index details should exist");
assert_eq!(details.num_shards, 8);
assert_eq!(details.sharding_specs.len(), 1);
let field = &details.sharding_specs[0].fields[0];
assert_eq!(field.transform.as_deref(), Some("bucket"));
assert_eq!(
field.parameters.get("num_buckets").map(String::as_str),
Some("8")
);
assert_eq!(field.source_ids.len(), 1);
let source_id = field.source_ids[0];
let source_field = dataset.schema().field("text").expect("text field exists");
assert_eq!(source_id, source_field.id);
}
#[tokio::test]
async fn test_initialize_mem_wal_bucket_sharding_without_primary_key() {
let vector_dim = 128;
let schema = create_append_only_schema(vector_dim);
let uri = format!(
"memory://test_bucket_sharding_no_primary_key_{}",
Uuid::new_v4()
);
let initial_batch = create_test_batch(&schema, 0, 100, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
dataset
.initialize_mem_wal()
.bucket_sharding("id", 8)
.execute()
.await
.expect("Failed to initialize append-only MemWAL");
let details = dataset
.mem_wal_index_details()
.await
.expect("Failed to read MemWAL index details")
.expect("MemWAL index details should exist");
assert_eq!(details.num_shards, 8);
assert_eq!(details.sharding_specs.len(), 1);
let field = &details.sharding_specs[0].fields[0];
assert_eq!(field.transform.as_deref(), Some("bucket"));
assert_eq!(
field.parameters.get("num_buckets").map(String::as_str),
Some("8")
);
}
#[tokio::test]
async fn test_initialize_mem_wal_unsharded() {
let vector_dim = 128;
let schema = create_test_schema(vector_dim);
let uri = format!("memory://test_unsharded_{}", Uuid::new_v4());
let initial_batch = create_test_batch(&schema, 0, 100, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
dataset
.initialize_mem_wal()
.unsharded()
.execute()
.await
.expect("Failed to initialize MemWAL");
let details = dataset
.mem_wal_index_details()
.await
.expect("Failed to read MemWAL index details")
.expect("MemWAL index details should exist");
assert_eq!(details.num_shards, 1);
assert_eq!(details.sharding_specs.len(), 1);
assert_eq!(
details.sharding_specs[0].fields[0].transform.as_deref(),
Some("unsharded")
);
}
#[tokio::test]
async fn test_initialize_mem_wal_identity_sharding() {
let vector_dim = 128;
let schema = create_test_schema(vector_dim);
let uri = format!("memory://test_identity_sharding_{}", Uuid::new_v4());
let initial_batch = create_test_batch(&schema, 0, 100, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
// A column that does not exist is rejected.
let result = dataset
.initialize_mem_wal()
.identity_sharding("nonexistent")
.execute()
.await;
assert!(
result.is_err(),
"an unknown identity column should be rejected"
);
// A non-scalar column cannot be a shard key.
let result = dataset
.initialize_mem_wal()
.identity_sharding("vector")
.execute()
.await;
assert!(
result.is_err(),
"a non-scalar identity column should be rejected"
);
dataset
.initialize_mem_wal()
.identity_sharding("text")
.execute()
.await
.expect("Failed to initialize MemWAL");
let details = dataset
.mem_wal_index_details()
.await
.expect("Failed to read MemWAL index details")
.expect("MemWAL index details should exist");
// Identity sharding has an open-ended shard count.
assert_eq!(details.num_shards, 0);
assert_eq!(details.sharding_specs.len(), 1);
let field = &details.sharding_specs[0].fields[0];
assert_eq!(field.transform.as_deref(), Some("identity"));
assert_eq!(field.result_type.as_str(), "utf8");
assert_eq!(field.source_ids.len(), 1);
}
/// Quick smoke test for shard writer - runs against memory://
/// Run with: cargo test -p lance shard_writer_tests::test_shard_writer_smoke -- --nocapture
#[tokio::test]
async fn test_shard_writer_smoke() {
let vector_dim = 128;
let batch_size = 20;
let num_batches = 100;
let schema = create_test_schema(vector_dim);
let uri = format!("memory://test_shard_writer_{}", Uuid::new_v4());
// Create initial dataset
let initial_batch = create_test_batch(&schema, 0, 100, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
// Initialize MemWAL (no indexes for smoke test)
dataset
.initialize_mem_wal()
.execute()
.await
.expect("Failed to initialize MemWAL");
// Create shard writer
let shard_id = Uuid::new_v4();
let config = ShardWriterConfig::new(shard_id)
.with_durable_write(false)
.with_sync_indexed_write(false);
let writer = dataset
.mem_wal_writer(shard_id, config)
.await
.expect("Failed to create writer");
// Pre-generate batches
let batches: Vec<RecordBatch> = (0..num_batches)
.map(|i| create_test_batch(&schema, (i * batch_size) as i64, batch_size, vector_dim))
.collect();
// Write all batches in a single put call for efficiency
writer.put(batches).await.expect("Failed to write");
writer.close().await.expect("Failed to close");
}
#[tokio::test]
async fn test_shard_writer_with_vector_index_searches_active_memtable() {
let vector_dim = 32;
let batch_size = 20;
let target_id = 1_000i64 + 37;
let schema = create_test_schema(vector_dim);
let uri = format!("memory://test_shard_writer_hnsw_{}", Uuid::new_v4());
let initial_batch = create_test_batch(&schema, 0, 256, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
let vector_params = VectorIndexParams::ivf_flat(1, MetricType::L2);
dataset
.create_index(
&["vector"],
IndexType::Vector,
Some("vector_idx".to_string()),
&vector_params,
true,
)
.await
.expect("Failed to create base vector index");
dataset
.initialize_mem_wal()
.maintained_indexes(["vector_idx"])
.execute()
.await
.expect("Failed to initialize MemWAL");
let shard_id = Uuid::new_v4();
let config = ShardWriterConfig::new(shard_id)
.with_durable_write(true)
.with_sync_indexed_write(true);
let writer = dataset
.mem_wal_writer(shard_id, config)
.await
.expect("Failed to create writer");
let batches: Vec<RecordBatch> = (0..4)
.map(|i| {
create_test_batch(
&schema,
1_000 + (i * batch_size) as i64,
batch_size,
vector_dim,
)
})
.collect();
writer.put(batches).await.expect("Failed to write");
let query = Float32Array::from_iter_values(
(0..vector_dim as usize).map(|d| (target_id as f32 * 0.1 + d as f32 * 0.01).sin()),
);
let mut scanner = writer.scan().await.unwrap();
scanner.nearest("vector", Arc::new(query), 80);
let result = scanner.try_into_batch().await.expect("Failed to scan");
assert!(result.num_rows() > 0, "vector query returned no rows");
let id_col = result
.column_by_name("id")
.unwrap()
.as_any()
.downcast_ref::<Int64Array>()
.unwrap();
let dist_col = result
.column_by_name("_distance")
.unwrap()
.as_any()
.downcast_ref::<Float32Array>()
.unwrap();
let target_idx = (0..result.num_rows())
.find(|&idx| id_col.value(idx) == target_id)
.expect("target vector was not returned by active MemTable HNSW search");
assert!(
dist_col.value(target_idx) < 1e-6,
"expected self-match distance near zero, got {}",
dist_col.value(target_idx)
);
writer.close().await.expect("Failed to close");
}
/// Test shard writer against S3 with IVF-PQ, BTree, and FTS indexes (requires DATASET_PREFIX env var)
/// Run with: DATASET_PREFIX=s3://bucket/path cargo test -p lance --release shard_writer_tests::test_shard_writer_s3_ivfpq -- --nocapture --ignored
#[tokio::test]
#[ignore]
async fn test_shard_writer_s3_ivfpq() {
let prefix = std::env::var("DATASET_PREFIX").expect("DATASET_PREFIX not set");
let vector_dim = 512;
let batch_size = 20;
let num_batches = 10000;
let num_partitions = 16;
let num_sub_vectors = 64; // 512 / 8 = 64 subvectors
let schema = create_test_schema(vector_dim);
let uri = format!(
"{}/test_s3_{}",
prefix.trim_end_matches('/'),
Uuid::new_v4()
);
// Create initial dataset with enough data for IVF-PQ training
let initial_batch = create_test_batch(&schema, 0, 1000, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
// Create BTree index on id column
let scalar_params = ScalarIndexParams::default();
dataset
.create_index(
&["id"],
IndexType::BTree,
Some("id_btree".to_string()),
&scalar_params,
false,
)
.await
.expect("Failed to create BTree index");
// Create FTS index on text column
let fts_params = InvertedIndexParams::default();
dataset
.create_index(
&["text"],
IndexType::Inverted,
Some("text_fts".to_string()),
&fts_params,
false,
)
.await
.expect("Failed to create FTS index");
// Create IVF-PQ index on dataset
let ivf_params = IvfBuildParams {
num_partitions: Some(num_partitions),
..Default::default()
};
let pq_params = PQBuildParams {
num_sub_vectors,
num_bits: 8,
..Default::default()
};
let vector_params =
VectorIndexParams::with_ivf_pq_params(MetricType::L2, ivf_params, pq_params);
dataset
.create_index(
&["vector"],
IndexType::Vector,
Some("vector_idx".to_string()),
&vector_params,
true,
)
.await
.expect("Failed to create IVF-PQ index");
// Initialize MemWAL with all three indexes
dataset
.initialize_mem_wal()
.maintained_indexes(["id_btree", "text_fts", "vector_idx"])
.execute()
.await
.expect("Failed to initialize MemWAL");
// Create shard writer with default config
let shard_id = Uuid::new_v4();
let config = ShardWriterConfig::new(shard_id)
.with_durable_write(false)
.with_sync_indexed_write(false);
let writer = dataset
.mem_wal_writer(shard_id, config)
.await
.expect("Failed to create writer");
// Pre-generate batches
let batches: Vec<RecordBatch> = (0..num_batches)
.map(|i| create_test_batch(&schema, (i * batch_size) as i64, batch_size, vector_dim))
.collect();
// Write all batches in a single put call for efficiency
writer.put(batches).await.expect("Failed to write");
writer.close().await.expect("Failed to close");
}
/// End-to-end correctness test for ShardWriter with multiple memtable flushes.
///
/// This test verifies:
/// 1. Multiple memtable flushes are triggered via small memtable size
/// 2. File system layout is correct (WAL files, manifest, generation directories)
/// 3. WAL entries contain expected data
/// 4. Data can be read after each flush cycle
/// 5. Manifest tracks flushed generations correctly
///
/// Run with: cargo test -p lance shard_writer_tests::test_shard_writer_e2e_correctness -- --nocapture
#[tokio::test]
async fn test_shard_writer_e2e_correctness() {
use std::time::Duration;
use tempfile::TempDir;
let vector_dim = 32;
let rows_per_batch = 50;
// Write enough to trigger ~3 memtable flushes with 50KB memtable size
// Each batch is ~6KB (50 rows * 32 dims * 4 bytes/float + overhead)
let num_write_rounds = 3;
let batches_per_round = 3;
// Create temp directory for the test
let temp_dir = TempDir::new().expect("Failed to create temp dir");
let uri = format!("file://{}", temp_dir.path().display());
let schema = create_test_schema(vector_dim);
// Create initial dataset with enough rows for IVF-PQ training
let initial_batch = create_test_batch(&schema, 0, 500, vector_dim);
let batches = RecordBatchIterator::new([Ok(initial_batch)], schema.clone());
let mut dataset = Dataset::write(batches, &uri, Some(WriteParams::default()))
.await
.expect("Failed to create dataset");
// Create BTree index
dataset
.create_index(
&["id"],
IndexType::BTree,
Some("id_btree".to_string()),
&ScalarIndexParams::default(),
false,
)
.await
.expect("Failed to create BTree index");
// Initialize MemWAL with BTree index only (simpler for this test)
dataset
.initialize_mem_wal()
.maintained_indexes(["id_btree"])
.execute()
.await
.expect("Failed to initialize MemWAL");
// Create shard writer with small memtable size to trigger flushes
let shard_id = Uuid::new_v4();
let config = ShardWriterConfig::new(shard_id)
.with_durable_write(true) // Ensure WAL files are written
.with_sync_indexed_write(true)
.with_max_memtable_size(50 * 1024) // 50KB - triggers flush after ~8 batches
.with_max_wal_buffer_size(10 * 1024) // 10KB WAL buffer
.with_max_wal_flush_interval(Duration::from_millis(50)); // Fast flush
let writer = dataset
.mem_wal_writer(shard_id, config)
.await
.expect("Failed to create writer");
let mut total_rows_written = 0i64;
// Write data in rounds
for _round in 0..num_write_rounds {
let start_id = 500 + total_rows_written;
let batches_to_write: Vec<RecordBatch> = (0..batches_per_round)
.map(|i| {
create_test_batch(
&schema,
start_id + (i * rows_per_batch) as i64,
rows_per_batch,
vector_dim,
)
})
.collect();
writer.put(batches_to_write).await.expect("Failed to write");
total_rows_written += (batches_per_round * rows_per_batch) as i64;
// Give time for WAL flush and potential memtable flush
tokio::time::sleep(Duration::from_millis(150)).await;
}
// Close writer to ensure final flush
writer.close().await.expect("Failed to close");
// === VERIFY FILE SYSTEM LAYOUT ===
let mem_wal_dir = temp_dir.path().join("_mem_wal").join(shard_id.to_string());
assert!(mem_wal_dir.exists(), "MemWAL directory should exist");
// Check WAL directory
let wal_dir = mem_wal_dir.join("wal");
assert!(wal_dir.exists(), "WAL directory should exist");
let wal_files: Vec<_> = std::fs::read_dir(&wal_dir)
.expect("Failed to read WAL dir")
.filter_map(|e| e.ok())
.collect();
assert!(
!wal_files.is_empty(),
"WAL directory should contain at least one file"
);
// Check manifest directory
let manifest_dir = mem_wal_dir.join("manifest");
assert!(manifest_dir.exists(), "Manifest directory should exist");
let manifest_files: Vec<_> = std::fs::read_dir(&manifest_dir)
.expect("Failed to read manifest dir")
.filter_map(|e| e.ok())
.collect();
assert!(
!manifest_files.is_empty(),
"Manifest directory should contain at least one file"
);
// Read and verify manifest
let (store, base_path) = lance_io::object_store::ObjectStore::from_uri(&uri)
.await
.expect("Failed to open store");
let manifest_store =
super::super::manifest::ShardManifestStore::new(store, &base_path, shard_id, 2);
let manifest = manifest_store
.read_latest()
.await
.expect("Failed to read manifest")
.expect("Manifest should exist");
// Verify flushed generations exist on disk
assert!(
!manifest.flushed_generations.is_empty(),
"Should have at least one flushed generation"
);
for flushed_gen in &manifest.flushed_generations {
// The path stored in manifest is relative to the shard directory
// Construct full path: temp_dir/_mem_wal/shard_id/generation_folder
let gen_path = temp_dir
.path()
.join("_mem_wal")
.join(shard_id.to_string())
.join(&flushed_gen.path);
// The generation directory should exist
assert!(
gen_path.exists(),
"Flushed generation directory should exist at {:?}",
gen_path
);
// Verify generation directory has files
let gen_contents_count = std::fs::read_dir(&gen_path)
.expect("Failed to read gen dir")
.filter_map(|e| e.ok())
.count();
assert!(
gen_contents_count > 0,
"Generation directory should have files"
);
}
// === VERIFY WAL ENTRIES ===
// Verify WAL files have correct extension
for wal_file in wal_files.iter().take(1) {
let wal_path = wal_file.path();
let file_name = wal_path.file_name().unwrap().to_string_lossy();
assert!(
file_name.ends_with(".arrow"),
"WAL file should have .arrow extension"
);
}
// === VERIFY DATA CAN BE READ FROM NEW WRITER ===
// Re-open dataset and create new writer to verify recovery
let dataset = Dataset::open(&uri).await.expect("Failed to reopen dataset");
let new_shard_id = Uuid::new_v4();
let new_config = ShardWriterConfig::new(new_shard_id)
.with_durable_write(false)
.with_sync_indexed_write(true);
let new_writer = dataset
.mem_wal_writer(new_shard_id, new_config)
.await
.expect("Failed to create new writer");
// Write a test batch to verify the new shard works
let verify_batch = create_test_batch(&schema, 10000, 10, vector_dim);
new_writer
.put(vec![verify_batch])
.await
.expect("Failed to write to new shard");
let scanner = new_writer.scan().await.unwrap();
let result = scanner.try_into_batch().await.expect("Failed to scan");
assert_eq!(result.num_rows(), 10, "New shard should have 10 rows");
new_writer
.close()
.await
.expect("Failed to close new writer");
}
}