alopex-core 0.5.0

Core storage engine for Alopex DB - LSM-tree, columnar storage, and vector index
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
//! SSTable implementation for the LSM-tree engine.
//!
//! This is a standalone (per-table) format used by the LSM layer. It is intentionally independent
//! from the legacy `storage::sstable` implementation.
//!
//! # Ordering / MVCC
//!
//! Entries are stored in sorted order by:
//! 1) `user_key` ascending (lexicographic)
//! 2) `timestamp` descending
//! 3) `sequence` descending
//!
//! Reads at a given `read_timestamp` return the first entry for a key whose `timestamp` is
//! `<= read_timestamp` (tombstones included).
//!
//! # On-disk layout (spec §3.3.2)
//!
//! ```text
//! Header (32B)
//! Data Blocks...
//! Index Block
//! Bloom Filter (optional)
//! Footer (48B)
//! ```
//!
//! Each data block is:
//! - `entry_count: u32`
//! - `uncompressed_len: u32`
//! - `payload: [u8]` (entries bytes, possibly compressed)
//! - `crc32: u32` over `entry_count || uncompressed_len || payload`
//!
//! The index block maps key ranges to block offsets/sizes.

use std::cmp::Ordering as CmpOrdering;
use std::fs::{File, OpenOptions};
use std::io::{Read, Seek, SeekFrom, Write};
use std::path::{Path, PathBuf};
use std::sync::Arc;

use crate::error::{Error, Result};
use crate::lsm::buffer_pool::{BlockId, BufferPool, DataBlock};
use crate::lsm::metrics::LsmMetrics;
use crate::storage::{create_compressor, CompressionV2, Compressor};
use crate::types::{Key, Value};

const SST_MAGIC: [u8; 4] = *b"ASST";
const SST_MAGIC_REVERSE: [u8; 4] = *b"TSSA";
const SST_VERSION: u16 = 1;

const HEADER_SIZE: usize = 32;
const FOOTER_SIZE: usize = 48;

const FLAG_BLOOM_PRESENT: u16 = 1 << 0;
const FLAG_COMPRESSION_MASK: u16 = 0b11 << 1;

fn crc32(bytes: &[u8]) -> u32 {
    crc32fast::hash(bytes)
}

fn compression_to_flag(compression: CompressionType) -> u16 {
    match compression {
        CompressionType::None => 0,
        CompressionType::Lz4 => 1,
        CompressionType::Zstd { .. } => 2,
    }
}

fn compression_from_flag(flag: u16, zstd_level: i32) -> Result<CompressionType> {
    match flag {
        0 => Ok(CompressionType::None),
        1 => Ok(CompressionType::Lz4),
        2 => Ok(CompressionType::Zstd { level: zstd_level }),
        _ => Err(Error::InvalidFormat(
            "unknown SSTable compression flag".into(),
        )),
    }
}

fn compression_to_v2(compression: CompressionType) -> CompressionV2 {
    match compression {
        CompressionType::None => CompressionV2::None,
        CompressionType::Lz4 => CompressionV2::Lz4,
        CompressionType::Zstd { level } => CompressionV2::Zstd { level },
    }
}

/// SSTable configuration.
#[derive(Debug, Clone, Copy, PartialEq)]
pub struct SSTableConfig {
    /// Block size (default: 4KB).
    pub block_size: usize,
    /// Enable Bloom filter.
    pub enable_bloom_filter: bool,
    /// Desired Bloom filter false positive rate (default: 1%).
    pub bloom_filter_fpr: f64,
    /// Compression algorithm.
    pub compression: CompressionType,
}

impl Default for SSTableConfig {
    fn default() -> Self {
        Self {
            block_size: 4096,
            enable_bloom_filter: true,
            bloom_filter_fpr: 0.01,
            compression: CompressionType::None,
        }
    }
}

/// Compression algorithm for data blocks.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum CompressionType {
    /// No compression.
    None,
    /// LZ4 compression (feature-gated).
    Lz4,
    /// Zstd compression with a configurable level (feature-gated).
    Zstd {
        /// Compression level.
        level: i32,
    },
}

/// SSTable entry type.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum SSTableEntryType {
    /// Put operation.
    Put = 0,
    /// Delete tombstone.
    Delete = 1,
}

impl TryFrom<u8> for SSTableEntryType {
    type Error = Error;

    fn try_from(value: u8) -> Result<Self> {
        match value {
            0 => Ok(Self::Put),
            1 => Ok(Self::Delete),
            _ => Err(Error::InvalidFormat("unknown SSTable entry type".into())),
        }
    }
}

/// One MVCC version stored in an SSTable.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct SSTableEntry {
    /// User key bytes.
    pub key: Key,
    /// Optional value (None = tombstone).
    pub value: Option<Value>,
    /// MVCC timestamp.
    pub timestamp: u64,
    /// Sequence number for tie-breaking.
    pub sequence: u64,
}

fn compare_entries(a: (&[u8], u64, u64), b: (&[u8], u64, u64)) -> CmpOrdering {
    match a.0.cmp(b.0) {
        CmpOrdering::Equal => match b.1.cmp(&a.1) {
            CmpOrdering::Equal => b.2.cmp(&a.2),
            other => other,
        },
        other => other,
    }
}

/// SSTable header (32 bytes).
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub struct SSTableHeader {
    /// Format version.
    pub version: u16,
    /// Flags (bitset).
    pub flags: u16,
    /// Number of data blocks.
    pub block_count: u32,
    /// Number of entries.
    pub entry_count: u64,
    /// Minimum key length observed.
    pub min_key_len: u32,
    /// Maximum key length observed.
    pub max_key_len: u32,
    /// Reserved (currently used for compression flag in low bits and zstd level in high bits).
    pub reserved: u32,
}

impl SSTableHeader {
    /// Serialize to fixed-size bytes.
    pub fn to_bytes(&self) -> [u8; HEADER_SIZE] {
        let mut out = [0u8; HEADER_SIZE];
        out[0..4].copy_from_slice(&SST_MAGIC);
        out[4..6].copy_from_slice(&self.version.to_le_bytes());
        out[6..8].copy_from_slice(&self.flags.to_le_bytes());
        out[8..12].copy_from_slice(&self.block_count.to_le_bytes());
        out[12..20].copy_from_slice(&self.entry_count.to_le_bytes());
        out[20..24].copy_from_slice(&self.min_key_len.to_le_bytes());
        out[24..28].copy_from_slice(&self.max_key_len.to_le_bytes());
        out[28..32].copy_from_slice(&self.reserved.to_le_bytes());
        out
    }

