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
// Copyright (c) 2017-present, PingCAP, Inc. Licensed under Apache-2.0.

use std::cell::{Cell, RefCell};
use std::marker::PhantomData;
use std::path::Path;
use std::sync::{mpsc, Arc, Mutex};
use std::thread::{Builder as ThreadBuilder, JoinHandle};
use std::time::{Duration, Instant};

use log::{error, info};
use protobuf::{parse_from_bytes, Message};

use crate::config::{Config, RecoveryMode};
use crate::consistency::ConsistencyChecker;
use crate::env::{DefaultFileSystem, FileSystem};
use crate::event_listener::EventListener;
use crate::file_pipe_log::debug::LogItemReader;
use crate::file_pipe_log::{DefaultMachineFactory, FilePipeLog, FilePipeLogBuilder};
use crate::log_batch::{Command, LogBatch, MessageExt};
use crate::memtable::{EntryIndex, MemTableRecoverContextFactory, MemTables};
use crate::metrics::*;
use crate::pipe_log::{FileBlockHandle, FileId, LogQueue, PipeLog};
use crate::purge::{PurgeHook, PurgeManager};
use crate::write_barrier::{WriteBarrier, Writer};
use crate::{perf_context, Error, GlobalStats, Result};

const METRICS_FLUSH_INTERVAL: Duration = Duration::from_secs(30);
/// Max times for `write`.
const MAX_WRITE_ATTEMPT: u64 = 2;

pub struct Engine<F = DefaultFileSystem, P = FilePipeLog<F>>
where
    F: FileSystem,
    P: PipeLog,
{
    cfg: Arc<Config>,
    listeners: Vec<Arc<dyn EventListener>>,

    #[allow(dead_code)]
    stats: Arc<GlobalStats>,
    memtables: MemTables,
    pipe_log: Arc<P>,
    purge_manager: PurgeManager<P>,

    write_barrier: WriteBarrier<LogBatch, Result<FileBlockHandle>>,

    tx: Mutex<mpsc::Sender<()>>,
    metrics_flusher: Option<JoinHandle<()>>,

    _phantom: PhantomData<F>,
}

impl Engine<DefaultFileSystem, FilePipeLog<DefaultFileSystem>> {
    pub fn open(cfg: Config) -> Result<Engine<DefaultFileSystem, FilePipeLog<DefaultFileSystem>>> {
        Self::open_with_listeners(cfg, vec![])
    }

    pub fn open_with_listeners(
        cfg: Config,
        listeners: Vec<Arc<dyn EventListener>>,
    ) -> Result<Engine<DefaultFileSystem, FilePipeLog<DefaultFileSystem>>> {
        Self::open_with(cfg, Arc::new(DefaultFileSystem), listeners)
    }
}

impl<F> Engine<F, FilePipeLog<F>>
where
    F: FileSystem,
{
    pub fn open_with_file_system(
        cfg: Config,
        file_system: Arc<F>,
    ) -> Result<Engine<F, FilePipeLog<F>>> {
        Self::open_with(cfg, file_system, vec![])
    }

    pub fn open_with(
        mut cfg: Config,
        file_system: Arc<F>,
        mut listeners: Vec<Arc<dyn EventListener>>,
    ) -> Result<Engine<F, FilePipeLog<F>>> {
        cfg.sanitize()?;
        listeners.push(Arc::new(PurgeHook::default()) as Arc<dyn EventListener>);

        let start = Instant::now();
        let mut builder = FilePipeLogBuilder::new(cfg.clone(), file_system, listeners.clone());
        builder.scan()?;
        let factory = MemTableRecoverContextFactory::new(&cfg);
        let (append, rewrite) = builder.recover(&factory)?;
        let pipe_log = Arc::new(builder.finish()?);
        rewrite.merge_append_context(append);
        let (memtables, stats) = rewrite.finish();
        info!("Recovering raft logs takes {:?}", start.elapsed());

        let cfg = Arc::new(cfg);
        let purge_manager = PurgeManager::new(
            cfg.clone(),
            memtables.clone(),
            pipe_log.clone(),
            stats.clone(),
            listeners.clone(),
        );

        let (tx, rx) = mpsc::channel();
        let stats_clone = stats.clone();
        let memtables_clone = memtables.clone();
        let metrics_flusher = ThreadBuilder::new()
            .name("re-metrics".into())
            .spawn(move || loop {
                stats_clone.flush_metrics();
                memtables_clone.flush_metrics();
                if rx.recv_timeout(METRICS_FLUSH_INTERVAL).is_ok() {
                    break;
                }
            })?;

        Ok(Self {
            cfg,
            listeners,
            stats,
            memtables,
            pipe_log,
            purge_manager,
            write_barrier: Default::default(),
            tx: Mutex::new(tx),
            metrics_flusher: Some(metrics_flusher),
            _phantom: PhantomData,
        })
    }
}

impl<F, P> Engine<F, P>
where
    F: FileSystem,
    P: PipeLog,
{
    /// Writes the content of `log_batch` into the engine and returns written
    /// bytes. If `sync` is true, the write will be followed by a call to
    /// `fdatasync` on the log file.
    pub fn write(&self, log_batch: &mut LogBatch, mut sync: bool) -> Result<usize> {
        if log_batch.is_empty() {
            return Ok(0);
        }
        let start = Instant::now();
        let len = log_batch.finish_populate(
            self.cfg.batch_compression_threshold.0 as usize,
            self.cfg.compression_level,
        )?;
        debug_assert!(len > 0);

        let mut attempt_count = 0_u64;
        let block_handle = loop {
            // Max retry count is limited to `WRITE_MAX_RETRY_TIMES`, that is, 2.
            // If the first `append` retry because of NOSPC error, the next `append`
            // should success, unless there exists several abnormal cases in the IO device.
            // In that case, `Engine::write` must return `Err`.
            attempt_count += 1;
            let mut writer = Writer::new(log_batch, sync);
            // Snapshot and clear the current perf context temporarily, so the write group
            // leader will collect the perf context diff later.
            let mut perf_context = take_perf_context();
            let before_enter = Instant::now();
            if let Some(mut group) = self.write_barrier.enter(&mut writer) {
                let now = Instant::now();
                let _t = StopWatch::new_with(&*ENGINE_WRITE_LEADER_DURATION_HISTOGRAM, now);
                for writer in group.iter_mut() {
                    writer.entered_time = Some(now);
                    sync |= writer.sync;
                    let log_batch = writer.mut_payload();
                    let res = self.pipe_log.append(LogQueue::Append, log_batch);
                    writer.set_output(res);
                }
                perf_context!(log_write_duration).observe_since(now);
                if sync {
                    // As per trait protocol, this error should be retriable. But we panic anyway to
                    // save the trouble of propagating it to other group members.
                    self.pipe_log.sync(LogQueue::Append).expect("pipe::sync()");
                }
                // Pass the perf context diff to all the writers.
                let diff = get_perf_context();
                for writer in group.iter_mut() {
                    writer.perf_context_diff = diff.clone();
                }
            }
            let entered_time = writer.entered_time.unwrap();
            perf_context.write_wait_duration +=
                entered_time.saturating_duration_since(before_enter);
            debug_assert_eq!(writer.perf_context_diff.write_wait_duration, Duration::ZERO);
            perf_context += &writer.perf_context_diff;
            set_perf_context(perf_context);
            // Retry if `writer.finish()` returns a special 'Error::TryAgain', remarking
            // that there still exists free space for this `LogBatch`.
            match writer.finish() {
                Ok(handle) => {
                    ENGINE_WRITE_PREPROCESS_DURATION_HISTOGRAM
                        .observe(entered_time.saturating_duration_since(start).as_secs_f64());
                    break handle;
                }
                Err(Error::TryAgain(e)) => {
                    if attempt_count >= MAX_WRITE_ATTEMPT {
                        // A special err, we will retry this LogBatch `append` by appending
                        // this writer to the next write group, and the current write leader
                        // will not hang on this write and will return timely.
                        return Err(Error::TryAgain(format!(
                            "Failed to write logbatch, exceed MAX_WRITE_ATTEMPT: ({MAX_WRITE_ATTEMPT}), err: {e}",
                        )));
                    }
                    info!("got err: {e}, try to write this LogBatch again");
                }
                Err(e) => {
                    return Err(e);
                }
            }
        };
        let mut now = Instant::now();
        log_batch.finish_write(block_handle);
        self.memtables.apply_append_writes(log_batch.drain());
        for listener in &self.listeners {
            listener.post_apply_memtables(block_handle.id);
        }
        let end = Instant::now();
        let apply_duration = end.saturating_duration_since(now);
        ENGINE_WRITE_APPLY_DURATION_HISTOGRAM.observe(apply_duration.as_secs_f64());
        perf_context!(apply_duration).observe(apply_duration);
        now = end;
        ENGINE_WRITE_DURATION_HISTOGRAM.observe(now.saturating_duration_since(start).as_secs_f64());
        ENGINE_WRITE_SIZE_HISTOGRAM.observe(len as f64);
        Ok(len)
    }

    /// Synchronizes the Raft engine.
    pub fn sync(&self) -> Result<()> {
        self.write(&mut LogBatch::default(), true)?;
        Ok(())
    }

    pub fn get_message<S: Message>(&self, region_id: u64, key: &[u8]) -> Result<Option<S>> {
        let _t = StopWatch::new(&*ENGINE_READ_MESSAGE_DURATION_HISTOGRAM);
        if let Some(memtable) = self.memtables.get(region_id) {
            if let Some(value) = memtable.read().get(key) {
                return Ok(Some(parse_from_bytes(&value)?));
            }
        }
        Ok(None)
    }

    pub fn get(&self, region_id: u64, key: &[u8]) -> Option<Vec<u8>> {
        let _t = StopWatch::new(&*ENGINE_READ_MESSAGE_DURATION_HISTOGRAM);
        if let Some(memtable) = self.memtables.get(region_id) {
            return memtable.read().get(key);
        }
        None
    }

    /// Iterates over [start_key, end_key) range of Raft Group key-values and
    /// yields messages of the required type. Unparsable items are skipped.
    pub fn scan_messages<S, C>(
        &self,
        region_id: u64,
        start_key: Option<&[u8]>,
        end_key: Option<&[u8]>,
        reverse: bool,
        mut callback: C,
    ) -> Result<()>
    where
        S: Message,
        C: FnMut(&[u8], S) -> bool,
    {
        self.scan_raw_messages(region_id, start_key, end_key, reverse, move |k, raw_v| {
            if let Ok(v) = parse_from_bytes(raw_v) {
                callback(k, v)
            } else {
                true
            }
        })
    }

    /// Iterates over [start_key, end_key) range of Raft Group key-values and
    /// yields all key value pairs as bytes.
    pub fn scan_raw_messages<C>(
        &self,
        region_id: u64,
        start_key: Option<&[u8]>,
        end_key: Option<&[u8]>,
        reverse: bool,
        callback: C,
    ) -> Result<()>
    where
        C: FnMut(&[u8], &[u8]) -> bool,
    {
        let _t = StopWatch::new(&*ENGINE_READ_MESSAGE_DURATION_HISTOGRAM);
        if let Some(memtable) = self.memtables.get(region_id) {
            memtable
                .read()
                .scan(start_key, end_key, reverse, callback)?;
        }
        Ok(())
    }

    pub fn get_entry<M: MessageExt>(
        &self,
        region_id: u64,
        log_idx: u64,
    ) -> Result<Option<M::Entry>> {
        let _t = StopWatch::new(&*ENGINE_READ_ENTRY_DURATION_HISTOGRAM);
        if let Some(memtable) = self.memtables.get(region_id) {
            if let Some(idx) = memtable.read().get_entry(log_idx) {
                ENGINE_READ_ENTRY_COUNT_HISTOGRAM.observe(1.0);
                return Ok(Some(read_entry_from_file::<M, _>(
                    self.pipe_log.as_ref(),
                    &idx,
                )?));
            }
        }
        Ok(None)
    }

    /// Purges expired logs files and returns a set of Raft group ids that need
    /// to be compacted.
    pub fn purge_expired_files(&self) -> Result<Vec<u64>> {
        self.purge_manager.purge_expired_files()
    }

    /// Returns count of fetched entries.
    pub fn fetch_entries_to<M: MessageExt>(
        &self,
        region_id: u64,
        begin: u64,
        end: u64,
        max_size: Option<usize>,
        vec: &mut Vec<M::Entry>,
    ) -> Result<usize> {
        let _t = StopWatch::new(&*ENGINE_READ_ENTRY_DURATION_HISTOGRAM);
        if let Some(memtable) = self.memtables.get(region_id) {
            let mut ents_idx: Vec<EntryIndex> = Vec::with_capacity((end - begin) as usize);
            memtable
                .read()
                .fetch_entries_to(begin, end, max_size, &mut ents_idx)?;
            for i in ents_idx.iter() {
                vec.push(read_entry_from_file::<M, _>(self.pipe_log.as_ref(), i)?);
            }
            ENGINE_READ_ENTRY_COUNT_HISTOGRAM.observe(ents_idx.len() as f64);
            return Ok(ents_idx.len());
        }
        Ok(0)
    }

    pub fn first_index(&self, region_id: u64) -> Option<u64> {
        if let Some(memtable) = self.memtables.get(region_id) {
            return memtable.read().first_index();
        }
        None
    }

    pub fn last_index(&self, region_id: u64) -> Option<u64> {
        if let Some(memtable) = self.memtables.get(region_id) {
            return memtable.read().last_index();
        }
        None
    }

    /// Deletes log entries before `index` in the specified Raft group. Returns
    /// the number of deleted entries.
    pub fn compact_to(&self, region_id: u64, index: u64) -> u64 {
        let first_index = match self.first_index(region_id) {
            Some(index) => index,
            None => return 0,
        };

        let mut log_batch = LogBatch::default();
        log_batch.add_command(region_id, Command::Compact { index });
        if let Err(e) = self.write(&mut log_batch, false) {
            error!("Failed to write Compact command: {e}");
        }

        self.first_index(region_id).unwrap_or(index) - first_index
    }

    pub fn raft_groups(&self) -> Vec<u64> {
        self.memtables.fold(vec![], |mut v, m| {
            v.push(m.region_id());
            v
        })
    }

    /// Returns `true` if the engine contains no Raft Group. Empty Raft Group
    /// that isn't cleaned is counted as well.
    pub fn is_empty(&self) -> bool {
        self.memtables.is_empty()
    }

    /// Returns the sequence number range of active log files in the specific
    /// log queue.
    /// For testing only.
    pub fn file_span(&self, queue: LogQueue) -> (u64, u64) {
        self.pipe_log.file_span(queue)
    }

    pub fn get_used_size(&self) -> usize {
        self.pipe_log.total_size(LogQueue::Append) + self.pipe_log.total_size(LogQueue::Rewrite)
    }

    pub fn path(&self) -> &str {
        self.cfg.dir.as_str()
    }

    #[cfg(feature = "internals")]
    pub fn purge_manager(&self) -> &PurgeManager<P> {
        &self.purge_manager
    }
}