    /// Deserialize and validate.
    pub fn from_bytes(bytes: &[u8; HEADER_SIZE]) -> Result<Self> {
        if bytes[0..4] != SST_MAGIC {
            return Err(Error::InvalidFormat("SSTable header magic mismatch".into()));
        }
        let version = u16::from_le_bytes(bytes[4..6].try_into().expect("fixed slice length"));
        if version != SST_VERSION {
            return Err(Error::InvalidFormat(format!(
                "unsupported SSTable version: {version}"
            )));
        }
        let flags = u16::from_le_bytes(bytes[6..8].try_into().expect("fixed slice length"));
        let block_count = u32::from_le_bytes(bytes[8..12].try_into().expect("fixed slice length"));
        let entry_count = u64::from_le_bytes(bytes[12..20].try_into().expect("fixed slice length"));
        let min_key_len = u32::from_le_bytes(bytes[20..24].try_into().expect("fixed slice length"));
        let max_key_len = u32::from_le_bytes(bytes[24..28].try_into().expect("fixed slice length"));
        let reserved = u32::from_le_bytes(bytes[28..32].try_into().expect("fixed slice length"));
        Ok(Self {
            version,
            flags,
            block_count,
            entry_count,
            min_key_len,
            max_key_len,
            reserved,
        })
    }
}

/// SSTable footer (48 bytes).
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub struct SSTableFooter {
    /// Index block offset (from file start).
    pub index_offset: u64,
    /// Index block size in bytes.
    pub index_size: u32,
    /// Bloom filter offset (from file start) or 0 if absent.
    pub bloom_offset: u64,
    /// Bloom filter size in bytes.
    pub bloom_size: u32,
    /// Minimum timestamp stored in this table.
    pub min_timestamp: u64,
    /// Maximum timestamp stored in this table.
    pub max_timestamp: u64,
    /// CRC32 over the entire file excluding the footer.
    pub file_crc32: u32,
}

impl SSTableFooter {
    /// Serialize to fixed-size bytes.
    pub fn to_bytes(&self) -> [u8; FOOTER_SIZE] {
        let mut out = [0u8; FOOTER_SIZE];
        out[0..8].copy_from_slice(&self.index_offset.to_le_bytes());
        out[8..12].copy_from_slice(&self.index_size.to_le_bytes());
        out[12..20].copy_from_slice(&self.bloom_offset.to_le_bytes());
        out[20..24].copy_from_slice(&self.bloom_size.to_le_bytes());
        out[24..32].copy_from_slice(&self.min_timestamp.to_le_bytes());
        out[32..40].copy_from_slice(&self.max_timestamp.to_le_bytes());
        out[40..44].copy_from_slice(&self.file_crc32.to_le_bytes());
        out[44..48].copy_from_slice(&SST_MAGIC_REVERSE);
        out
    }

    /// Deserialize and validate.
    pub fn from_bytes(bytes: &[u8; FOOTER_SIZE]) -> Result<Self> {
        if bytes[44..48] != SST_MAGIC_REVERSE {
            return Err(Error::InvalidFormat("SSTable footer magic mismatch".into()));
        }
        let index_offset = u64::from_le_bytes(bytes[0..8].try_into().expect("fixed slice length"));
        let index_size = u32::from_le_bytes(bytes[8..12].try_into().expect("fixed slice length"));
        let bloom_offset =
            u64::from_le_bytes(bytes[12..20].try_into().expect("fixed slice length"));
        let bloom_size = u32::from_le_bytes(bytes[20..24].try_into().expect("fixed slice length"));
        let min_timestamp =
            u64::from_le_bytes(bytes[24..32].try_into().expect("fixed slice length"));
        let max_timestamp =
            u64::from_le_bytes(bytes[32..40].try_into().expect("fixed slice length"));
        let file_crc32 = u32::from_le_bytes(bytes[40..44].try_into().expect("fixed slice length"));
        Ok(Self {
            index_offset,
            index_size,
            bloom_offset,
            bloom_size,
            min_timestamp,
            max_timestamp,
            file_crc32,
        })
    }
}

/// Index entry for a data block.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct SSTableIndexEntry {
    /// Block start offset.
    pub offset: u64,
    /// Block size in bytes.
    pub size: u32,
    /// First user key contained.
    pub first_key: Key,
    /// Last user key contained.
    pub last_key: Key,
}

fn encode_u32(out: &mut Vec<u8>, v: u32) {
    out.extend_from_slice(&v.to_le_bytes());
}

fn encode_u64(out: &mut Vec<u8>, v: u64) {
    out.extend_from_slice(&v.to_le_bytes());
}

fn decode_u32(input: &[u8], cursor: &mut usize) -> Result<u32> {
    if input.len() < *cursor + 4 {
        return Err(Error::InvalidFormat("SSTable block/index truncated".into()));
    }
    let v = u32::from_le_bytes(
        input[*cursor..*cursor + 4]
            .try_into()
            .expect("fixed slice length"),
    );
    *cursor += 4;
    Ok(v)
}

fn decode_u64(input: &[u8], cursor: &mut usize) -> Result<u64> {
    if input.len() < *cursor + 8 {
        return Err(Error::InvalidFormat("SSTable block/index truncated".into()));
    }
    let v = u64::from_le_bytes(
        input[*cursor..*cursor + 8]
            .try_into()
            .expect("fixed slice length"),
    );
    *cursor += 8;
    Ok(v)
}

fn encode_index(entries: &[SSTableIndexEntry]) -> Vec<u8> {
    let mut out = Vec::new();
    encode_u32(&mut out, entries.len() as u32);
    for e in entries {
        encode_u64(&mut out, e.offset);
        encode_u32(&mut out, e.size);
        encode_u32(&mut out, e.first_key.len() as u32);
        out.extend_from_slice(&e.first_key);
        encode_u32(&mut out, e.last_key.len() as u32);
        out.extend_from_slice(&e.last_key);
    }
    out
}

fn decode_index(bytes: &[u8]) -> Result<Vec<SSTableIndexEntry>> {
    let mut cursor = 0usize;
    let count = decode_u32(bytes, &mut cursor)? as usize;
    let mut out = Vec::with_capacity(count);
    for _ in 0..count {
        let offset = decode_u64(bytes, &mut cursor)?;
        let size = decode_u32(bytes, &mut cursor)?;
        let first_len = decode_u32(bytes, &mut cursor)? as usize;
        if bytes.len() < cursor + first_len {
            return Err(Error::InvalidFormat(
                "SSTable index truncated (first key)".into(),
            ));
        }
        let first_key = bytes[cursor..cursor + first_len].to_vec();
        cursor += first_len;
        let last_len = decode_u32(bytes, &mut cursor)? as usize;
        if bytes.len() < cursor + last_len {
            return Err(Error::InvalidFormat(
                "SSTable index truncated (last key)".into(),
            ));
        }
        let last_key = bytes[cursor..cursor + last_len].to_vec();
        cursor += last_len;
        out.push(SSTableIndexEntry {
            offset,
            size,
            first_key,
            last_key,
        });
    }
    if cursor != bytes.len() {
        return Err(Error::InvalidFormat(
            "SSTable index block has trailing bytes".into(),
        ));
    }
    Ok(out)
}

/// Simple Bloom filter (bitset + k hashes).
#[derive(Debug, Clone)]
pub struct BloomFilter {
    k: u32,
    bits: Vec<u8>,
}

impl BloomFilter {
    fn new(k: u32, bits: Vec<u8>) -> Self {
        Self { k, bits }
    }

    fn bit_len(&self) -> u32 {
        (self.bits.len() as u32) * 8
    }

    fn set_bit(&mut self, bit: u32) {
        let idx = (bit / 8) as usize;
        let off = (bit % 8) as u8;
        self.bits[idx] |= 1u8 << off;
    }

    fn get_bit(&self, bit: u32) -> bool {
        let idx = (bit / 8) as usize;
        let off = (bit % 8) as u8;
        (self.bits[idx] & (1u8 << off)) != 0
    }

    fn hashes(&self, key: &[u8]) -> (u32, u32) {
        let h1 = crc32fast::hash(key);
        let mut tmp = Vec::with_capacity(key.len() + 1);
        tmp.extend_from_slice(key);
        tmp.push(0xA5);
        let h2 = crc32fast::hash(&tmp);
        (h1, h2.max(1))
    }

    fn insert(&mut self, key: &[u8]) {
        let (h1, h2) = self.hashes(key);
        let m = self.bit_len();
        for i in 0..self.k {
            let bit = h1.wrapping_add(i.wrapping_mul(h2)) % m;
            self.set_bit(bit);
        }
    }

    /// Check if the key may be present.
    pub fn may_contain(&self, key: &[u8]) -> bool {
        let (h1, h2) = self.hashes(key);
        let m = self.bit_len();
        for i in 0..self.k {
            let bit = h1.wrapping_add(i.wrapping_mul(h2)) % m;
            if !self.get_bit(bit) {
                return false;
            }
        }
        true
    }

    fn to_bytes(&self) -> Vec<u8> {
        let mut out = Vec::new();
        encode_u32(&mut out, self.k);
        encode_u32(&mut out, self.bits.len() as u32);
        out.extend_from_slice(&self.bits);
        out
    }

    fn from_bytes(bytes: &[u8]) -> Result<Self> {
        let mut cursor = 0usize;
        let k = decode_u32(bytes, &mut cursor)?;
        let len = decode_u32(bytes, &mut cursor)? as usize;
        if bytes.len() < cursor + len {
            return Err(Error::InvalidFormat("SSTable bloom truncated".into()));
        }
        let bits = bytes[cursor..cursor + len].to_vec();
        cursor += len;
        if cursor != bytes.len() {
            return Err(Error::InvalidFormat(
                "SSTable bloom has trailing bytes".into(),
            ));
        }
        Ok(Self::new(k, bits))
    }
}

fn bloom_params(entry_count: u64, fpr: f64) -> (u32, usize) {
    let n = entry_count.max(1) as f64;
    let p = fpr.clamp(1e-9, 0.5);
    let m_bits = (-(n * p.ln()) / (2f64.ln().powi(2))).ceil().max(8.0);
    let k = ((m_bits / n) * 2f64.ln()).round().max(1.0);
    let m_bytes = (m_bits as usize).div_ceil(8);
    (k as u32, m_bytes.max(1))
}

/// Writer for a single SSTable file.
pub struct SSTableWriter {
    file: File,
    config: SSTableConfig,
    compressor: Box<dyn Compressor>,
    data_block_buf: Vec<u8>,
    data_block_entry_count: u32,
    block_index: Vec<SSTableIndexEntry>,
    entry_count: u64,
    last_key: Option<(Key, u64, u64)>,
    first_key_in_block: Option<Key>,
    last_key_in_block: Option<Key>,
    min_key_len: u32,
    max_key_len: u32,
    min_timestamp: u64,
    max_timestamp: u64,
    bloom: Option<BloomFilter>,
    /// Distinct user keys encountered (used to build Bloom filter at finish()).
    ///
    /// Note: this is an in-memory buffer. For very large tables it can still be substantial; a
    /// future improvement is to support incremental Bloom insertion with an upfront size estimate.
    bloom_keys: Vec<Key>,
    last_bloom_key: Option<Key>,
    closed: bool,
}

impl SSTableWriter {
    /// Create a new SSTable file.
    pub fn create(path: &Path, config: SSTableConfig) -> Result<Self> {
        if config.block_size < 64 {
            return Err(Error::InvalidFormat("SSTable block_size too small".into()));
        }

        if let Some(parent) = path.parent() {
            std::fs::create_dir_all(parent)?;
        }
        let file = OpenOptions::new()
            .create(true)
            .truncate(true)
            .read(true)
            .write(true)
            .open(path)?;

        let compression_v2 = compression_to_v2(config.compression);
        let compressor = create_compressor(compression_v2)
            .map_err(|e| Error::InvalidFormat(format!("SSTable compression unavailable: {e:?}")))?;

        let mut writer = Self {
            file,
            config,
            compressor,
            data_block_buf: Vec::new(),
            data_block_entry_count: 0,
            block_index: Vec::new(),
            entry_count: 0,
            last_key: None,
            first_key_in_block: None,
            last_key_in_block: None,
            min_key_len: u32::MAX,
            max_key_len: 0,
            min_timestamp: u64::MAX,
            max_timestamp: 0,
            bloom: None,
            bloom_keys: Vec::new(),
            last_bloom_key: None,
            closed: false,
        };

        // Placeholder header.
        let header = SSTableHeader {
            version: SST_VERSION,
            flags: 0,
            block_count: 0,
            entry_count: 0,
            min_key_len: 0,
            max_key_len: 0,
            reserved: 0,
        };
        writer.file.write_all(&header.to_bytes())?;

        Ok(writer)
    }

    fn write_bytes(&mut self, bytes: &[u8]) -> Result<()> {
        self.file.write_all(bytes)?;
        Ok(())
    }