impl<F, P> Drop for Engine<F, P>
where
    F: FileSystem,
    P: PipeLog,
{
    fn drop(&mut self) {
        self.tx.lock().unwrap().send(()).unwrap();
        if let Some(t) = self.metrics_flusher.take() {
            t.join().unwrap();
        }
    }
}

impl Engine<DefaultFileSystem, FilePipeLog<DefaultFileSystem>> {
    pub fn consistency_check(path: &Path) -> Result<Vec<(u64, u64)>> {
        Self::consistency_check_with_file_system(path, Arc::new(DefaultFileSystem))
    }

    #[cfg(feature = "scripting")]
    pub fn unsafe_repair(path: &Path, queue: Option<LogQueue>, script: String) -> Result<()> {
        Self::unsafe_repair_with_file_system(path, queue, script, Arc::new(DefaultFileSystem))
    }

    pub fn dump(path: &Path) -> Result<LogItemReader<DefaultFileSystem>> {
        Self::dump_with_file_system(path, Arc::new(DefaultFileSystem))
    }
}

impl<F> Engine<F, FilePipeLog<F>>
where
    F: FileSystem,
{
    /// Returns a list of corrupted Raft groups, including their ids and last
    /// valid log index. Head or tail corruption cannot be detected.
    pub fn consistency_check_with_file_system(
        path: &Path,
        file_system: Arc<F>,
    ) -> Result<Vec<(u64, u64)>> {
        if !path.exists() {
            return Err(Error::InvalidArgument(format!(
                "raft-engine directory '{}' does not exist.",
                path.to_str().unwrap()
            )));
        }

        let cfg = Config {
            dir: path.to_str().unwrap().to_owned(),
            recovery_mode: RecoveryMode::TolerateAnyCorruption,
            ..Default::default()
        };
        let mut builder = FilePipeLogBuilder::new(cfg, file_system, Vec::new());
        builder.scan()?;
        let (append, rewrite) =
            builder.recover(&DefaultMachineFactory::<ConsistencyChecker>::default())?;
        let mut map = rewrite.finish();
        for (id, index) in append.finish() {
            map.entry(id).or_insert(index);
        }
        let mut list: Vec<(u64, u64)> = map.into_iter().collect();
        list.sort_unstable();
        Ok(list)
    }

    #[cfg(feature = "scripting")]
    pub fn unsafe_repair_with_file_system(
        path: &Path,
        queue: Option<LogQueue>,
        script: String,
        file_system: Arc<F>,
    ) -> Result<()> {
        use crate::file_pipe_log::{RecoveryConfig, ReplayMachine};

        if !path.exists() {
            return Err(Error::InvalidArgument(format!(
                "raft-engine directory '{}' does not exist.",
                path.to_str().unwrap()
            )));
        }

        let cfg = Config {
            dir: path.to_str().unwrap().to_owned(),
            recovery_mode: RecoveryMode::TolerateAnyCorruption,
            ..Default::default()
        };
        let recovery_mode = cfg.recovery_mode;
        let read_block_size = cfg.recovery_read_block_size.0;
        let mut builder = FilePipeLogBuilder::new(cfg, file_system.clone(), Vec::new());
        builder.scan()?;
        let factory = crate::filter::RhaiFilterMachineFactory::from_script(script);
        let mut machine = None;
        if queue.is_none() || queue.unwrap() == LogQueue::Append {
            machine = Some(builder.recover_queue(
                file_system.clone(),
                RecoveryConfig {
                    queue: LogQueue::Append,
                    mode: recovery_mode,
                    concurrency: 1,
                    read_block_size,
                },
                &factory,
            )?);
        }
        if queue.is_none() || queue.unwrap() == LogQueue::Rewrite {
            let machine2 = builder.recover_queue(
                file_system.clone(),
                RecoveryConfig {
                    queue: LogQueue::Rewrite,
                    mode: recovery_mode,
                    concurrency: 1,
                    read_block_size,
                },
                &factory,
            )?;
            if let Some(machine) = &mut machine {
                machine.merge(machine2, LogQueue::Rewrite)?;
            }
        }
        if let Some(machine) = machine {
            machine.finish(file_system.as_ref(), path)?;
        }
        Ok(())
    }

    /// Dumps all operations.
    pub fn dump_with_file_system(path: &Path, file_system: Arc<F>) -> Result<LogItemReader<F>> {
        if !path.exists() {
            return Err(Error::InvalidArgument(format!(
                "raft-engine directory or file '{}' does not exist.",
                path.to_str().unwrap()
            )));
        }

        if path.is_dir() {
            LogItemReader::new_directory_reader(file_system, path)
        } else {
            LogItemReader::new_file_reader(file_system, path)
        }
    }
}

struct BlockCache {
    key: Cell<FileBlockHandle>,
    block: RefCell<Vec<u8>>,
}

impl BlockCache {
    fn new() -> Self {
        BlockCache {
            key: Cell::new(FileBlockHandle {
                id: FileId::new(LogQueue::Append, 0),
                offset: 0,
                len: 0,
            }),
            block: RefCell::new(Vec::new()),
        }
    }

    fn insert(&self, key: FileBlockHandle, block: Vec<u8>) {
        self.key.set(key);
        self.block.replace(block);
    }
}

thread_local! {
    static BLOCK_CACHE: BlockCache = BlockCache::new();
}

pub(crate) fn read_entry_from_file<M, P>(pipe_log: &P, idx: &EntryIndex) -> Result<M::Entry>
where
    M: MessageExt,
    P: PipeLog,
{
    BLOCK_CACHE.with(|cache| {
        if cache.key.get() != idx.entries.unwrap() {
            cache.insert(
                idx.entries.unwrap(),
                LogBatch::decode_entries_block(
                    &pipe_log.read_bytes(idx.entries.unwrap())?,
                    idx.entries.unwrap(),
                    idx.compression_type,
                )?,
            );
        }
        let e = parse_from_bytes(
            &cache.block.borrow()
                [idx.entry_offset as usize..(idx.entry_offset + idx.entry_len) as usize],
        )?;
        assert_eq!(M::index(&e), idx.index);
        Ok(e)
    })
}

pub(crate) fn read_entry_bytes_from_file<P>(pipe_log: &P, idx: &EntryIndex) -> Result<Vec<u8>>
where
    P: PipeLog,
{
    BLOCK_CACHE.with(|cache| {
        if cache.key.get() != idx.entries.unwrap() {
            cache.insert(
                idx.entries.unwrap(),
                LogBatch::decode_entries_block(
                    &pipe_log.read_bytes(idx.entries.unwrap())?,
                    idx.entries.unwrap(),
                    idx.compression_type,
                )?,
            );
        }
        Ok(cache.block.borrow()
            [idx.entry_offset as usize..(idx.entry_offset + idx.entry_len) as usize]
            .to_owned())
    })
}

#[cfg(test)]
pub(crate) mod tests {
    use super::*;
    use crate::env::{ObfuscatedFileSystem, Permission};
    use crate::file_pipe_log::{parse_reserved_file_name, FileNameExt};
    use crate::log_batch::AtomicGroupBuilder;
    use crate::pipe_log::Version;
    use crate::test_util::{generate_entries, PanicGuard};
    use crate::util::ReadableSize;
    use kvproto::raft_serverpb::RaftLocalState;
    use raft::eraftpb::Entry;
    use std::collections::{BTreeSet, HashSet};
    use std::fs::OpenOptions;
    use std::path::PathBuf;