    fn current_offset(&mut self) -> Result<u64> {
        Ok(self.file.stream_position()?)
    }

    fn encode_entry_into_block(&mut self, entry: &SSTableEntry) -> Result<()> {
        let entry_type = if entry.value.is_some() {
            SSTableEntryType::Put
        } else {
            SSTableEntryType::Delete
        };
        let key_len = entry.key.len();
        if key_len > u32::MAX as usize {
            return Err(Error::InvalidFormat("SSTable key too large".into()));
        }
        let val_len = entry.value.as_ref().map(|v| v.len()).unwrap_or(0);
        if val_len > u32::MAX as usize {
            return Err(Error::InvalidFormat("SSTable value too large".into()));
        }

        self.min_key_len = self.min_key_len.min(key_len as u32);
        self.max_key_len = self.max_key_len.max(key_len as u32);
        self.min_timestamp = self.min_timestamp.min(entry.timestamp);
        self.max_timestamp = self.max_timestamp.max(entry.timestamp);

        if self.first_key_in_block.is_none() {
            self.first_key_in_block = Some(entry.key.clone());
        }
        self.last_key_in_block = Some(entry.key.clone());

        self.data_block_buf.push(entry_type as u8);
        encode_u64(&mut self.data_block_buf, entry.timestamp);
        encode_u64(&mut self.data_block_buf, entry.sequence);
        encode_u32(&mut self.data_block_buf, key_len as u32);
        encode_u32(&mut self.data_block_buf, val_len as u32);
        self.data_block_buf.extend_from_slice(&entry.key);
        if let Some(v) = &entry.value {
            self.data_block_buf.extend_from_slice(v);
        }
        Ok(())
    }

    fn flush_block_if_needed(&mut self, upcoming_len: usize) -> Result<()> {
        let projected = self.data_block_buf.len().saturating_add(upcoming_len);
        let overhead = 4 /*entry_count*/ + 4 /*uncompressed_len*/ + 4 /*crc*/;
        if self.data_block_entry_count > 0 && projected + overhead > self.config.block_size {
            self.finish_block()?;
        }
        Ok(())
    }

    fn finish_block(&mut self) -> Result<()> {
        if self.data_block_entry_count == 0 {
            return Ok(());
        }

        let entry_count = self.data_block_entry_count;
        let uncompressed = std::mem::take(&mut self.data_block_buf);
        let compressed = self
            .compressor
            .compress(&uncompressed)
            .map_err(|e| Error::InvalidFormat(format!("SSTable compression failed: {e:?}")))?;

        let uncompressed_len: u32 = uncompressed
            .len()
            .try_into()
            .map_err(|_| Error::InvalidFormat("SSTable block too large".into()))?;
        let mut block = Vec::with_capacity(4 + 4 + compressed.len() + 4);
        block.extend_from_slice(&entry_count.to_le_bytes());
        block.extend_from_slice(&uncompressed_len.to_le_bytes());
        block.extend_from_slice(&compressed);
        let c = crc32(&block);
        block.extend_from_slice(&c.to_le_bytes());

        let offset = self.current_offset()?;
        self.write_bytes(&block)?;

        let first_key = self
            .first_key_in_block
            .take()
            .expect("block has at least one entry");
        let last_key = self
            .last_key_in_block
            .take()
            .expect("block has at least one entry");
        let size = block
            .len()
            .try_into()
            .map_err(|_| Error::InvalidFormat("SSTable block size overflow".into()))?;
        self.block_index.push(SSTableIndexEntry {
            offset,
            size,
            first_key,
            last_key,
        });

        self.data_block_entry_count = 0;
        Ok(())
    }

    /// Append an entry (must be in sorted order).
    pub fn append(&mut self, entry: SSTableEntry) -> Result<()> {
        if self.closed {
            return Err(Error::InvalidFormat("SSTable writer already closed".into()));
        }

        if let Some((k, ts, seq)) = &self.last_key {
            let ord = compare_entries(
                (k.as_slice(), *ts, *seq),
                (entry.key.as_slice(), entry.timestamp, entry.sequence),
            );
            if ord != CmpOrdering::Less {
                return Err(Error::InvalidFormat(
                    "SSTable entries must be appended in sorted order".into(),
                ));
            }
        }
        self.last_key = Some((entry.key.clone(), entry.timestamp, entry.sequence));

        if self.config.enable_bloom_filter
            && self
                .last_bloom_key
                .as_ref()
                .is_none_or(|k| k.as_slice() != entry.key.as_slice())
        {
            self.bloom_keys.push(entry.key.clone());
            self.last_bloom_key = Some(entry.key.clone());
        }

        // Estimate entry size in block (uncompressed).
        let val_len = entry.value.as_ref().map(|v| v.len()).unwrap_or(0);
        let upcoming = 1 + 8 + 8 + 4 + 4 + entry.key.len() + val_len;
        self.flush_block_if_needed(upcoming)?;
        self.encode_entry_into_block(&entry)?;
        self.data_block_entry_count += 1;
        self.entry_count += 1;
        Ok(())
    }

    /// Finish writing and fsync the SSTable.
    pub fn finish(mut self) -> Result<()> {
        if self.closed {
            return Err(Error::InvalidFormat("SSTable writer already closed".into()));
        }
        self.closed = true;

        // Build Bloom filter now that entry count is known (using buffered keys).
        if self.config.enable_bloom_filter {
            let (k, bytes) = bloom_params(self.entry_count, self.config.bloom_filter_fpr);
            let mut bloom = BloomFilter::new(k, vec![0u8; bytes]);
            for k in &self.bloom_keys {
                bloom.insert(k);
            }
            self.bloom = Some(bloom);
        }

        self.finish_block()?;

        // Write index block.
        let index_offset = self.current_offset()?;
        let index_bytes = encode_index(&self.block_index);
        self.write_bytes(&index_bytes)?;
        let index_size = index_bytes
            .len()
            .try_into()
            .map_err(|_| Error::InvalidFormat("SSTable index size overflow".into()))?;

        // Write bloom filter (optional).
        let bloom_bytes = self.bloom.as_ref().map(BloomFilter::to_bytes);
        let (bloom_offset, bloom_size) = if let Some(bytes) = bloom_bytes {
            let off = self.current_offset()?;
            self.write_bytes(&bytes)?;
            let size = bytes
                .len()
                .try_into()
                .map_err(|_| Error::InvalidFormat("SSTable bloom size overflow".into()))?;
            (off, size)
        } else {
            (0, 0)
        };

        let footer_offset = self.current_offset()?;

        // Patch header at offset 0.
        let mut flags = 0u16;
        if bloom_size > 0 {
            flags |= FLAG_BLOOM_PRESENT;
        }
        let compression_flag = compression_to_flag(self.config.compression) & 0x3;
        flags |= (compression_flag << 1) & FLAG_COMPRESSION_MASK;
        let zstd_level = match self.config.compression {
            CompressionType::Zstd { level } => level,
            _ => 0,
        };
        let zstd_level = zstd_level.clamp(0, 255) as u32;
        let reserved = zstd_level << 8;
        let header = SSTableHeader {
            version: SST_VERSION,
            flags,
            block_count: self.block_index.len() as u32,
            entry_count: self.entry_count,
            min_key_len: if self.entry_count == 0 {
                0
            } else {
                self.min_key_len
            },
            max_key_len: self.max_key_len,
            reserved,
        };
        self.file.seek(SeekFrom::Start(0))?;
        self.file.write_all(&header.to_bytes())?;

        // Compute CRC over the entire file excluding the footer.
        self.file.seek(SeekFrom::Start(0))?;
        let mut hasher = crc32fast::Hasher::new();
        let mut remaining = footer_offset;
        let mut buf = vec![0u8; 64 * 1024];
        while remaining > 0 {
            let chunk = (remaining as usize).min(buf.len());
            self.file.read_exact(&mut buf[..chunk])?;
            hasher.update(&buf[..chunk]);
            remaining -= chunk as u64;
        }
        let file_crc32 = hasher.finalize();

        let footer = SSTableFooter {
            index_offset,
            index_size,
            bloom_offset,
            bloom_size,
            min_timestamp: if self.entry_count == 0 {
                0
            } else {
                self.min_timestamp
            },
            max_timestamp: self.max_timestamp,
            file_crc32,
        };

        // Footer is not part of file CRC.
        self.file.seek(SeekFrom::Start(footer_offset))?;
        self.file.write_all(&footer.to_bytes())?;
        self.file.sync_data()?;
        Ok(())
    }
}

/// Reader for SSTable files.
pub struct SSTableReader {
    file: File,
    _path: PathBuf,
    header: SSTableHeader,
    footer: SSTableFooter,
    index: Vec<SSTableIndexEntry>,
    bloom: Option<BloomFilter>,
    compressor: Box<dyn Compressor>,
}

impl SSTableReader {
    /// Open and validate an SSTable file.
    pub fn open(path: &Path) -> Result<Self> {
        let mut file = OpenOptions::new().read(true).open(path)?;
        let file_len = file.metadata()?.len();
        if file_len < (HEADER_SIZE + FOOTER_SIZE) as u64 {
            return Err(Error::InvalidFormat("SSTable file too small".into()));
        }
        let data_end = file_len - (FOOTER_SIZE as u64);

        let mut header_bytes = [0u8; HEADER_SIZE];
        file.seek(SeekFrom::Start(0))?;
        file.read_exact(&mut header_bytes)?;
        let header = SSTableHeader::from_bytes(&header_bytes)?;

        let mut footer_bytes = [0u8; FOOTER_SIZE];
        file.seek(SeekFrom::Start(file_len - (FOOTER_SIZE as u64)))?;
        file.read_exact(&mut footer_bytes)?;
        let footer = SSTableFooter::from_bytes(&footer_bytes)?;

        let header_bloom_present = (header.flags & FLAG_BLOOM_PRESENT) != 0;
        let footer_bloom_present = footer.bloom_size > 0;
        if header_bloom_present != footer_bloom_present {
            return Err(Error::InvalidFormat(
                "SSTable bloom presence mismatch between header flag and footer size".into(),
            ));
        }

        // Validate footer ranges early to avoid OOM / invalid seeks on corrupted files.
        let index_end = footer
            .index_offset
            .checked_add(footer.index_size as u64)
            .ok_or_else(|| Error::InvalidFormat("SSTable index range overflow".into()))?;
        if footer.index_offset < HEADER_SIZE as u64 || index_end > data_end {
            return Err(Error::InvalidFormat(
                "SSTable index range is out of bounds".into(),
            ));
        }

        let has_bloom = footer_bloom_present;
        let bloom_end = footer
            .bloom_offset
            .checked_add(footer.bloom_size as u64)
            .ok_or_else(|| Error::InvalidFormat("SSTable bloom range overflow".into()))?;
        if has_bloom {
            if footer.bloom_offset < HEADER_SIZE as u64 || bloom_end > data_end {
                return Err(Error::InvalidFormat(
                    "SSTable bloom range is out of bounds".into(),
                ));
            }
            let index_range = (footer.index_offset, index_end);
            let bloom_range = (footer.bloom_offset, bloom_end);
            let overlaps = index_range.0 < bloom_range.1 && bloom_range.0 < index_range.1;
            if overlaps {
                return Err(Error::InvalidFormat(
                    "SSTable index and bloom ranges overlap".into(),
                ));
            }
        }

        // Validate file CRC.
        file.seek(SeekFrom::Start(0))?;
        let mut hasher = crc32fast::Hasher::new();
        let mut remaining = data_end;
        let mut buf = vec![0u8; 64 * 1024];
        while remaining > 0 {
            let chunk = (remaining as usize).min(buf.len());
            file.read_exact(&mut buf[..chunk])?;
            hasher.update(&buf[..chunk]);
            remaining -= chunk as u64;
        }
        let computed = hasher.finalize();
        if computed != footer.file_crc32 {
            return Err(Error::ChecksumMismatch);
        }

        // Decode compression from header flags/reserved.
        let compression_flag = (header.flags & FLAG_COMPRESSION_MASK) >> 1;
        let zstd_level = ((header.reserved >> 8) & 0x00FF) as i32;
        let compression = compression_from_flag(compression_flag, zstd_level)?;
        let compressor = create_compressor(compression_to_v2(compression))
            .map_err(|e| Error::InvalidFormat(format!("SSTable compression unavailable: {e:?}")))?;

        // Load index block.
        file.seek(SeekFrom::Start(footer.index_offset))?;
        let mut index_bytes = vec![0u8; footer.index_size as usize];
        file.read_exact(&mut index_bytes)?;
        let index = decode_index(&index_bytes)?;

        // Load bloom filter if present.
        let bloom = if footer_bloom_present {
            file.seek(SeekFrom::Start(footer.bloom_offset))?;
            let mut bloom_bytes = vec![0u8; footer.bloom_size as usize];
            file.read_exact(&mut bloom_bytes)?;
            Some(BloomFilter::from_bytes(&bloom_bytes)?)
        } else {
            None
        };

        Ok(Self {
            file,
            _path: path.to_path_buf(),
            header,
            footer,
            index,
            bloom,
            compressor,
        })
    }