    pub(crate) type RaftLogEngine<F = DefaultFileSystem> = Engine<F>;
    impl<F: FileSystem> RaftLogEngine<F> {
        fn append(&self, rid: u64, start_index: u64, end_index: u64, data: Option<&[u8]>) {
            let entries = generate_entries(start_index, end_index, data);
            if !entries.is_empty() {
                let mut batch = LogBatch::default();
                batch.add_entries::<Entry>(rid, &entries).unwrap();
                batch
                    .put_message(
                        rid,
                        b"last_index".to_vec(),
                        &RaftLocalState {
                            last_index: entries[entries.len() - 1].index,
                            ..Default::default()
                        },
                    )
                    .unwrap();
                self.write(&mut batch, true).unwrap();
            }
        }

        fn clean(&self, rid: u64) {
            let mut log_batch = LogBatch::default();
            log_batch.add_command(rid, Command::Clean);
            self.write(&mut log_batch, true).unwrap();
        }

        fn decode_last_index(&self, rid: u64) -> Option<u64> {
            self.get_message::<RaftLocalState>(rid, b"last_index")
                .unwrap()
                .map(|s| s.last_index)
        }

        fn reopen(self) -> Self {
            let cfg: Config = self.cfg.as_ref().clone();
            let file_system = self.pipe_log.file_system();
            let mut listeners = self.listeners.clone();
            listeners.pop();
            drop(self);
            RaftLogEngine::open_with(cfg, file_system, listeners).unwrap()
        }

        fn scan_entries<FR: Fn(u64, LogQueue, &[u8])>(
            &self,
            rid: u64,
            start: u64,
            end: u64,
            reader: FR,
        ) {
            let mut entries = Vec::new();
            self.fetch_entries_to::<Entry>(
                rid,
                self.first_index(rid).unwrap(),
                self.last_index(rid).unwrap() + 1,
                None,
                &mut entries,
            )
            .unwrap();
            assert_eq!(entries.first().unwrap().index, start, "{rid}");
            assert_eq!(entries.last().unwrap().index + 1, end);
            assert_eq!(
                entries.last().unwrap().index,
                self.decode_last_index(rid).unwrap()
            );
            assert_eq!(entries.len(), (end - start) as usize);
            for e in entries.iter() {
                let entry_index = self
                    .memtables
                    .get(rid)
                    .unwrap()
                    .read()
                    .get_entry(e.index)
                    .unwrap();
                assert_eq!(&self.get_entry::<Entry>(rid, e.index).unwrap().unwrap(), e);
                reader(e.index, entry_index.entries.unwrap().id.queue, &e.data);
            }
        }

        fn file_count(&self, queue: Option<LogQueue>) -> usize {
            if let Some(queue) = queue {
                let (a, b) = self.file_span(queue);
                (b - a + 1) as usize
            } else {
                self.file_count(Some(LogQueue::Append)) + self.file_count(Some(LogQueue::Rewrite))
            }
        }
    }

    #[test]
    fn test_empty_engine() {
        let dir = tempfile::Builder::new()
            .prefix("test_empty_engine")
            .tempdir()
            .unwrap();
        let mut sub_dir = PathBuf::from(dir.as_ref());
        sub_dir.push("raft-engine");
        let cfg = Config {
            dir: sub_dir.to_str().unwrap().to_owned(),
            ..Default::default()
        };
        RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
            .unwrap();
    }

    #[test]
    fn test_get_entry() {
        let normal_batch_size = 10;
        let compressed_batch_size = 5120;
        for &entry_size in &[normal_batch_size, compressed_batch_size] {
            let dir = tempfile::Builder::new()
                .prefix("test_get_entry")
                .tempdir()
                .unwrap();
            let cfg = Config {
                dir: dir.path().to_str().unwrap().to_owned(),
                target_file_size: ReadableSize(1),
                ..Default::default()
            };

            let engine = RaftLogEngine::open_with_file_system(
                cfg.clone(),
                Arc::new(ObfuscatedFileSystem::default()),
            )
            .unwrap();
            assert_eq!(engine.path(), dir.path().to_str().unwrap());
            let data = vec![b'x'; entry_size];
            for i in 10..20 {
                let rid = i;
                let index = i;
                engine.append(rid, index, index + 2, Some(&data));
            }
            for i in 10..20 {
                let rid = i;
                let index = i;
                engine.scan_entries(rid, index, index + 2, |_, q, d| {
                    assert_eq!(q, LogQueue::Append);
                    assert_eq!(d, &data);
                });
            }

            // Recover the engine.
            let engine = engine.reopen();
            for i in 10..20 {
                let rid = i;
                let index = i;
                engine.scan_entries(rid, index, index + 2, |_, q, d| {
                    assert_eq!(q, LogQueue::Append);
                    assert_eq!(d, &data);
                });
            }
        }
    }

    #[test]
    fn test_clean_raft_group() {
        fn run_steps(steps: &[Option<(u64, u64)>]) {
            let rid = 1;
            let data = vec![b'x'; 1024];

            for rewrite_step in 1..=steps.len() {
                for exit_purge in [None, Some(1), Some(2)] {
                    let _guard = PanicGuard::with_prompt(format!(
                        "case: [{steps:?}, {rewrite_step}, {exit_purge:?}]",
                    ));
                    let dir = tempfile::Builder::new()
                        .prefix("test_clean_raft_group")
                        .tempdir()
                        .unwrap();
                    let cfg = Config {
                        dir: dir.path().to_str().unwrap().to_owned(),
                        target_file_size: ReadableSize(1),
                        ..Default::default()
                    };
                    let engine = RaftLogEngine::open_with_file_system(
                        cfg.clone(),
                        Arc::new(ObfuscatedFileSystem::default()),
                    )
                    .unwrap();

                    for (i, step) in steps.iter().enumerate() {
                        if let Some((start, end)) = *step {
                            engine.append(rid, start, end, Some(&data));
                        } else {
                            engine.clean(rid);
                        }
                        if i + 1 == rewrite_step {
                            engine
                                .purge_manager
                                .must_rewrite_append_queue(None, exit_purge);
                        }
                    }

                    let engine = engine.reopen();
                    if let Some((start, end)) = *steps.last().unwrap() {
                        engine.scan_entries(rid, start, end, |_, _, d| {
                            assert_eq!(d, &data);
                        });
                    } else {
                        assert!(engine.raft_groups().is_empty());
                    }

                    engine.purge_manager.must_rewrite_append_queue(None, None);
                    let engine = engine.reopen();
                    if let Some((start, end)) = *steps.last().unwrap() {
                        engine.scan_entries(rid, start, end, |_, _, d| {
                            assert_eq!(d, &data);
                        });
                    } else {
                        assert!(engine.raft_groups().is_empty());
                    }
                }
            }
        }

        run_steps(&[Some((1, 5)), None, Some((2, 6)), None, Some((3, 7)), None]);
        run_steps(&[Some((1, 5)), None, Some((2, 6)), None, Some((3, 7))]);
        run_steps(&[Some((1, 5)), None, Some((2, 6)), None]);
        run_steps(&[Some((1, 5)), None, Some((2, 6))]);
        run_steps(&[Some((1, 5)), None]);
    }

    #[test]
    fn test_key_value_scan() {
        fn key(i: u64) -> Vec<u8> {
            format!("k{i}").as_bytes().to_vec()
        }
        fn value(i: u64) -> Vec<u8> {
            format!("v{i}").as_bytes().to_vec()
        }
        fn rich_value(i: u64) -> RaftLocalState {
            RaftLocalState {
                last_index: i,
                ..Default::default()
            }
        }

        let dir = tempfile::Builder::new()
            .prefix("test_key_value_scan")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            ..Default::default()
        };
        let rid = 1;
        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();

        engine
            .scan_messages::<RaftLocalState, _>(rid, None, None, false, |_, _| {
                panic!("unexpected message.");
            })
            .unwrap();

        let mut batch = LogBatch::default();
        let mut res = Vec::new();
        let mut rich_res = Vec::new();
        batch.put(rid, key(1), value(1)).unwrap();
        batch.put(rid, key(2), value(2)).unwrap();
        batch.put(rid, key(3), value(3)).unwrap();
        engine.write(&mut batch, false).unwrap();

        engine
            .scan_raw_messages(rid, None, None, false, |k, v| {
                res.push((k.to_vec(), v.to_vec()));
                true
            })
            .unwrap();
        assert_eq!(
            res,
            vec![(key(1), value(1)), (key(2), value(2)), (key(3), value(3))]
        );
        res.clear();
        engine
            .scan_raw_messages(rid, None, None, true, |k, v| {
                res.push((k.to_vec(), v.to_vec()));
                true
            })
            .unwrap();
        assert_eq!(
            res,
            vec![(key(3), value(3)), (key(2), value(2)), (key(1), value(1))]
        );
        res.clear();
        engine
            .scan_messages::<RaftLocalState, _>(rid, None, None, false, |_, _| {
                panic!("unexpected message.")
            })
            .unwrap();

        batch.put_message(rid, key(22), &rich_value(22)).unwrap();
        batch.put_message(rid, key(33), &rich_value(33)).unwrap();
        engine.write(&mut batch, false).unwrap();