    /// Return the overall key range for this SSTable based on index entries.
    pub fn key_range(&self) -> Option<(Key, Key)> {
        let first = self.index.first()?.first_key.clone();
        let last = self.index.last()?.last_key.clone();
        Some((first, last))
    }

    /// Return the number of entries.
    pub fn entry_count(&self) -> u64 {
        self.header.entry_count
    }

    /// Minimum timestamp stored in this table.
    pub fn min_timestamp(&self) -> u64 {
        self.footer.min_timestamp
    }

    /// Maximum timestamp stored in this table.
    pub fn max_timestamp(&self) -> u64 {
        self.footer.max_timestamp
    }

    /// Get the latest visible entry for the key at `read_timestamp`.
    pub fn get(&mut self, key: &[u8], read_timestamp: u64) -> Result<Option<SSTableEntry>> {
        self.get_cached(None, None, 0, key, read_timestamp)
    }

    /// バッファプールを利用して `get` を行う。
    pub fn get_with_buffer_pool(
        &mut self,
        buffer_pool: &BufferPool,
        metrics: &LsmMetrics,
        file_id: u64,
        key: &[u8],
        read_timestamp: u64,
    ) -> Result<Option<SSTableEntry>> {
        self.get_cached(
            Some(buffer_pool),
            Some(metrics),
            file_id,
            key,
            read_timestamp,
        )
    }

    fn get_cached(
        &mut self,
        buffer_pool: Option<&BufferPool>,
        metrics: Option<&LsmMetrics>,
        file_id: u64,
        key: &[u8],
        read_timestamp: u64,
    ) -> Result<Option<SSTableEntry>> {
        if let Some(bloom) = &self.bloom {
            if !bloom.may_contain(key) {
                return Ok(None);
            }
        }

        let mut left = 0usize;
        let mut right = self.index.len();
        while left < right {
            let mid = (left + right) / 2;
            if self.index[mid].last_key.as_slice() < key {
                left = mid + 1;
            } else {
                right = mid;
            }
        }
        if left >= self.index.len() {
            return Ok(None);
        }
        let mut idx = left;
        while idx < self.index.len() {
            let block_meta = &self.index[idx];
            if block_meta.first_key.as_slice() > key {
                break;
            }
            if key <= block_meta.last_key.as_slice() {
                let block = self.read_block_cached(buffer_pool, metrics, file_id, idx)?;
                let entries = self.decode_block_entries(&block)?;
                if let Some(found) = find_in_entries(&entries, key, read_timestamp) {
                    return Ok(Some(found));
                }
            }
            idx += 1;
        }
        Ok(None)
    }

    fn read_block_cached(
        &mut self,
        buffer_pool: Option<&BufferPool>,
        metrics: Option<&LsmMetrics>,
        file_id: u64,
        idx: usize,
    ) -> Result<Vec<u8>> {
        let entry = self
            .index
            .get(idx)
            .ok_or_else(|| Error::InvalidFormat("SSTable index out of bounds".into()))?;

        if let Some(pool) = buffer_pool {
            let id = BlockId {
                file_id,
                block_offset: entry.offset,
            };
            if let Some(hit) = pool.get(&id) {
                return Ok(hit.as_slice().to_vec());
            }
        }

        self.file.seek(SeekFrom::Start(entry.offset))?;
        let mut bytes = vec![0u8; entry.size as usize];
        self.file.read_exact(&mut bytes)?;
        if let Some(m) = metrics {
            m.add_sstable_read_bytes(bytes.len() as u64);
        }

        if bytes.len() < 4 + 4 {
            return Err(Error::InvalidFormat("SSTable block too small".into()));
        }
        let stored_crc = u32::from_le_bytes(
            bytes[bytes.len() - 4..]
                .try_into()
                .expect("fixed slice length"),
        );
        let computed_crc = crc32(&bytes[..bytes.len() - 4]);
        if stored_crc != computed_crc {
            return Err(Error::ChecksumMismatch);
        }

        if let Some(pool) = buffer_pool {
            let id = BlockId {
                file_id,
                block_offset: entry.offset,
            };
            let _ = pool.put(id, Arc::new(DataBlock::new(bytes.clone())));
        }
        Ok(bytes)
    }

    fn decode_block_entries(&mut self, block: &[u8]) -> Result<Vec<SSTableEntry>> {
        if block.len() < 4 + 4 + 4 {
            return Err(Error::InvalidFormat("SSTable block too small".into()));
        }
        let entry_count =
            u32::from_le_bytes(block[0..4].try_into().expect("fixed slice length")) as usize;
        let uncompressed_len =
            u32::from_le_bytes(block[4..8].try_into().expect("fixed slice length")) as usize;
        let payload = &block[8..block.len() - 4];
        let decompressed = self
            .compressor
            .decompress(payload, uncompressed_len)
            .map_err(|e| Error::InvalidFormat(format!("SSTable decompression failed: {e:?}")))?;
        decode_entries(&decompressed, entry_count)
    }

    /// Scan keys with the given prefix, returning the latest visible version per key.
    ///
    /// Note: tombstones are returned as entries with `value == None`.
    pub fn scan_prefix(&mut self, prefix: &[u8], read_timestamp: u64) -> Result<Vec<SSTableEntry>> {
        self.scan_prefix_cached(None, None, 0, prefix, read_timestamp)
    }

    /// バッファプールを利用して `scan_prefix` を行う。
    pub fn scan_prefix_with_buffer_pool(
        &mut self,
        buffer_pool: &BufferPool,
        metrics: &LsmMetrics,
        file_id: u64,
        prefix: &[u8],
        read_timestamp: u64,
    ) -> Result<Vec<SSTableEntry>> {
        self.scan_prefix_cached(
            Some(buffer_pool),
            Some(metrics),
            file_id,
            prefix,
            read_timestamp,
        )
    }

    fn scan_prefix_cached(
        &mut self,
        buffer_pool: Option<&BufferPool>,
        metrics: Option<&LsmMetrics>,
        file_id: u64,
        prefix: &[u8],
        read_timestamp: u64,
    ) -> Result<Vec<SSTableEntry>> {
        let end = next_prefix(prefix);
        self.scan_range_cached(
            buffer_pool,
            metrics,
            file_id,
            prefix,
            end.as_deref().unwrap_or(&[]),
            read_timestamp,
        )
    }