        engine
            .scan_messages(rid, None, None, false, |k, v| {
                rich_res.push((k.to_vec(), v));
                false
            })
            .unwrap();
        assert_eq!(rich_res, vec![(key(22), rich_value(22))]);
        rich_res.clear();
        engine
            .scan_messages(rid, None, None, true, |k, v| {
                rich_res.push((k.to_vec(), v));
                false
            })
            .unwrap();
        assert_eq!(rich_res, vec![(key(33), rich_value(33))]);
        rich_res.clear();
    }

    #[test]
    fn test_delete_key_value() {
        let dir = tempfile::Builder::new()
            .prefix("test_delete_key_value")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            ..Default::default()
        };
        let rid = 1;
        let key = b"key".to_vec();
        let (v1, v2) = (b"v1".to_vec(), b"v2".to_vec());
        let mut batch_1 = LogBatch::default();
        batch_1.put(rid, key.clone(), v1).unwrap();
        let mut batch_2 = LogBatch::default();
        batch_2.put(rid, key.clone(), v2.clone()).unwrap();
        let mut delete_batch = LogBatch::default();
        delete_batch.delete(rid, key.clone());

        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();
        assert_eq!(
            engine.get_message::<RaftLocalState>(rid, &key).unwrap(),
            None
        );
        assert_eq!(engine.get(rid, &key), None);

        // put | delete
        //     ^ rewrite
        engine.write(&mut batch_1.clone(), true).unwrap();
        assert!(engine.get_message::<RaftLocalState>(rid, &key).is_err());
        engine.purge_manager.must_rewrite_append_queue(None, None);
        engine.write(&mut delete_batch.clone(), true).unwrap();
        let engine = engine.reopen();
        assert_eq!(engine.get(rid, &key), None);
        assert_eq!(
            engine.get_message::<RaftLocalState>(rid, &key).unwrap(),
            None
        );

        // Incomplete purge.
        engine.write(&mut batch_1.clone(), true).unwrap();
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        engine.write(&mut delete_batch.clone(), true).unwrap();
        let engine = engine.reopen();
        assert_eq!(engine.get(rid, &key), None);

        // TODO: Preserve kv tombstone during rewrite and activate this test case.
        // put | delete |
        //              ^ rewrite
        // let engine = engine.reopen();
        // engine.write(&mut batch_1.clone(), true).unwrap();
        // engine.write(&mut delete_batch.clone(), true).unwrap();
        // engine.purge_manager.must_rewrite_append_queue(None, None);
        // let engine = engine.reopen();
        // assert_eq!(engine.get(rid, &key), None);

        // put | delete | put
        //     ^ rewrite
        let engine = engine.reopen();
        engine.write(&mut batch_1.clone(), true).unwrap();
        engine.purge_manager.must_rewrite_append_queue(None, None);
        engine.write(&mut delete_batch.clone(), true).unwrap();
        engine.write(&mut batch_2.clone(), true).unwrap();
        let engine = engine.reopen();
        assert_eq!(engine.get(rid, &key).unwrap(), v2);
        // Incomplete purge.
        engine.write(&mut batch_1.clone(), true).unwrap();
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        engine.write(&mut delete_batch.clone(), true).unwrap();
        engine.write(&mut batch_2.clone(), true).unwrap();
        let engine = engine.reopen();
        assert_eq!(engine.get(rid, &key).unwrap(), v2);

        // put | delete | put
        //              ^ rewrite
        let engine = engine.reopen();
        engine.write(&mut batch_1.clone(), true).unwrap();
        engine.write(&mut delete_batch.clone(), true).unwrap();
        engine.purge_manager.must_rewrite_append_queue(None, None);
        engine.write(&mut batch_2.clone(), true).unwrap();
        let engine = engine.reopen();
        assert_eq!(engine.get(rid, &key).unwrap(), v2);
        // Incomplete purge.
        engine.write(&mut batch_1.clone(), true).unwrap();
        engine.write(&mut delete_batch.clone(), true).unwrap();
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        engine.write(&mut batch_2.clone(), true).unwrap();
        let engine = engine.reopen();
        assert_eq!(engine.get(rid, &key).unwrap(), v2);

        // put | delete | put |
        //                    ^ rewrite
        let engine = engine.reopen();
        engine.write(&mut batch_1.clone(), true).unwrap();
        engine.write(&mut delete_batch.clone(), true).unwrap();
        engine.write(&mut batch_2.clone(), true).unwrap();
        engine.purge_manager.must_rewrite_append_queue(None, None);
        let engine = engine.reopen();
        assert_eq!(engine.get(rid, &key).unwrap(), v2);
        // Incomplete purge.
        let engine = engine.reopen();
        engine.write(&mut batch_1.clone(), true).unwrap();
        engine.write(&mut delete_batch.clone(), true).unwrap();
        engine.write(&mut batch_2.clone(), true).unwrap();
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        let engine = engine.reopen();
        assert_eq!(engine.get(rid, &key).unwrap(), v2);
    }

    #[test]
    fn test_compact_raft_group() {
        let dir = tempfile::Builder::new()
            .prefix("test_compact_raft_group")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            ..Default::default()
        };
        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();
        let data = vec![b'x'; 1024];

        // rewrite:[1  ..10]
        // append:   [5..10]
        let mut rid = 7;
        engine.append(rid, 1, 10, Some(&data));
        // Files are not purged.
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        let mut compact_log = LogBatch::default();
        compact_log.add_command(rid, Command::Compact { index: 5 });
        engine.write(&mut compact_log, true).unwrap();
        let engine = engine.reopen();
        engine.scan_entries(rid, 5, 10, |_, q, d| {
            assert_eq!(q, LogQueue::Append);
            assert_eq!(d, &data);
        });
        assert_eq!(engine.stats.live_entries(LogQueue::Append), 6); // 5 entries + 1 kv

        // rewrite:   [20..25]
        // append: [10   ..25]
        rid += 1;
        engine.append(rid, 5, 15, Some(&data));
        let mut compact_log = LogBatch::default();
        compact_log.add_command(rid, Command::Compact { index: 10 });
        engine.write(&mut compact_log, true).unwrap();
        engine.append(rid, 15, 25, Some(&data));
        // Files are not purged.
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        // Simulate loss of buffered write.
        let mut compact_log = LogBatch::default();
        compact_log.add_command(rid, Command::Compact { index: 20 });
        engine.memtables.apply_append_writes(compact_log.drain());
        engine.purge_manager.must_rewrite_rewrite_queue();
        let engine = engine.reopen();
        engine.scan_entries(rid, 10, 25, |_, q, d| {
            assert_eq!(q, LogQueue::Append);
            assert_eq!(d, &data);
        });
        assert_eq!(engine.stats.live_entries(LogQueue::Append), 22); // 20 entries + 2 kv
        engine.clean(rid - 1);
        assert_eq!(engine.stats.live_entries(LogQueue::Append), 16);
        // rewrite: [20..25][10..25]
        // append: [10..25]
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        let engine = engine.reopen();
        engine.scan_entries(rid, 10, 25, |_, q, d| {
            assert_eq!(q, LogQueue::Append);
            assert_eq!(d, &data);
        });

        // rewrite:[10..15][15  ..25]
        // append:           [20..25]
        rid += 1;
        engine.append(rid, 5, 15, Some(&data));
        let mut compact_log = LogBatch::default();
        compact_log.add_command(rid, Command::Compact { index: 10 });
        engine.write(&mut compact_log, true).unwrap();
        engine.purge_manager.must_rewrite_append_queue(None, None);
        engine.append(rid, 15, 25, Some(&data));
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        let mut compact_log = LogBatch::default();
        compact_log.add_command(rid, Command::Compact { index: 20 });
        engine.write(&mut compact_log, true).unwrap();
        let engine = engine.reopen();
        engine.scan_entries(rid, 20, 25, |_, q, d| {
            assert_eq!(q, LogQueue::Append);
            assert_eq!(d, &data);
        });

        // rewrite:[1..5] [10..15]
        // append:        [10..15]
        rid += 1;
        engine.append(rid, 1, 5, Some(&data));
        engine.purge_manager.must_rewrite_append_queue(None, None);
        engine.append(rid, 5, 15, Some(&data));
        let mut compact_log = LogBatch::default();
        compact_log.add_command(rid, Command::Compact { index: 10 });
        engine.write(&mut compact_log, true).unwrap();
        // Files are not purged.
        engine
            .purge_manager
            .must_rewrite_append_queue(None, Some(2));
        let engine = engine.reopen();
        engine.scan_entries(rid, 10, 15, |_, q, d| {
            assert_eq!(q, LogQueue::Append);
            assert_eq!(d, &data);
        });
    }

    #[test]
    fn test_purge_triggered_by_compact() {
        let dir = tempfile::Builder::new()
            .prefix("test_purge_triggered_by_compact")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize::kb(5),
            purge_threshold: ReadableSize::kb(150),
            ..Default::default()
        };

        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();
        let data = vec![b'x'; 1024];
        for index in 0..100 {
            engine.append(1, index, index + 1, Some(&data));
        }

        // GC all log entries. Won't trigger purge because total size is not enough.
        let count = engine.compact_to(1, 100);
        assert_eq!(count, 100);
        assert!(!engine
            .purge_manager
            .needs_rewrite_log_files(LogQueue::Append));

        // Append more logs to make total size greater than `purge_threshold`.
        for index in 100..250 {
            engine.append(1, index, index + 1, Some(&data));
        }

        // GC first 101 log entries.
        assert_eq!(engine.compact_to(1, 101), 1);
        // Needs to purge because the total size is greater than `purge_threshold`.
        assert!(engine
            .purge_manager
            .needs_rewrite_log_files(LogQueue::Append));

        let old_min_file_seq = engine.file_span(LogQueue::Append).0;
        let will_force_compact = engine.purge_expired_files().unwrap();
        let new_min_file_seq = engine.file_span(LogQueue::Append).0;
        // Some entries are rewritten.
        assert!(new_min_file_seq > old_min_file_seq);
        // No regions need to be force compacted because the threshold is not reached.
        assert!(will_force_compact.is_empty());
        // After purge, entries and raft state are still available.
        assert!(engine.get_entry::<Entry>(1, 101).unwrap().is_some());

        assert_eq!(engine.compact_to(1, 102), 1);
        // Needs to purge because the total size is greater than `purge_threshold`.
        assert!(engine
            .purge_manager
            .needs_rewrite_log_files(LogQueue::Append));
        let will_force_compact = engine.purge_expired_files().unwrap();
        // The region needs to be force compacted because the threshold is reached.
        assert!(!will_force_compact.is_empty());
        assert_eq!(will_force_compact[0], 1);
    }

    #[test]
    fn test_purge_trigger_force_rewrite() {
        let dir = tempfile::Builder::new()
            .prefix("test_purge_trigger_force_write")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize::kb(1),
            purge_threshold: ReadableSize::kb(10),
            ..Default::default()
        };

        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();
        let data = vec![b'x'; 1024];
        // write 50 small entries into region 1~3, it should trigger force compact.
        for rid in 1..=3 {
            for index in 0..50 {
                engine.append(rid, index, index + 1, Some(&data[..10]));
            }
        }
        // write some small entries to trigger purge.
        for rid in 4..=50 {
            engine.append(rid, 1, 2, Some(&data));
        }

        let check_purge = |pending_regions: Vec<u64>| {
            let mut compact_regions = engine.purge_expired_files().unwrap();
            // sort key in order.
            compact_regions.sort_unstable();
            assert_eq!(compact_regions, pending_regions);
        };

        for _ in 0..9 {
            check_purge(vec![1, 2, 3]);
        }

        // 10th, rewritten, but still needs to be compacted.
        check_purge(vec![1, 2, 3]);
        for rid in 1..=3 {
            let memtable = engine.memtables.get(rid).unwrap();
            assert_eq!(memtable.read().rewrite_count(), 50);
        }

        // compact and write some new data to trigger compact again.
        for rid in 2..=50 {
            let last_idx = engine.last_index(rid).unwrap();
            engine.compact_to(rid, last_idx);
            engine.append(rid, last_idx, last_idx + 1, Some(&data));
        }
        // after write, region 1 can trigger compact again.
        check_purge(vec![1]);
    }

    #[test]
    fn test_rewrite_and_recover() {
        let dir = tempfile::Builder::new()
            .prefix("test_rewrite_and_recover")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize::kb(5),
            purge_threshold: ReadableSize::kb(80),
            ..Default::default()
        };
        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();
        let data = vec![b'x'; 1024];

        // Put 100 entries into 10 regions.
        for index in 1..=10 {
            for rid in 1..=10 {
                engine.append(rid, index, index + 1, Some(&data));
            }
        }
        engine.append(11, 1, 11, Some(&data));

        // The engine needs purge, and all old entries should be rewritten.
        assert!(engine
            .purge_manager
            .needs_rewrite_log_files(LogQueue::Append));
        assert!(engine.purge_expired_files().unwrap().is_empty());
        assert!(engine.file_span(LogQueue::Append).0 > 1);

        let rewrite_file_size = engine.pipe_log.total_size(LogQueue::Rewrite);
        assert!(rewrite_file_size > 59); // The rewrite queue isn't empty.

        // All entries should be available.
        for rid in 1..=10 {
            engine.scan_entries(rid, 1, 11, |_, _, d| {
                assert_eq!(d, &data);
            });
        }

        engine.clean(11);
        let cleaned_region_ids = engine.memtables.cleaned_region_ids();
        assert_eq!(cleaned_region_ids.len(), 1);

        let engine = engine.reopen();
        assert_eq!(engine.memtables.cleaned_region_ids(), cleaned_region_ids);

        for rid in 1..=10 {
            engine.scan_entries(rid, 1, 11, |_, _, d| {
                assert_eq!(d, &data);
            });
        }

        // Rewrite again to check the rewrite queue is healthy.
        for index in 11..=20 {
            for rid in 1..=10 {
                engine.append(rid, index, index + 1, Some(&data));
            }
        }

        assert!(engine
            .purge_manager
            .needs_rewrite_log_files(LogQueue::Append));
        assert!(engine.purge_expired_files().unwrap().is_empty());
    }

    #[test]
    fn test_empty_protobuf_message() {
        let dir = tempfile::Builder::new()
            .prefix("test_empty_protobuf_message")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            ..Default::default()
        };
        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();

        let mut log_batch = LogBatch::default();
        let empty_entry = Entry::new();
        assert_eq!(empty_entry.compute_size(), 0);
        log_batch
            .add_entries::<Entry>(0, &[empty_entry.clone()])
            .unwrap();
        engine.write(&mut log_batch, false).unwrap();
        let empty_state = RaftLocalState::new();
        assert_eq!(empty_state.compute_size(), 0);
        log_batch
            .put_message(1, b"key".to_vec(), &empty_state)
            .unwrap();
        engine.write(&mut log_batch, false).unwrap();
        log_batch
            .add_entries::<Entry>(2, &[empty_entry.clone()])
            .unwrap();
        log_batch
            .put_message(2, b"key".to_vec(), &empty_state)
            .unwrap();
        engine.write(&mut log_batch, true).unwrap();

        let engine = engine.reopen();
        assert_eq!(
            engine.get_entry::<Entry>(0, 0).unwrap().unwrap(),
            empty_entry
        );
        assert_eq!(
            engine.get_entry::<Entry>(2, 0).unwrap().unwrap(),
            empty_entry
        );
        assert_eq!(
            engine
                .get_message::<RaftLocalState>(1, b"key")
                .unwrap()
                .unwrap(),
            empty_state
        );
        assert_eq!(
            engine
                .get_message::<RaftLocalState>(2, b"key")
                .unwrap()
                .unwrap(),
            empty_state
        );
    }

    #[test]
    fn test_empty_batch() {
        let dir = tempfile::Builder::new()
            .prefix("test_empty_batch")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            ..Default::default()
        };
        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();
        let data = vec![b'x'; 16];
        let cases = [[false, false], [false, true], [true, true]];
        for (i, writes) in cases.iter().enumerate() {
            let rid = i as u64;
            let mut batch = LogBatch::default();
            for &has_data in writes {
                if has_data {
                    batch.put(rid, b"key".to_vec(), data.clone()).unwrap();
                }
                engine.write(&mut batch, true).unwrap();
                assert!(batch.is_empty());
            }
        }
    }

    #[test]
    fn test_dirty_recovery() {
        let dir = tempfile::Builder::new()
            .prefix("test_dirty_recovery")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            ..Default::default()
        };
        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();
        let data = vec![b'x'; 1024];

        for rid in 1..21 {
            engine.append(rid, 1, 21, Some(&data));
        }

        // Create an unrelated sub-directory.
        std::fs::create_dir(dir.path().join(Path::new("random_dir"))).unwrap();
        // Create an unrelated file.
        let _f = std::fs::File::create(dir.path().join(Path::new("random_file"))).unwrap();

        let engine = engine.reopen();
        for rid in 1..21 {
            engine.scan_entries(rid, 1, 21, |_, _, d| {
                assert_eq!(d, &data);
            });
        }
    }

    #[test]
    fn test_large_rewrite_batch() {
        let dir = tempfile::Builder::new()
            .prefix("test_large_rewrite_batch")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            ..Default::default()
        };
        let engine =
            RaftLogEngine::open_with_file_system(cfg, Arc::new(ObfuscatedFileSystem::default()))
                .unwrap();
        let data = vec![b'x'; 2 * 1024 * 1024];

        for rid in 1..=3 {
            engine.append(rid, 1, 11, Some(&data));
        }

        let old_active_file = engine.file_span(LogQueue::Append).1;
        engine.purge_manager.must_rewrite_append_queue(None, None);
        assert_eq!(engine.file_span(LogQueue::Append).0, old_active_file + 1);
        let old_active_file = engine.file_span(LogQueue::Rewrite).1;
        engine.purge_manager.must_rewrite_rewrite_queue();
        assert!(engine.file_span(LogQueue::Rewrite).0 > old_active_file);

        for rid in engine.raft_groups() {
            let mut total = 0;
            engine
                .scan_raw_messages(rid, None, None, false, |k, _| {
                    assert!(!crate::is_internal_key(k, None));
                    total += 1;
                    true
                })
                .unwrap();
            assert_eq!(total, 1);
        }
        assert_eq!(engine.raft_groups().len(), 3);

        let engine = engine.reopen();
        for rid in 1..=3 {
            engine.scan_entries(rid, 1, 11, |_, _, d| {
                assert_eq!(d, &data);
            });
        }
    }

    #[test]
    fn test_combination_of_version_and_recycle() {
        fn test_engine_ops(cfg_v1: &Config, cfg_v2: &Config) {
            let rid = 1;
            let data = vec![b'7'; 1024];
            {
                // open engine with format_version - Version::V1
                let engine = RaftLogEngine::open(cfg_v1.clone()).unwrap();
                engine.append(rid, 0, 20, Some(&data));
                let append_first = engine.file_span(LogQueue::Append).0;
                engine.compact_to(rid, 18);
                engine.purge_expired_files().unwrap();
                assert!(engine.file_span(LogQueue::Append).0 > append_first);
                assert_eq!(engine.first_index(rid).unwrap(), 18);
                assert_eq!(engine.last_index(rid).unwrap(), 19);
            }
            {
                // open engine with format_version - Version::V2
                let engine = RaftLogEngine::open(cfg_v2.clone()).unwrap();
                assert_eq!(engine.first_index(rid).unwrap(), 18);
                assert_eq!(engine.last_index(rid).unwrap(), 19);
                engine.append(rid, 20, 40, Some(&data));
                let append_first = engine.file_span(LogQueue::Append).0;
                engine.compact_to(rid, 38);
                engine.purge_expired_files().unwrap();
                assert!(engine.file_span(LogQueue::Append).0 > append_first);
                assert_eq!(engine.first_index(rid).unwrap(), 38);
                assert_eq!(engine.last_index(rid).unwrap(), 39);
            }
            {
                // reopen engine with format_version - Version::V1
                let engine = RaftLogEngine::open(cfg_v1.clone()).unwrap();
                assert_eq!(engine.first_index(rid).unwrap(), 38);
                assert_eq!(engine.last_index(rid).unwrap(), 39);
            }
        }
        // test engine on mutable versions
        {
            let dir = tempfile::Builder::new()
                .prefix("test_mutable_format_version")
                .tempdir()
                .unwrap();
            // config with v1
            let cfg_v1 = Config {
                dir: dir.path().to_str().unwrap().to_owned(),
                target_file_size: ReadableSize(1),
                purge_threshold: ReadableSize(1),
                format_version: Version::V1,
                enable_log_recycle: false,
                ..Default::default()
            };
            // config with v2
            let cfg_v2 = Config {
                dir: dir.path().to_str().unwrap().to_owned(),
                target_file_size: ReadableSize(1),
                purge_threshold: ReadableSize(1),
                format_version: Version::V2,
                enable_log_recycle: false,
                ..Default::default()
            };
            test_engine_ops(&cfg_v1, &cfg_v2);
        }
        // test engine when enable_log_recycle == true
        {
            let dir = tempfile::Builder::new()
                .prefix("test_enable_log_recycle")
                .tempdir()
                .unwrap();
            // config with v1
            let cfg_v1 = Config {
                dir: dir.path().to_str().unwrap().to_owned(),
                target_file_size: ReadableSize(1),
                purge_threshold: ReadableSize(1),
                format_version: Version::V1,
                enable_log_recycle: false,
                ..Default::default()
            };
            // config with v2
            let cfg_v2 = Config {
                dir: dir.path().to_str().unwrap().to_owned(),
                target_file_size: ReadableSize(1),
                purge_threshold: ReadableSize(1),
                format_version: Version::V2,
                enable_log_recycle: true,
                prefill_for_recycle: true,
                ..Default::default()
            };
            test_engine_ops(&cfg_v1, &cfg_v2);
        }
    }

    /// Test cases related to tools ///

    #[test]
    fn test_dump_file_or_directory() {
        let dir = tempfile::Builder::new()
            .prefix("test_dump_file_or_directory")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 1024];
        let fs = Arc::new(ObfuscatedFileSystem::default());

        let mut batches = vec![vec![LogBatch::default()]];
        let mut batch = LogBatch::default();
        batch
            .add_entries::<Entry>(7, &generate_entries(1, 11, Some(&entry_data)))
            .unwrap();
        batch.add_command(7, Command::Clean);
        batch.put(7, b"key".to_vec(), b"value".to_vec()).unwrap();
        batch.delete(7, b"key2".to_vec());
        batches.push(vec![batch.clone()]);
        let mut batch2 = LogBatch::default();
        batch2.put(8, b"key3".to_vec(), b"value".to_vec()).unwrap();
        batch2
            .add_entries::<Entry>(8, &generate_entries(5, 15, Some(&entry_data)))
            .unwrap();
        batches.push(vec![batch, batch2]);

        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            ..Default::default()
        };

        let engine = RaftLogEngine::open_with_file_system(cfg, fs.clone()).unwrap();
        for bs in batches.iter_mut() {
            for batch in bs.iter_mut() {
                engine.write(batch, false).unwrap();
            }

            engine.sync().unwrap();
        }

        drop(engine);
        //dump dir with raft groups. 8 element in raft groups 7 and 2 elements in raft
        // groups 8
        let dump_it = Engine::dump_with_file_system(dir.path(), fs.clone()).unwrap();
        let total = dump_it
            .inspect(|i| {
                i.as_ref().unwrap();
            })
            .count();
        assert!(total == 10);

        //dump file
        let file_id = FileId {
            queue: LogQueue::Rewrite,
            seq: 1,
        };
        let dump_it = Engine::dump_with_file_system(
            file_id.build_file_path(dir.path()).as_path(),
            fs.clone(),
        )
        .unwrap();
        let total = dump_it
            .inspect(|i| {
                i.as_ref().unwrap();
            })
            .count();
        assert!(0 == total);

        //dump dir that does not exists
        assert!(Engine::dump_with_file_system(Path::new("/not_exists_dir"), fs.clone()).is_err());

        //dump file that does not exists
        let mut not_exists_file = PathBuf::from(dir.as_ref());
        not_exists_file.push("not_exists_file");
        assert!(Engine::dump_with_file_system(not_exists_file.as_path(), fs).is_err());
    }

    #[cfg(feature = "scripting")]
    #[test]
    fn test_repair_default() {
        let dir = tempfile::Builder::new()
            .prefix("test_repair_default")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 128];
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1), // Create lots of files.
            ..Default::default()
        };
        let fs = Arc::new(ObfuscatedFileSystem::default());

        let engine = RaftLogEngine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
        for rid in 1..=50 {
            engine.append(rid, 1, 6, Some(&entry_data));
        }
        for rid in 25..=50 {
            engine.append(rid, 6, 11, Some(&entry_data));
        }
        drop(engine);

        let script1 = "".to_owned();
        RaftLogEngine::unsafe_repair_with_file_system(
            dir.path(),
            None, /* queue */
            script1,
            fs.clone(),
        )
        .unwrap();
        let script2 = "
            fn filter_append(id, first, count, rewrite_count, queue, ifirst, ilast) {
                0
            }
            fn filter_compact(id, first, count, rewrite_count, queue, compact_to) {
                0
            }
            fn filter_clean(id, first, count, rewrite_count, queue) {
                0
            }
        "
        .to_owned();
        RaftLogEngine::unsafe_repair_with_file_system(
            dir.path(),
            None, /* queue */
            script2,
            fs.clone(),
        )
        .unwrap();

        let engine = RaftLogEngine::open_with_file_system(cfg, fs).unwrap();
        for rid in 1..25 {
            engine.scan_entries(rid, 1, 6, |_, _, d| {
                assert_eq!(d, &entry_data);
            });
        }
        for rid in 25..=50 {
            engine.scan_entries(rid, 1, 11, |_, _, d| {
                assert_eq!(d, &entry_data);
            });
        }
    }

    #[cfg(feature = "scripting")]
    #[test]
    fn test_repair_discard_entries() {
        let dir = tempfile::Builder::new()
            .prefix("test_repair_discard")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 128];
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1), // Create lots of files.
            ..Default::default()
        };
        let fs = Arc::new(ObfuscatedFileSystem::default());

        let engine = RaftLogEngine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
        for rid in 1..=50 {
            engine.append(rid, 1, 6, Some(&entry_data));
        }
        for rid in 25..=50 {
            engine.append(rid, 6, 11, Some(&entry_data));
        }
        drop(engine);

        let incoming_emptied = [1, 25];
        let existing_emptied = [2, 26];
        let script = "
            fn filter_append(id, first, count, rewrite_count, queue, ifirst, ilast) {
                if id == 1 {
                    return 1;
                } else if id == 2 {
                    return 2;
                } else if id == 25 {
                    return 1;
                } else if id == 26 {
                    return 2;
                }
                0 // default
            }
        "
        .to_owned();
        RaftLogEngine::unsafe_repair_with_file_system(
            dir.path(),
            None, /* queue */
            script,
            fs.clone(),
        )
        .unwrap();

        let engine = RaftLogEngine::open_with_file_system(cfg, fs).unwrap();
        for rid in 1..25 {
            if existing_emptied.contains(&rid) || incoming_emptied.contains(&rid) {
                continue;
            }
            engine.scan_entries(rid, 1, 6, |_, _, d| {
                assert_eq!(d, &entry_data);
            });
        }
        for rid in 25..=50 {
            if existing_emptied.contains(&rid) || incoming_emptied.contains(&rid) {
                continue;
            }
            engine.scan_entries(rid, 1, 11, |_, _, d| {
                assert_eq!(d, &entry_data);
            });
        }
        for rid in existing_emptied {
            let first_index = if rid < 25 { 1 } else { 6 };
            let last_index = if rid < 25 { 5 } else { 10 };
            engine.scan_entries(rid, first_index, last_index + 1, |_, _, d| {
                assert_eq!(d, &entry_data);
            });
        }
        for rid in incoming_emptied {
            let last_index = if rid < 25 { 5 } else { 10 };
            assert_eq!(engine.first_index(rid), None);
            assert_eq!(engine.last_index(rid), None);
            assert_eq!(engine.decode_last_index(rid), Some(last_index));
        }
    }

    #[test]
    fn test_tail_corruption() {
        let dir = tempfile::Builder::new()
            .prefix("test_tail_corruption")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 16];
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            // One big file.
            target_file_size: ReadableSize::gb(10),
            ..Default::default()
        };
        let fs = Arc::new(ObfuscatedFileSystem::default());

        let engine = RaftLogEngine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
        for rid in 1..=50 {
            engine.append(rid, 1, 6, Some(&entry_data));
        }
        for rid in 25..=50 {
            engine.append(rid, 6, 11, Some(&entry_data));
        }
        let (_, last_file_seq) = engine.file_span(LogQueue::Append);
        drop(engine);

        let last_file = FileId {
            queue: LogQueue::Append,
            seq: last_file_seq,
        };
        let f = OpenOptions::new()
            .write(true)
            .open(last_file.build_file_path(dir.path()))
            .unwrap();

        // Corrupt a log batch.
        f.set_len(f.metadata().unwrap().len() - 1).unwrap();
        RaftLogEngine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();

        // Corrupt the file header.
        f.set_len(1).unwrap();
        RaftLogEngine::open_with_file_system(cfg, fs).unwrap();
    }

    #[test]
    fn test_reopen_with_wrong_file_system() {
        let dir = tempfile::Builder::new()
            .prefix("test_reopen_with_wrong_file_system")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 128];
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            ..Default::default()
        };
        let fs = Arc::new(ObfuscatedFileSystem::default());

        let engine = RaftLogEngine::open_with_file_system(cfg.clone(), fs.clone()).unwrap();
        for rid in 1..=10 {
            engine.append(rid, 1, 11, Some(&entry_data));
        }
        drop(engine);

        assert!(RaftLogEngine::open(cfg.clone()).is_err());

        let engine = RaftLogEngine::open_with_file_system(cfg, fs).unwrap();
        for rid in 1..10 {
            engine.scan_entries(rid, 1, 11, |_, _, d| {
                assert_eq!(d, &entry_data);
            });
        }
    }

    #[cfg(feature = "nightly")]
    #[bench]
    fn bench_engine_fetch_entries(b: &mut test::Bencher) {
        use rand::{thread_rng, Rng};

        let dir = tempfile::Builder::new()
            .prefix("bench_engine_fetch_entries")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 1024];
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            ..Default::default()
        };
        let engine = RaftLogEngine::open(cfg).unwrap();
        for i in 0..10 {
            for rid in 1..=100 {
                engine.append(rid, 1 + i * 10, 1 + i * 10 + 10, Some(&entry_data));
            }
        }
        let mut vec: Vec<Entry> = Vec::new();
        b.iter(move || {
            let region_id = thread_rng().gen_range(1..=100);
            engine
                .fetch_entries_to::<Entry>(region_id, 1, 101, None, &mut vec)
                .unwrap();
            vec.clear();
        });
    }

    #[test]
    fn test_engine_is_empty() {
        let dir = tempfile::Builder::new()
            .prefix("test_engine_is_empty")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 128];
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            ..Default::default()
        };
        let fs = Arc::new(ObfuscatedFileSystem::default());
        let rid = 1;

        let engine = RaftLogEngine::open_with_file_system(cfg, fs).unwrap();
        assert!(engine.is_empty());
        engine.append(rid, 1, 11, Some(&entry_data));
        assert!(!engine.is_empty());

        let mut log_batch = LogBatch::default();
        log_batch.add_command(rid, Command::Compact { index: 11 });
        log_batch.delete(rid, b"last_index".to_vec());
        engine.write(&mut log_batch, true).unwrap();
        assert!(!engine.is_empty());

        engine.clean(rid);
        assert!(engine.is_empty());
    }

    pub struct DeleteMonitoredFileSystem {
        inner: ObfuscatedFileSystem,
        append_metadata: Mutex<BTreeSet<u64>>,
        reserved_metadata: Mutex<BTreeSet<u64>>,
    }

    impl DeleteMonitoredFileSystem {
        fn new() -> Self {
            Self {
                inner: ObfuscatedFileSystem::default(),
                append_metadata: Mutex::new(BTreeSet::new()),
                reserved_metadata: Mutex::new(BTreeSet::new()),
            }
        }

        fn update_metadata(&self, path: &Path, delete: bool) -> bool {
            let path = path.file_name().unwrap().to_str().unwrap();
            let parse_append = FileId::parse_file_name(path);
            let parse_reserved = parse_reserved_file_name(path);
            match (parse_append, parse_reserved) {
                (Some(id), None) if id.queue == LogQueue::Append => {
                    if delete {
                        self.append_metadata.lock().unwrap().remove(&id.seq)
                    } else {
                        self.append_metadata.lock().unwrap().insert(id.seq)
                    }
                }
                (None, Some(seq)) => {
                    if delete {
                        self.reserved_metadata.lock().unwrap().remove(&seq)
                    } else {
                        self.reserved_metadata.lock().unwrap().insert(seq)
                    }
                }
                _ => false,
            }
        }
    }

    impl FileSystem for DeleteMonitoredFileSystem {
        type Handle = <ObfuscatedFileSystem as FileSystem>::Handle;
        type Reader = <ObfuscatedFileSystem as FileSystem>::Reader;
        type Writer = <ObfuscatedFileSystem as FileSystem>::Writer;

        fn create<P: AsRef<Path>>(&self, path: P) -> std::io::Result<Self::Handle> {
            let handle = self.inner.create(&path)?;
            self.update_metadata(path.as_ref(), false);
            Ok(handle)
        }

        fn open<P: AsRef<Path>>(&self, path: P, perm: Permission) -> std::io::Result<Self::Handle> {
            let handle = self.inner.open(&path, perm)?;
            self.update_metadata(path.as_ref(), false);
            Ok(handle)
        }

        fn delete<P: AsRef<Path>>(&self, path: P) -> std::io::Result<()> {
            self.inner.delete(&path)?;
            self.update_metadata(path.as_ref(), true);
            Ok(())
        }

        fn rename<P: AsRef<Path>>(&self, src_path: P, dst_path: P) -> std::io::Result<()> {
            self.inner.rename(src_path.as_ref(), dst_path.as_ref())?;
            self.update_metadata(src_path.as_ref(), true);
            self.update_metadata(dst_path.as_ref(), false);
            Ok(())
        }

        fn reuse<P: AsRef<Path>>(&self, src_path: P, dst_path: P) -> std::io::Result<()> {
            self.inner.reuse(src_path.as_ref(), dst_path.as_ref())?;
            self.update_metadata(src_path.as_ref(), true);
            self.update_metadata(dst_path.as_ref(), false);
            Ok(())
        }

        fn delete_metadata<P: AsRef<Path>>(&self, path: P) -> std::io::Result<()> {
            self.inner.delete_metadata(&path)?;
            self.update_metadata(path.as_ref(), true);
            Ok(())
        }

        fn exists_metadata<P: AsRef<Path>>(&self, path: P) -> bool {
            if self.inner.exists_metadata(&path) {
                return true;
            }
            let path = path.as_ref().file_name().unwrap().to_str().unwrap();
            let parse_append = FileId::parse_file_name(path);
            let parse_reserved = parse_reserved_file_name(path);
            match (parse_append, parse_reserved) {
                (Some(id), None) if id.queue == LogQueue::Append => {
                    self.append_metadata.lock().unwrap().contains(&id.seq)
                }
                (None, Some(seq)) => self.reserved_metadata.lock().unwrap().contains(&seq),
                _ => false,
            }
        }

        fn new_reader(&self, h: Arc<Self::Handle>) -> std::io::Result<Self::Reader> {
            self.inner.new_reader(h)
        }

        fn new_writer(&self, h: Arc<Self::Handle>) -> std::io::Result<Self::Writer> {
            self.inner.new_writer(h)
        }
    }

    #[test]
    fn test_managed_file_deletion() {
        let dir = tempfile::Builder::new()
            .prefix("test_managed_file_deletion")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 128];
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            purge_threshold: ReadableSize(1),
            enable_log_recycle: false,
            ..Default::default()
        };
        let fs = Arc::new(DeleteMonitoredFileSystem::new());
        let engine = RaftLogEngine::open_with_file_system(cfg, fs.clone()).unwrap();
        for rid in 1..=10 {
            engine.append(rid, 1, 11, Some(&entry_data));
        }
        for rid in 1..=5 {
            engine.clean(rid);
        }
        let (start, _) = engine.file_span(LogQueue::Append);
        engine.purge_expired_files().unwrap();
        // some active files have been deleted.
        assert!(start < engine.file_span(LogQueue::Append).0);
        // corresponding physical files have been deleted too.
        assert_eq!(engine.file_count(None), fs.inner.file_count());
        let start = engine.file_span(LogQueue::Append).0;
        // metadata have been deleted.
        assert_eq!(
            fs.append_metadata.lock().unwrap().iter().next().unwrap(),
            &start
        );

        let engine = engine.reopen();
        assert_eq!(engine.file_count(None), fs.inner.file_count());
        let (start, _) = engine.file_span(LogQueue::Append);
        assert_eq!(
            fs.append_metadata.lock().unwrap().iter().next().unwrap(),
            &start
        );

        // Simulate stale metadata.
        for i in start / 2..start {
            fs.append_metadata.lock().unwrap().insert(i);
        }
        let engine = engine.reopen();
        let (start, _) = engine.file_span(LogQueue::Append);
        assert_eq!(
            fs.append_metadata.lock().unwrap().iter().next().unwrap(),
            &start
        );
    }

    #[test]
    fn test_managed_file_reuse() {
        let dir = tempfile::Builder::new()
            .prefix("test_managed_file_reuse")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 16];
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            purge_threshold: ReadableSize(50),
            format_version: Version::V2,
            enable_log_recycle: true,
            prefill_for_recycle: true,
            ..Default::default()
        };
        let fs = Arc::new(DeleteMonitoredFileSystem::new());
        let engine = RaftLogEngine::open_with_file_system(cfg, fs.clone()).unwrap();

        let reserved_start = *fs.reserved_metadata.lock().unwrap().first().unwrap();
        for rid in 1..=10 {
            engine.append(rid, 1, 11, Some(&entry_data));
        }
        for rid in 1..=10 {
            engine.clean(rid);
        }
        // Purge all files.
        engine.purge_manager.must_rewrite_append_queue(None, None);
        assert_eq!(engine.file_count(Some(LogQueue::Append)), 1);
        // Reserved files have been reused.
        let reserved_start_1 = *fs.reserved_metadata.lock().unwrap().first().unwrap();
        assert!(reserved_start < reserved_start_1);
        // Reuse more.
        for rid in 1..=5 {
            engine.append(rid, 1, 11, Some(&entry_data));
        }
        let reserved_start_2 = *fs.reserved_metadata.lock().unwrap().first().unwrap();
        assert!(reserved_start_1 < reserved_start_2);

        let file_count = fs.inner.file_count();
        let start_1 = *fs.append_metadata.lock().unwrap().first().unwrap();
        let engine = engine.reopen();
        // Recycled files are reserved, but stale append files are renamed. The total
        // count should stay unchanged.
        assert_eq!(file_count, fs.inner.file_count());
        let start_2 = *fs.append_metadata.lock().unwrap().first().unwrap();
        assert!(start_1 < start_2);
        let reserved_start_3 = *fs.reserved_metadata.lock().unwrap().first().unwrap();
        assert_eq!(reserved_start_2, reserved_start_3);

        // Reuse all of reserved files.
        for rid in 1..=50 {
            engine.append(rid, 1, 11, Some(&entry_data));
        }
        assert!(fs.reserved_metadata.lock().unwrap().is_empty());
        for rid in 1..=50 {
            engine.clean(rid);
        }
        engine.purge_manager.must_rewrite_append_queue(None, None);
        // Then reuse a recycled append file.
        engine.append(1, 1, 11, Some(&entry_data));
        assert_eq!(engine.file_count(Some(LogQueue::Append)), 2);
        let start_3 = *fs.append_metadata.lock().unwrap().first().unwrap();
        assert!(start_2 < start_3);
    }

    #[test]
    fn test_simple_write_perf_context() {
        let dir = tempfile::Builder::new()
            .prefix("test_simple_write_perf_context")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            ..Default::default()
        };
        let rid = 1;
        let entry_size = 5120;
        let engine = RaftLogEngine::open(cfg).unwrap();
        let data = vec![b'x'; entry_size];
        let old_perf_context = get_perf_context();
        engine.append(rid, 1, 5, Some(&data));
        let new_perf_context = get_perf_context();
        assert_ne!(
            old_perf_context.log_populating_duration,
            new_perf_context.log_populating_duration
        );
        assert_ne!(
            old_perf_context.log_write_duration,
            new_perf_context.log_write_duration
        );
        assert_ne!(
            old_perf_context.apply_duration,
            new_perf_context.apply_duration
        );
    }

    #[test]
    fn test_recycle_no_signing_files() {
        let dir = tempfile::Builder::new()
            .prefix("test_recycle_no_signing_files")
            .tempdir()
            .unwrap();
        let entry_data = vec![b'x'; 128];
        let fs = Arc::new(DeleteMonitoredFileSystem::new());
        let cfg_v1 = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            purge_threshold: ReadableSize(1024),
            format_version: Version::V1,
            enable_log_recycle: false,
            ..Default::default()
        };
        let cfg_v2 = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            target_file_size: ReadableSize(1),
            purge_threshold: ReadableSize(15),
            format_version: Version::V2,
            enable_log_recycle: true,
            prefill_for_recycle: false,
            ..Default::default()
        };
        assert!(cfg_v2.recycle_capacity() > 0);
        // Prepare files with format_version V1
        {
            let engine = RaftLogEngine::open_with_file_system(cfg_v1.clone(), fs.clone()).unwrap();
            for rid in 1..=10 {
                engine.append(rid, 1, 11, Some(&entry_data));
            }
        }
        // Reopen the Engine with V2 and purge
        {
            let engine = RaftLogEngine::open_with_file_system(cfg_v2.clone(), fs.clone()).unwrap();
            let (start, _) = engine.file_span(LogQueue::Append);
            for rid in 6..=10 {
                engine.append(rid, 11, 20, Some(&entry_data));
            }
            // Mark region_id -> 6 obsolete.
            engine.clean(6);
            // the [1, 12] files are recycled
            engine.purge_expired_files().unwrap();
            assert_eq!(engine.file_count(Some(LogQueue::Append)), 5);
            assert!(start < engine.file_span(LogQueue::Append).0);
        }
        // Reopen the Engine with V1 -> V2 and purge
        {
            let engine = RaftLogEngine::open_with_file_system(cfg_v1, fs.clone()).unwrap();
            let (start, _) = engine.file_span(LogQueue::Append);
            for rid in 6..=10 {
                engine.append(rid, 20, 30, Some(&entry_data));
            }
            for rid in 6..=10 {
                engine.append(rid, 30, 40, Some(&entry_data));
            }
            for rid in 1..=5 {
                engine.append(rid, 11, 20, Some(&entry_data));
            }
            assert_eq!(engine.file_span(LogQueue::Append).0, start);
            let file_count = engine.file_count(Some(LogQueue::Append));
            drop(engine);
            let engine = RaftLogEngine::open_with_file_system(cfg_v2, fs).unwrap();
            assert_eq!(engine.file_span(LogQueue::Append).0, start);
            assert_eq!(engine.file_count(Some(LogQueue::Append)), file_count);
            // Mark all regions obsolete.
            for rid in 1..=10 {
                engine.clean(rid);
            }
            let (start, _) = engine.file_span(LogQueue::Append);
            // the [13, 32] files are purged
            engine.purge_expired_files().unwrap();
            assert_eq!(engine.file_count(Some(LogQueue::Append)), 1);
            assert!(engine.file_span(LogQueue::Append).0 > start);
        }
    }

    #[test]
    fn test_start_engine_with_resize_recycle_capacity() {
        let dir = tempfile::Builder::new()
            .prefix("test_start_engine_with_resize_recycle_capacity")
            .tempdir()
            .unwrap();
        let path = dir.path().to_str().unwrap();
        let file_system = Arc::new(DeleteMonitoredFileSystem::new());
        let entry_data = vec![b'x'; 512];

        // Case 1: start an engine with no-recycle.
        let cfg = Config {
            dir: path.to_owned(),
            enable_log_recycle: false,
            ..Default::default()
        };
        let engine = RaftLogEngine::open_with_file_system(cfg, file_system.clone()).unwrap();
        let (start, _) = engine.file_span(LogQueue::Append);
        // Only one valid file left, the last one => active_file.
        assert_eq!(engine.file_count(Some(LogQueue::Append)), 1);
        assert_eq!(file_system.inner.file_count(), engine.file_count(None));
        // Append data.
        for rid in 1..=5 {
            engine.append(rid, 1, 10, Some(&entry_data));
        }
        assert_eq!(engine.file_span(LogQueue::Append).0, start);
        assert_eq!(file_system.inner.file_count(), engine.file_count(None));
        drop(engine);

        // Case 2: restart the engine with a common size of recycling capacity.
        let cfg = Config {
            dir: path.to_owned(),
            target_file_size: ReadableSize(1),
            purge_threshold: ReadableSize(80), // common size of capacity
            enable_log_recycle: true,
            prefill_for_recycle: true,
            ..Default::default()
        };
        let engine =
            RaftLogEngine::open_with_file_system(cfg.clone(), file_system.clone()).unwrap();
        let (start, end) = engine.file_span(LogQueue::Append);
        // Only one valid file left, the last one => active_file.
        assert_eq!(start, end);
        let recycled_count = file_system.inner.file_count() - engine.file_count(None);
        assert!(recycled_count > 0);
        // Append data. Several recycled files have been reused.
        for rid in 1..=5 {
            engine.append(rid, 10, 20, Some(&entry_data));
        }
        assert_eq!(engine.file_span(LogQueue::Append).0, start);
        assert!(recycled_count > file_system.inner.file_count() - engine.file_count(None));
        let (start, end) = engine.file_span(LogQueue::Append);
        let recycled_count = file_system.inner.file_count() - engine.file_count(None);
        drop(engine);

        // Case 3: restart the engine with a smaller capacity. Redundant recycled files
        // will be cleared.
        let cfg_v2 = Config {
            target_file_size: ReadableSize(1),
            purge_threshold: ReadableSize(50),
            ..cfg
        };
        let engine =
            RaftLogEngine::open_with_file_system(cfg_v2.clone(), file_system.clone()).unwrap();
        assert_eq!(engine.file_span(LogQueue::Append), (start, end));
        assert!(recycled_count > file_system.inner.file_count() - engine.file_count(None));
        // Recycled files have filled the LogQueue::Append, purge_expired_files won't
        // truely remove files from it.
        engine.purge_expired_files().unwrap();
        assert_eq!(engine.file_span(LogQueue::Append), (start, end));
        for rid in 1..=10 {
            engine.append(rid, 20, 31, Some(&entry_data));
        }
        assert!(engine.file_span(LogQueue::Append).1 > end);
        let engine = engine.reopen();
        assert!(recycled_count > file_system.inner.file_count() - engine.file_count(None));
        drop(engine);

        // Case 4: restart the engine without log recycling. Recycled logs should be
        // cleared.
        let cfg_v3 = Config {
            target_file_size: ReadableSize::kb(2),
            purge_threshold: ReadableSize::kb(100),
            enable_log_recycle: false,
            prefill_for_recycle: false,
            ..cfg_v2
        };
        let engine = RaftLogEngine::open_with_file_system(cfg_v3, file_system.clone()).unwrap();
        assert_eq!(file_system.inner.file_count(), engine.file_count(None));
    }

    #[test]
    fn test_rewrite_atomic_group() {
        let dir = tempfile::Builder::new()
            .prefix("test_rewrite_atomic_group")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            // Make sure each file gets replayed individually.
            recovery_threads: 100,
            target_file_size: ReadableSize(1),
            ..Default::default()
        };
        let fs = Arc::new(ObfuscatedFileSystem::default());
        let key = vec![b'x'; 2];
        let value = vec![b'y'; 8];

        let engine = RaftLogEngine::open_with_file_system(cfg, fs).unwrap();
        let mut data = HashSet::new();
        let mut rid = 1;
        // Directly write to pipe log.
        let mut log_batch = LogBatch::default();
        let flush = |lb: &mut LogBatch| {
            lb.finish_populate(0, None).unwrap();
            engine.pipe_log.append(LogQueue::Rewrite, lb).unwrap();
            lb.drain();
        };
        {
            // begin.
            let mut builder = AtomicGroupBuilder::with_id(3);
            builder.begin(&mut log_batch);
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            flush(&mut log_batch);
            engine.pipe_log.rotate(LogQueue::Rewrite).unwrap();
        }
        {
            // begin - unrelated - end.
            let mut builder = AtomicGroupBuilder::with_id(3);
            builder.begin(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            // plug a unrelated write.
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            builder.end(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            engine.pipe_log.rotate(LogQueue::Rewrite).unwrap();
        }
        {
            // begin - middle - middle - end.
            let mut builder = AtomicGroupBuilder::with_id(3);
            builder.begin(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            builder.add(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            builder.add(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            builder.end(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            engine.pipe_log.rotate(LogQueue::Rewrite).unwrap();
        }
        {
            // begin - begin - end.
            let mut builder = AtomicGroupBuilder::with_id(3);
            builder.begin(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            flush(&mut log_batch);
            let mut builder = AtomicGroupBuilder::with_id(3);
            builder.begin(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            builder.end(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            engine.pipe_log.rotate(LogQueue::Rewrite).unwrap();
        }
        {
            // end - middle - end.
            // We must change id to avoid getting merged with last group.
            // It is actually not possible in real life to only have "begin" missing.
            let mut builder = AtomicGroupBuilder::with_id(4);
            builder.begin(&mut LogBatch::default());
            builder.end(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            flush(&mut log_batch);
            let mut builder = AtomicGroupBuilder::with_id(4);
            builder.begin(&mut LogBatch::default());
            builder.add(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            flush(&mut log_batch);
            builder.end(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            flush(&mut log_batch);
            engine.pipe_log.rotate(LogQueue::Rewrite).unwrap();
        }
        {
            // end - begin - end
            let mut builder = AtomicGroupBuilder::with_id(5);
            builder.begin(&mut LogBatch::default());
            builder.end(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            flush(&mut log_batch);
            let mut builder = AtomicGroupBuilder::with_id(5);
            builder.begin(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            builder.end(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            engine.pipe_log.rotate(LogQueue::Rewrite).unwrap();
        }
        {
            // begin - end - begin - end.
            let mut builder = AtomicGroupBuilder::with_id(6);
            builder.begin(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            builder.end(&mut log_batch);
            flush(&mut log_batch);
            let mut builder = AtomicGroupBuilder::with_id(7);
            builder.begin(&mut log_batch);
            flush(&mut log_batch);
            builder.end(&mut log_batch);
            rid += 1;
            log_batch.put(rid, key.clone(), value.clone()).unwrap();
            data.insert(rid);
            flush(&mut log_batch);
            engine.pipe_log.rotate(LogQueue::Rewrite).unwrap();
        }
        engine.pipe_log.sync(LogQueue::Rewrite).unwrap();

        let engine = engine.reopen();
        for rid in engine.raft_groups() {
            assert!(data.remove(&rid), "{}", rid);
            assert_eq!(engine.get(rid, &key).unwrap(), value);
        }
        assert!(data.is_empty(), "data loss {:?}", data);
    }

    #[test]
    fn test_internal_key_filter() {
        let dir = tempfile::Builder::new()
            .prefix("test_internal_key_filter")
            .tempdir()
            .unwrap();
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            ..Default::default()
        };
        let fs = Arc::new(ObfuscatedFileSystem::default());
        let engine = RaftLogEngine::open_with_file_system(cfg, fs).unwrap();
        let value = vec![b'y'; 8];
        let mut log_batch = LogBatch::default();
        log_batch.put_unchecked(1, crate::make_internal_key(&[1]), value.clone());
        log_batch.put_unchecked(2, crate::make_internal_key(&[1]), value.clone());
        engine.write(&mut log_batch, false).unwrap();
        // Apply of append filtered.
        assert!(engine.raft_groups().is_empty());

        let engine = engine.reopen();
        // Replay of append filtered.
        assert!(engine.raft_groups().is_empty());

        log_batch.put_unchecked(3, crate::make_internal_key(&[1]), value.clone());
        log_batch.put_unchecked(4, crate::make_internal_key(&[1]), value);
        log_batch.finish_populate(0, None).unwrap();
        let block_handle = engine
            .pipe_log
            .append(LogQueue::Rewrite, &mut log_batch)
            .unwrap();
        log_batch.finish_write(block_handle);
        engine
            .memtables
            .apply_rewrite_writes(log_batch.drain(), None, 0);
        // Apply of rewrite filtered.
        assert!(engine.raft_groups().is_empty());

        let engine = engine.reopen();
        // Replay of rewrite filtered.
        assert!(engine.raft_groups().is_empty());
    }

    #[test]
    fn test_start_engine_with_multi_dirs() {
        let dir = tempfile::Builder::new()
            .prefix("test_start_engine_with_multi_dirs_default")
            .tempdir()
            .unwrap();
        let spill_dir = tempfile::Builder::new()
            .prefix("test_start_engine_with_multi_dirs_spill")
            .tempdir()
            .unwrap();
        fn number_of_files(p: &Path) -> usize {
            let mut r = 0;
            std::fs::read_dir(p).unwrap().for_each(|e| {
                if e.unwrap()
                    .path()
                    .file_name()
                    .unwrap()
                    .to_str()
                    .unwrap()
                    .starts_with("000")
                {
                    r += 1;
                }
            });
            r
        }
        let file_system = Arc::new(DeleteMonitoredFileSystem::new());
        let entry_data = vec![b'x'; 512];

        // Preparations for multi-dirs.
        let cfg = Config {
            dir: dir.path().to_str().unwrap().to_owned(),
            spill_dir: Some(spill_dir.path().to_str().unwrap().to_owned()),
            enable_log_recycle: false,
            target_file_size: ReadableSize(1),
            ..Default::default()
        };
        {
            // Step 1: write data into the main directory.
            let engine =
                RaftLogEngine::open_with_file_system(cfg.clone(), file_system.clone()).unwrap();
            for rid in 1..=10 {
                engine.append(rid, 1, 10, Some(&entry_data));
            }
            drop(engine);

            // Step 2: select several log files and move them into the `spill_dir`
            // directory.
            let mut moved = 0;
            for e in std::fs::read_dir(dir.path()).unwrap() {
                let p = e.unwrap().path();
                let file_name = p.file_name().unwrap().to_str().unwrap();
                if let Some(FileId {
                    queue: LogQueue::Append,
                    seq: _,
                }) = FileId::parse_file_name(file_name)
                {
                    file_system
                        .rename(&p, &spill_dir.path().join(file_name))
                        .unwrap();
                    moved += 1;
                    if moved == 4 {
                        break;
                    }
                }
            }
        }

        // Restart the engine with recycle and prefill. Test reusing files from both
        // dirs.
        let cfg_2 = Config {
            enable_log_recycle: true,
            prefill_for_recycle: true,
            purge_threshold: ReadableSize(40),
            ..cfg.clone()
        };
        let engine = RaftLogEngine::open_with_file_system(cfg_2, file_system.clone()).unwrap();
        assert!(number_of_files(spill_dir.path()) > 0);
        for rid in 1..=10 {
            assert_eq!(engine.first_index(rid).unwrap(), 1);
            engine.clean(rid);
        }
        engine.purge_manager.must_rewrite_append_queue(None, None);
        let file_count = file_system.inner.file_count();
        assert_eq!(
            number_of_files(spill_dir.path()) + number_of_files(dir.path()),
            file_count
        );
        assert!(file_count > engine.file_count(None));
        // Append data, recycled files are reused.
        for rid in 1..=30 {
            engine.append(rid, 20, 30, Some(&entry_data));
        }
        // No new file is created.
        assert_eq!(file_count, file_system.inner.file_count());
        assert!(number_of_files(spill_dir.path()) > 0);

        let cfg_3 = Config {
            enable_log_recycle: false,
            purge_threshold: ReadableSize(40),
            ..cfg
        };
        drop(engine);
        let engine = RaftLogEngine::open_with_file_system(cfg_3, file_system).unwrap();
        assert!(number_of_files(spill_dir.path()) > 0);
        for rid in 1..=10 {
            assert_eq!(engine.first_index(rid).unwrap(), 20);
        }

        // abnormal case - duplicate FileSeq among different dirs.
        {
            // Prerequisite: choose several files and duplicate them to main dir.
            let mut file_count = 0;
            for e in std::fs::read_dir(spill_dir.path()).unwrap() {
                let p = e.unwrap().path();
                let file_name = p.file_name().unwrap().to_str().unwrap();
                if let Some(FileId {
                    queue: LogQueue::Append,
                    seq: _,
                }) = FileId::parse_file_name(file_name)
                {
                    if file_count % 2 == 0 {
                        std::fs::copy(&p, dir.path().join(file_name)).unwrap();
                    }
                    file_count += 1;
                }
            }
        }
        let start = engine.file_span(LogQueue::Append).0;
        let engine = engine.reopen();
        // Duplicate log files will be skipped and cleared.
        assert!(engine.file_span(LogQueue::Append).0 > start);
    }
}