    /// Scan keys in `[start, end)`, returning the latest visible version per key.
    ///
    /// Note: tombstones are returned as entries with `value == None`.
    pub fn scan_range(
        &mut self,
        start: &[u8],
        end: &[u8],
        read_timestamp: u64,
    ) -> Result<Vec<SSTableEntry>> {
        self.scan_range_cached(None, None, 0, start, end, read_timestamp)
    }

    /// バッファプールを利用して `scan_range` を行う。
    pub fn scan_range_with_buffer_pool(
        &mut self,
        buffer_pool: &BufferPool,
        metrics: &LsmMetrics,
        file_id: u64,
        start: &[u8],
        end: &[u8],
        read_timestamp: u64,
    ) -> Result<Vec<SSTableEntry>> {
        self.scan_range_cached(
            Some(buffer_pool),
            Some(metrics),
            file_id,
            start,
            end,
            read_timestamp,
        )
    }

    fn scan_range_cached(
        &mut self,
        buffer_pool: Option<&BufferPool>,
        metrics: Option<&LsmMetrics>,
        file_id: u64,
        start: &[u8],
        end: &[u8],
        read_timestamp: u64,
    ) -> Result<Vec<SSTableEntry>> {
        let mut out = Vec::new();
        let mut last_user_key: Option<Vec<u8>> = None;

        for block_idx in 0..self.index.len() {
            let meta = &self.index[block_idx];
            if !end.is_empty() && meta.first_key.as_slice() >= end {
                break;
            }
            if meta.last_key.as_slice() < start {
                continue;
            }
            let block = self.read_block_cached(buffer_pool, metrics, file_id, block_idx)?;
            let entries = self.decode_block_entries(&block)?;
            for e in entries {
                if e.key.as_slice() < start {
                    continue;
                }
                if !end.is_empty() && e.key.as_slice() >= end {
                    return Ok(out);
                }
                if last_user_key.as_deref() == Some(e.key.as_slice()) {
                    continue;
                }
                if e.timestamp > read_timestamp {
                    continue;
                }
                last_user_key = Some(e.key.clone());
                out.push(e);
            }
        }
        Ok(out)
    }
}

fn next_prefix(prefix: &[u8]) -> Option<Vec<u8>> {
    if prefix.is_empty() {
        return None;
    }
    let mut out = prefix.to_vec();
    for i in (0..out.len()).rev() {
        if out[i] != 0xFF {
            out[i] = out[i].wrapping_add(1);
            out.truncate(i + 1);
            return Some(out);
        }
    }
    None
}

fn decode_entries(bytes: &[u8], expected_count: usize) -> Result<Vec<SSTableEntry>> {
    let mut cursor = 0usize;
    let mut out = Vec::with_capacity(expected_count);
    while cursor < bytes.len() {
        if bytes.len() < cursor + 1 + 8 + 8 + 4 + 4 {
            return Err(Error::InvalidFormat("SSTable entry truncated".into()));
        }
        let entry_type = SSTableEntryType::try_from(bytes[cursor])?;
        cursor += 1;
        let timestamp = u64::from_le_bytes(
            bytes[cursor..cursor + 8]
                .try_into()
                .expect("fixed slice length"),
        );
        cursor += 8;
        let sequence = u64::from_le_bytes(
            bytes[cursor..cursor + 8]
                .try_into()
                .expect("fixed slice length"),
        );
        cursor += 8;
        let key_len = u32::from_le_bytes(
            bytes[cursor..cursor + 4]
                .try_into()
                .expect("fixed slice length"),
        ) as usize;
        cursor += 4;
        let value_len = u32::from_le_bytes(
            bytes[cursor..cursor + 4]
                .try_into()
                .expect("fixed slice length"),
        ) as usize;
        cursor += 4;
        if bytes.len() < cursor + key_len + value_len {
            return Err(Error::InvalidFormat("SSTable entry truncated (kv)".into()));
        }
        let key = bytes[cursor..cursor + key_len].to_vec();
        cursor += key_len;
        let value = match entry_type {
            SSTableEntryType::Put => Some(bytes[cursor..cursor + value_len].to_vec()),
            SSTableEntryType::Delete => None,
        };
        cursor += value_len;
        out.push(SSTableEntry {
            key,
            value,
            timestamp,
            sequence,
        });
    }
    if out.len() != expected_count {
        return Err(Error::InvalidFormat("SSTable entry count mismatch".into()));
    }
    Ok(out)
}

fn find_in_entries(
    entries: &[SSTableEntry],
    key: &[u8],
    read_timestamp: u64,
) -> Option<SSTableEntry> {
    for e in entries {
        match e.key.as_slice().cmp(key) {
            CmpOrdering::Less => continue,
            CmpOrdering::Greater => break,
            CmpOrdering::Equal => {
                if e.timestamp <= read_timestamp {
                    return Some(e.clone());
                }
            }
        }
    }
    None
}

#[cfg(all(test, not(target_arch = "wasm32")))]
mod tests {
    use super::*;
    use tempfile::tempdir;

    #[test]
    fn sstable_header_footer_roundtrip() {
        let h = SSTableHeader {
            version: SST_VERSION,
            flags: 3,
            block_count: 10,
            entry_count: 99,
            min_key_len: 1,
            max_key_len: 7,
            reserved: 0xABCD,
        };
        let bytes = h.to_bytes();
        let d = SSTableHeader::from_bytes(&bytes).unwrap();
        assert_eq!(d, h);

        let f = SSTableFooter {
            index_offset: 123,
            index_size: 456,
            bloom_offset: 0,
            bloom_size: 0,
            min_timestamp: 10,
            max_timestamp: 20,
            file_crc32: 0xDEADBEEF,
        };
        let bytes = f.to_bytes();
        let d = SSTableFooter::from_bytes(&bytes).unwrap();
        assert_eq!(d, f);
    }

    #[test]
    fn sstable_writer_creates_file() {
        let dir = tempdir().unwrap();
        let path = dir.path().join("t.sst");
        let config = SSTableConfig {
            enable_bloom_filter: false,
            compression: CompressionType::None,
            ..Default::default()
        };
        let writer = SSTableWriter::create(&path, config).unwrap();
        drop(writer);
    }

    #[test]
    fn sstable_writer_reader_roundtrip_get_versions() {
        let dir = tempdir().unwrap();
        let path = dir.path().join("t2.sst");
        let config = SSTableConfig {
            enable_bloom_filter: true,
            compression: CompressionType::None,
            block_size: 256,
            ..Default::default()
        };

        let mut w = SSTableWriter::create(&path, config).unwrap();
        w.append(SSTableEntry {
            key: b"a".to_vec(),
            value: Some(b"v2".to_vec()),
            timestamp: 20,
            sequence: 1,
        })
        .unwrap();
        w.append(SSTableEntry {
            key: b"a".to_vec(),
            value: Some(b"v1".to_vec()),
            timestamp: 10,
            sequence: 1,
        })
        .unwrap();
        w.append(SSTableEntry {
            key: b"b".to_vec(),
            value: None,
            timestamp: 15,
            sequence: 1,
        })
        .unwrap();
        w.append(SSTableEntry {
            key: b"c".to_vec(),
            value: Some(Vec::new()),
            timestamp: 7,
            sequence: 1,
        })
        .unwrap();
        w.finish().unwrap();

        let mut r = SSTableReader::open(&path).unwrap();
        assert_eq!(r.get(b"a", 9).unwrap(), None);
        assert_eq!(
            r.get(b"a", 10).unwrap().unwrap().value.unwrap(),
            b"v1".to_vec()
        );
        assert_eq!(
            r.get(b"a", 20).unwrap().unwrap().value.unwrap(),
            b"v2".to_vec()
        );
        assert!(r.get(b"b", 99).unwrap().unwrap().value.is_none());
        assert_eq!(
            r.get(b"c", 99).unwrap().unwrap().value.unwrap(),
            Vec::<u8>::new()
        );
    }

    #[test]
    fn sstable_get_handles_versions_split_across_blocks() {
        let dir = tempdir().unwrap();
        let path = dir.path().join("t3.sst");
        let config = SSTableConfig {
            enable_bloom_filter: false,
            compression: CompressionType::None,
            block_size: 128,
            ..Default::default()
        };
        let mut w = SSTableWriter::create(&path, config).unwrap();
        w.append(SSTableEntry {
            key: b"k".to_vec(),
            value: Some(vec![0xAA; 64]),
            timestamp: 20,
            sequence: 1,
        })
        .unwrap();
        w.append(SSTableEntry {
            key: b"k".to_vec(),
            value: Some(vec![0xBB; 64]),
            timestamp: 10,
            sequence: 1,
        })
        .unwrap();
        w.finish().unwrap();

        let mut r = SSTableReader::open(&path).unwrap();
        assert_eq!(r.get(b"k", 15).unwrap().unwrap().timestamp, 10);
        assert_eq!(r.get(b"k", 25).unwrap().unwrap().timestamp, 20);
    }

    #[test]
    fn sstable_scan_prefix_returns_latest_visible_per_key_in_reopen() {
        let dir = tempdir().unwrap();
        let path = dir.path().join("scan_prefix.sst");
        let config = SSTableConfig {
            enable_bloom_filter: true,
            compression: CompressionType::None,
            block_size: 256,
            ..Default::default()
        };

        {
            let mut w = SSTableWriter::create(&path, config).unwrap();
            w.append(SSTableEntry {
                key: b"p:a".to_vec(),
                value: Some(b"v2".to_vec()),
                timestamp: 20,
                sequence: 1,
            })
            .unwrap();
            w.append(SSTableEntry {
                key: b"p:a".to_vec(),
                value: Some(b"v1".to_vec()),
                timestamp: 10,
                sequence: 1,
            })
            .unwrap();
            w.append(SSTableEntry {
                key: b"p:b".to_vec(),
                value: Some(b"x".to_vec()),
                timestamp: 15,
                sequence: 1,
            })
            .unwrap();
            w.append(SSTableEntry {
                key: b"p:c".to_vec(),
                value: None,
                timestamp: 12,
                sequence: 1,
            })
            .unwrap();
            w.append(SSTableEntry {
                key: b"q:z".to_vec(),
                value: Some(b"no".to_vec()),
                timestamp: 99,
                sequence: 1,
            })
            .unwrap();
            w.finish().unwrap();
        }

        // reopen して scan できること
        let mut r = SSTableReader::open(&path).unwrap();
        let got = r.scan_prefix(b"p:", 20).unwrap();
        assert_eq!(got.len(), 3);
        assert_eq!(got[0].key, b"p:a".to_vec());
        assert_eq!(got[0].value.as_deref(), Some(b"v2".as_slice()));
        assert_eq!(got[1].key, b"p:b".to_vec());
        assert_eq!(got[2].key, b"p:c".to_vec());
        assert!(got[2].value.is_none());

        // read_timestamp により、見える版が変わる
        let got = r.scan_prefix(b"p:", 15).unwrap();
        assert_eq!(got[0].key, b"p:a".to_vec());
        assert_eq!(got[0].value.as_deref(), Some(b"v1".as_slice()));
    }

    #[test]
    fn sstable_scan_range_is_end_exclusive() {
        let dir = tempdir().unwrap();
        let path = dir.path().join("scan_range.sst");
        let config = SSTableConfig {
            enable_bloom_filter: false,
            compression: CompressionType::None,
            block_size: 256,
            ..Default::default()
        };

        let mut w = SSTableWriter::create(&path, config).unwrap();
        w.append(SSTableEntry {
            key: b"a".to_vec(),
            value: Some(b"1".to_vec()),
            timestamp: 10,
            sequence: 1,
        })
        .unwrap();
        w.append(SSTableEntry {
            key: b"b".to_vec(),
            value: Some(b"2".to_vec()),
            timestamp: 10,
            sequence: 1,
        })
        .unwrap();
        w.append(SSTableEntry {
            key: b"c".to_vec(),
            value: Some(b"3".to_vec()),
            timestamp: 10,
            sequence: 1,
        })
        .unwrap();
        w.append(SSTableEntry {
            key: b"d".to_vec(),
            value: Some(b"4".to_vec()),
            timestamp: 10,
            sequence: 1,
        })
        .unwrap();
        w.finish().unwrap();

        let mut r = SSTableReader::open(&path).unwrap();
        let got = r.scan_range(b"b", b"d", 100).unwrap();
        let keys: Vec<_> = got.into_iter().map(|e| e.key).collect();
        assert_eq!(keys, vec![b"b".to_vec(), b"c".to_vec()]);
    }

    #[test]
    fn next_prefix_handles_all_ff() {
        assert_eq!(next_prefix(b""), None);
        assert_eq!(next_prefix(&[0x00]), Some(vec![0x01]));
        assert_eq!(next_prefix(&[0x01, 0x02]), Some(vec![0x01, 0x03]));
        assert_eq!(next_prefix(&[0x01, 0xFF]), Some(vec![0x02]));
        assert_eq!(next_prefix(&[0xFF]), None);
        assert_eq!(next_prefix(&[0xFF, 0xFF]), None);
    }
}