diff --git a/Cargo.toml b/Cargo.toml index 00f91585..a7028792 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -2,7 +2,7 @@ name = "lsm-tree" description = "A K.I.S.S. implementation of log-structured merge trees (LSM-trees/LSMTs)" license = "MIT OR Apache-2.0" -version = "2.4.0" +version = "2.5.0" edition = "2021" rust-version = "1.74.0" readme = "README.md" @@ -38,7 +38,7 @@ quick_cache = { version = "0.6.5", default-features = false, features = [] } rustc-hash = "2.0.0" self_cell = "1.0.4" tempfile = "3.12.0" -value-log = "1.3.0" +value-log = "1.4.1" varint-rs = "2.2.0" xxhash-rust = { version = "0.8.12", features = ["xxh3"] } @@ -57,6 +57,12 @@ harness = false path = "benches/tli.rs" required-features = [] +[[bench]] +name = "merge" +harness = false +path = "benches/merge.rs" +required-features = [] + [[bench]] name = "memtable" harness = false diff --git a/benches/merge.rs b/benches/merge.rs new file mode 100644 index 00000000..8072e893 --- /dev/null +++ b/benches/merge.rs @@ -0,0 +1,77 @@ +use criterion::{criterion_group, criterion_main, Criterion}; +use lsm_tree::merge::{BoxedIterator, Merger}; +use lsm_tree::{mvcc_stream::MvccStream, InternalValue, Memtable}; +use nanoid::nanoid; + +fn merger(c: &mut Criterion) { + for num in [2, 4, 8, 16, 30] { + c.bench_function(&format!("Merge {num}"), |b| { + let memtables = (0..num) + .map(|_| { + let table = Memtable::default(); + + for _ in 0..100 { + table.insert(InternalValue::from_components( + nanoid!(), + vec![], + 0, + lsm_tree::ValueType::Value, + )); + } + + table + }) + .collect::>(); + + b.iter_with_large_drop(|| { + let iters = memtables + .iter() + .map(|x| x.iter().map(Ok)) + .map(|x| Box::new(x) as BoxedIterator<'_>) + .collect(); + + let merger = Merger::new(iters); + + assert_eq!(num * 100, merger.count()); + }) + }); + } +} + +fn mvcc_stream(c: &mut Criterion) { + for num in [2, 4, 8, 16, 30] { + c.bench_function(&format!("MVCC stream {num} versions"), |b| { + let memtables = (0..num) + .map(|_| { + let table = Memtable::default(); + + for key in 'a'..='z' { + table.insert(InternalValue::from_components( + key.to_string(), + vec![], + num, + lsm_tree::ValueType::Value, + )); + } + + table + }) + .collect::>(); + + b.iter_with_large_drop(|| { + let iters = memtables + .iter() + .map(|x| x.iter().map(Ok)) + .map(|x| Box::new(x) as BoxedIterator<'_>) + .collect(); + + let merger = MvccStream::new(Merger::new(iters)); + + assert_eq!(26, merger.count()); + }) + }); + } +} + +criterion_group!(benches, merger, mvcc_stream); +criterion_main!(benches); diff --git a/benches/tli.rs b/benches/tli.rs index d81c2715..8afc445d 100644 --- a/benches/tli.rs +++ b/benches/tli.rs @@ -1,6 +1,6 @@ use criterion::{criterion_group, criterion_main, Criterion}; use lsm_tree::segment::{ - block_index::BlockIndex, value_block::BlockOffset, value_block::CachePolicy, + block_index::KeyedBlockIndex, value_block::BlockOffset, value_block::CachePolicy, }; fn tli_find_item(c: &mut Criterion) { diff --git a/src/abstract.rs b/src/abstract.rs index 0e9ddbf3..63de101b 100644 --- a/src/abstract.rs +++ b/src/abstract.rs @@ -559,16 +559,11 @@ pub trait AbstractTree { /// # Errors /// /// Will return `Err` if an IO error occurs. - fn insert, V: AsRef<[u8]>>(&self, key: K, value: V, seqno: SeqNo) -> (u32, u32); - - /// Inserts a key-value pair. - fn raw_insert_with_lock, V: AsRef<[u8]>>( + fn insert, V: Into>( &self, - lock: &RwLockWriteGuard<'_, Memtable>, key: K, value: V, seqno: SeqNo, - r#type: ValueType, ) -> (u32, u32); /// Removes an item from the tree. @@ -598,7 +593,7 @@ pub trait AbstractTree { /// # Errors /// /// Will return `Err` if an IO error occurs. - fn remove>(&self, key: K, seqno: SeqNo) -> (u32, u32); + fn remove>(&self, key: K, seqno: SeqNo) -> (u32, u32); /// Removes an item from the tree. /// @@ -632,5 +627,5 @@ pub trait AbstractTree { /// # Errors /// /// Will return `Err` if an IO error occurs. - fn remove_weak>(&self, key: K, seqno: SeqNo) -> (u32, u32); + fn remove_weak>(&self, key: K, seqno: SeqNo) -> (u32, u32); } diff --git a/src/blob_tree/gc/reader.rs b/src/blob_tree/gc/reader.rs index 2ab94957..6cef7e2a 100644 --- a/src/blob_tree/gc/reader.rs +++ b/src/blob_tree/gc/reader.rs @@ -20,7 +20,7 @@ impl<'a> GcReader<'a> { fn get_internal(&self, key: &[u8]) -> crate::Result> { let Some(item) = self .tree - .get_internal_entry_with_lock(self.memtable, key, true, None)? + .get_internal_entry_with_lock(self.memtable, key, None)? .map(|x| x.value) else { return Ok(None); diff --git a/src/blob_tree/mod.rs b/src/blob_tree/mod.rs index 153fc6ba..b38b1ccb 100644 --- a/src/blob_tree/mod.rs +++ b/src/blob_tree/mod.rs @@ -596,34 +596,18 @@ impl AbstractTree for BlobTree { ) } - fn raw_insert_with_lock, V: AsRef<[u8]>>( + fn insert, V: Into>( &self, - lock: &RwLockWriteGuard<'_, Memtable>, key: K, value: V, seqno: SeqNo, - r#type: ValueType, ) -> (u32, u32) { use value::MaybeInlineValue; // NOTE: Initially, we always write an inline value // On memtable flush, depending on the values' sizes, they will be separated // into inline or indirect values - let item = MaybeInlineValue::Inline(value.as_ref().into()); - - let value = item.encode_into_vec(); - - let value = InternalValue::from_components(key.as_ref(), value, seqno, r#type); - lock.insert(value) - } - - fn insert, V: AsRef<[u8]>>(&self, key: K, value: V, seqno: SeqNo) -> (u32, u32) { - use value::MaybeInlineValue; - - // NOTE: Initially, we always write an inline value - // On memtable flush, depending on the values' sizes, they will be separated - // into inline or indirect values - let item = MaybeInlineValue::Inline(value.as_ref().into()); + let item = MaybeInlineValue::Inline(value.into()); let value = item.encode_into_vec(); @@ -680,11 +664,11 @@ impl AbstractTree for BlobTree { } } - fn remove>(&self, key: K, seqno: SeqNo) -> (u32, u32) { + fn remove>(&self, key: K, seqno: SeqNo) -> (u32, u32) { self.index.remove(key, seqno) } - fn remove_weak>(&self, key: K, seqno: SeqNo) -> (u32, u32) { + fn remove_weak>(&self, key: K, seqno: SeqNo) -> (u32, u32) { self.index.remove_weak(key, seqno) } } diff --git a/src/bloom/bit_array.rs b/src/bloom/bit_array.rs index 779943dc..e4f0535c 100644 --- a/src/bloom/bit_array.rs +++ b/src/bloom/bit_array.rs @@ -39,18 +39,6 @@ impl BitArray { Self(bytes) } - /// Size in bytes - #[must_use] - pub fn len(&self) -> usize { - self.0.len() - } - - #[allow(unused)] - #[must_use] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - #[must_use] pub fn bytes(&self) -> &[u8] { &self.0 diff --git a/src/bloom/mod.rs b/src/bloom/mod.rs index 8ce117ac..cc45b203 100644 --- a/src/bloom/mod.rs +++ b/src/bloom/mod.rs @@ -88,7 +88,7 @@ impl BloomFilter { /// Size of bloom filter in bytes. #[must_use] pub fn len(&self) -> usize { - self.inner.len() + self.inner.bytes().len() } fn from_raw(m: usize, k: usize, bytes: Box<[u8]>) -> Self { diff --git a/src/compaction/fifo.rs b/src/compaction/fifo.rs index 7eea2681..2683f010 100644 --- a/src/compaction/fifo.rs +++ b/src/compaction/fifo.rs @@ -40,6 +40,10 @@ impl Strategy { } impl CompactionStrategy for Strategy { + fn get_name(&self) -> &'static str { + "FifoStrategy" + } + fn choose(&self, levels: &LevelManifest, config: &Config) -> Choice { let resolved_view = levels.resolved_view(); @@ -58,11 +62,8 @@ impl CompactionStrategy for Strategy { let lifetime_sec = lifetime_us / 1000 / 1000; if lifetime_sec > ttl_seconds.into() { - log::warn!( - "segment is older than configured TTL: {:?}", - segment.metadata.id, - ); - segment_ids_to_delete.insert(segment.metadata.id); + log::warn!("segment is older than configured TTL: {:?}", segment.id(),); + segment_ids_to_delete.insert(segment.id()); } } } @@ -86,11 +87,11 @@ impl CompactionStrategy for Strategy { bytes_to_delete = bytes_to_delete.saturating_sub(segment.metadata.file_size); - segment_ids_to_delete.insert(segment.metadata.id); + segment_ids_to_delete.insert(segment.id()); log::debug!( "dropping segment to reach configured size limit: {:?}", - segment.metadata.id, + segment.id(), ); } } @@ -124,7 +125,7 @@ mod tests { key_range::KeyRange, level_manifest::LevelManifest, segment::{ - block_index::two_level_index::TwoLevelBlockIndex, + block_index::{two_level_index::TwoLevelBlockIndex, BlockIndexImpl}, file_offsets::FileOffsets, meta::{Metadata, SegmentId}, value_block::BlockOffset, @@ -136,18 +137,18 @@ mod tests { use std::sync::Arc; use test_log::test; - #[cfg(feature = "bloom")] - use crate::bloom::BloomFilter; - #[allow(clippy::expect_used)] #[allow(clippy::cast_possible_truncation)] fn fixture_segment(id: SegmentId, created_at: u128) -> Segment { let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); + let block_index = TwoLevelBlockIndex::new((0, id).into(), block_cache.clone()); + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); + SegmentInner { tree_id: 0, descriptor_table: Arc::new(FileDescriptorTable::new(512, 1)), - block_index: Arc::new(TwoLevelBlockIndex::new((0, id).into(), block_cache.clone())), + block_index, offsets: FileOffsets { bloom_ptr: BlockOffset(0), @@ -180,7 +181,7 @@ mod tests { block_cache, #[cfg(feature = "bloom")] - bloom_filter: Some(BloomFilter::with_fp_rate(1, 0.1)), + bloom_filter: Some(crate::bloom::BloomFilter::with_fp_rate(1, 0.1)), } .into() } diff --git a/src/compaction/leveled.rs b/src/compaction/leveled.rs index 09d79ecf..0a000899 100644 --- a/src/compaction/leveled.rs +++ b/src/compaction/leveled.rs @@ -6,18 +6,152 @@ use super::{Choice, CompactionStrategy, Input as CompactionInput}; use crate::{ config::Config, key_range::KeyRange, - level_manifest::{level::Level, LevelManifest}, + level_manifest::{hidden_set::HiddenSet, level::Level, LevelManifest}, segment::Segment, HashSet, SegmentId, }; +/// Aggregates the key range of a list of segments. +fn aggregate_key_range(segments: &[Segment]) -> KeyRange { + KeyRange::aggregate(segments.iter().map(|x| &x.metadata.key_range)) +} + +/// Tries to find the most optimal compaction set from +/// one level into the other. +fn pick_minimal_compaction( + curr_level: &Level, + next_level: &Level, + hidden_set: &HiddenSet, +) -> Option<(HashSet, bool)> { + // assert!(curr_level.is_disjoint, "Lx is not disjoint"); + // assert!(next_level.is_disjoint, "Lx+1 is not disjoint"); + + struct Choice { + write_amp: f32, + segment_ids: HashSet, + can_trivial_move: bool, + } + + let mut choices = vec![]; + + let mut add_choice = |choice: Choice| { + let mut valid_choice = true; + + // IMPORTANT: Compaction is blocked because of other + // on-going compaction + valid_choice &= !choice.segment_ids.iter().any(|x| hidden_set.is_hidden(*x)); + + // NOTE: Keep compactions with 25 or less segments + // to make compactions not too large + valid_choice &= choice.can_trivial_move || choice.segment_ids.len() <= 25; + + if valid_choice { + choices.push(choice); + } + }; + + for size in 1..=next_level.len() { + let windows = next_level.windows(size); + + for window in windows { + if hidden_set.is_blocked(window.iter().map(Segment::id)) { + // IMPORTANT: Compaction is blocked because of other + // on-going compaction + continue; + } + + let key_range = aggregate_key_range(window); + + // Pull in all segments in current level into compaction + let curr_level_pull_in: Vec<_> = if curr_level.is_disjoint { + // IMPORTANT: Avoid "infectious spread" of key ranges + // Imagine these levels: + // + // A B C D E F + // L1 | ----- ----- ----- ----- ----- ----- + // L2 | ----- ----- ----- ----- ----- + // 1 2 3 4 5 + // + // If we took 1, we would also have to include A, + // but then we would also have to include 2, + // but then we would also have to include B, + // but then we would also have to include 3, + // ... + // + // Instead, we consider a window like 1 - 3 + // and then take A & B, because they are *contained* in that range + // Not including C is fine, because we are not shadowing data unexpectedly + curr_level.contained_segments(&key_range).collect() + } else { + // If the level is not disjoint, we just merge everything that overlaps + // to try and "repair" the level + curr_level.overlapping_segments(&key_range).collect() + }; + + if hidden_set.is_blocked(curr_level_pull_in.iter().map(|x| x.id())) { + // IMPORTANT: Compaction is blocked because of other + // on-going compaction + continue; + } + + let curr_level_size = curr_level_pull_in + .iter() + .map(|x| x.metadata.file_size) + .sum::(); + + // NOTE: Only consider compactions where we actually reach the amount + // of bytes we need to merge + if curr_level_size >= 1 { + let next_level_size = window.iter().map(|x| x.metadata.file_size).sum::(); + + let mut segment_ids: HashSet<_> = window.iter().map(Segment::id).collect(); + segment_ids.extend(curr_level_pull_in.iter().map(|x| x.id())); + + let write_amp = (next_level_size as f32) / (curr_level_size as f32); + + add_choice(Choice { + write_amp, + segment_ids, + can_trivial_move: false, + }); + } + } + } + + // NOTE: Find largest trivial move (if it exists) + 'trivial_move_search: for size in (1..=curr_level.len()).rev() { + let windows = curr_level.windows(size); + + for window in windows { + let key_range = aggregate_key_range(window); + + if next_level.overlapping_segments(&key_range).next().is_none() { + add_choice(Choice { + write_amp: 0.0, + segment_ids: window.iter().map(Segment::id).collect(), + can_trivial_move: true, + }); + break 'trivial_move_search; + } + } + } + + let minimum_effort_choice = choices.into_iter().min_by(|a, b| { + a.write_amp + .partial_cmp(&b.write_amp) + .unwrap_or(std::cmp::Ordering::Equal) + }); + + minimum_effort_choice.map(|c| (c.segment_ids, c.can_trivial_move)) +} + /// Levelled compaction strategy (LCS) /// -/// If a level reaches some threshold size, parts of it are merged into overlapping segments in the next level. +/// When a level reaches some threshold size, parts of it are merged into overlapping segments in the next level. /// -/// Each level Ln for n >= 1 can have up to ratio^n segments. +/// Each level Ln for n >= 2 can have up to `level_base_size * ratio^n` segments. /// -/// LCS suffers from comparatively high write amplification, but has decent read & space amplification. +/// LCS suffers from comparatively high write amplification, but has decent read amplification and great space amplification (~1.1x). /// /// LCS is the recommended compaction strategy to use. /// @@ -25,18 +159,18 @@ use crate::{ #[derive(Clone)] pub struct Strategy { /// When the number of segments in L0 reaches this threshold, - /// they are merged into L1 + /// they are merged into L1. /// /// Default = 4 /// - /// Same as `level0_file_num_compaction_trigger` in `RocksDB` + /// Same as `level0_file_num_compaction_trigger` in `RocksDB`. pub l0_threshold: u8, - /// Target segment size (compressed) + /// The target segment size as disk (possibly compressed). /// /// Default = 64 MiB /// - /// Same as `target_file_size_base` in `RocksDB` + /// Same as `target_file_size_base` in `RocksDB`. pub target_size: u32, /// Size ratio between levels of the LSM tree (a.k.a fanout, growth rate) @@ -44,136 +178,58 @@ pub struct Strategy { /// This is the exponential growth of the from one. /// level to the next /// - /// A level target size is: max_memtable_size * level_ratio.pow(#level + 1) + /// A level target size is: max_memtable_size * level_ratio.pow(#level + 1). #[allow(clippy::doc_markdown)] pub level_ratio: u8, + + /// The target size of L1. + /// + /// Currently hard coded to 256 MiB. + /// + /// Default = 256 MiB + pub level_base_size: u32, } impl Default for Strategy { fn default() -> Self { Self { l0_threshold: 4, - target_size: 64 * 1_024 * 1_024, - level_ratio: 8, // TODO: benchmark vs 10 + target_size:/* 64 Mib */ 64 * 1_024 * 1_024, + level_ratio: 10, + level_base_size:/* 256 MiB */ 256 * 1_024 * 1_024, } } } -fn aggregate_key_range(segments: &[Segment]) -> KeyRange { - KeyRange::aggregate(segments.iter().map(|x| &x.metadata.key_range)) -} - -fn desired_level_size_in_bytes(level_idx: u8, ratio: u8, target_size: u32) -> usize { - (ratio as usize).pow(u32::from(level_idx)) * (target_size as usize) -} - -fn pick_minimal_overlap( - curr_level: &Level, - next_level: &Level, - overshoot: u64, -) -> (HashSet, bool) { - let mut choices = vec![]; - - for size in 1..=curr_level.len() { - let windows = curr_level.windows(size); - - for window in windows { - let size_sum = window.iter().map(|x| x.metadata.file_size).sum::(); - - if size_sum >= overshoot { - // NOTE: Consider this window - - let mut segment_ids: HashSet = - window.iter().map(|x| x.metadata.id).collect(); - - // Get overlapping segments in next level - let key_range = aggregate_key_range(window); - - let next_level_overlapping_segments: Vec<_> = next_level - .overlapping_segments(&key_range) - .cloned() - .collect(); - - // Get overlapping segments in same level - let key_range = aggregate_key_range(&next_level_overlapping_segments); - - let curr_level_overlapping_segment_ids: Vec<_> = curr_level - .overlapping_segments(&key_range) - .filter(|x| !segment_ids.contains(&x.metadata.id)) - .collect(); - - // Calculate effort - let size_next_level = next_level_overlapping_segments - .iter() - .map(|x| x.metadata.file_size) - .sum::(); - - let size_curr_level = curr_level_overlapping_segment_ids - .iter() - .map(|x| x.metadata.file_size) - .sum::(); - - let effort = size_sum + size_next_level + size_curr_level; - - segment_ids.extend( - next_level_overlapping_segments - .iter() - .map(|x| x.metadata.id), - ); +impl Strategy { + /// Calculates the level target size. + /// + /// L1 = `level_base_size` + /// + /// L2 = `level_base_size * ratio` + /// + /// L3 = `level_base_size * ratio * ratio` + /// ... + fn level_target_size(&self, level_idx: u8) -> u64 { + assert!(level_idx >= 1, "level_target_size does not apply to L0"); - segment_ids.extend( - curr_level_overlapping_segment_ids - .iter() - .map(|x| x.metadata.id), - ); + let power = (self.level_ratio as usize).pow(u32::from(level_idx) - 1); - // TODO: need to calculate write_amp and choose minimum write_amp instead - // - // consider the segments in La = A to be the ones in the window - // and the segments in La+1 B to be the ones that overlap - // and r = A / B - // we want to avoid compactions that have a low ratio r - // because that means we don't clear out a lot of segments in La - // but have to rewrite a lot of segments in La+1 - // - // ultimately, we want the highest ratio - // to maximize the amount of segments we are getting rid of in La - // for the least amount of effort - choices.push(( - effort, - segment_ids, - next_level_overlapping_segments.is_empty(), - )); - } - } + (power * (self.level_base_size as usize)) as u64 } - - let minimum_effort_choice = choices.into_iter().min_by(|a, b| a.0.cmp(&b.0)); - let (_, set, can_trivial_move) = minimum_effort_choice.expect("should exist"); - - (set, can_trivial_move) } impl CompactionStrategy for Strategy { + fn get_name(&self) -> &'static str { + "LeveledStrategy" + } + #[allow(clippy::too_many_lines)] fn choose(&self, levels: &LevelManifest, _: &Config) -> Choice { - let resolved_view = levels.resolved_view(); - - // If there are any levels that already have a compactor working on it - // we can't touch those, because that could cause a race condition - // violating the leveled compaction invariance of having a single sorted - // run per level - // - // TODO: However, this can probably improved by checking two compaction - // workers just don't cross key ranges - let busy_levels = levels.busy_levels(); - - for (curr_level_index, level) in resolved_view - .iter() - .enumerate() - .skip(1) - .take(resolved_view.len() - 2) - .rev() + let view = &levels.levels; + + // L1+ compactions + for (curr_level_index, level) in view.iter().enumerate().skip(1).take(view.len() - 2).rev() { // NOTE: Level count is 255 max #[allow(clippy::cast_possible_truncation)] @@ -185,22 +241,35 @@ impl CompactionStrategy for Strategy { continue; } - if busy_levels.contains(&curr_level_index) || busy_levels.contains(&next_level_index) { - continue; - } + let level_size: u64 = level + .segments + .iter() + // NOTE: Take bytes that are already being compacted into account, + // otherwise we may be overcompensating + .filter(|x| !levels.hidden_set().is_hidden(x.id())) + .map(|x| x.metadata.file_size) + .sum(); - let desired_bytes = - desired_level_size_in_bytes(curr_level_index, self.level_ratio, self.target_size); + let desired_bytes = self.level_target_size(curr_level_index); - let overshoot = level.size().saturating_sub(desired_bytes as u64); + let overshoot = level_size.saturating_sub(desired_bytes); if overshoot > 0 { - let Some(next_level) = &resolved_view.get(next_level_index as usize) else { + let Some(next_level) = &view.get(next_level_index as usize) else { break; }; - let (segment_ids, can_trivial_move) = - pick_minimal_overlap(level, next_level, overshoot); + let Some((segment_ids, can_trivial_move)) = + pick_minimal_compaction(level, next_level, levels.hidden_set()) + else { + break; + }; + + // eprintln!( + // "merge {} segments, L{}->L{next_level_index}: {segment_ids:?}", + // segment_ids.len(), + // next_level_index - 1, + // ); let choice = CompactionInput { segment_ids, @@ -229,8 +298,11 @@ impl CompactionStrategy for Strategy { } } + // L0->L1 compactions { - let Some(first_level) = resolved_view.first() else { + let busy_levels = levels.busy_levels(); + + let Some(first_level) = view.first() else { return Choice::DoNothing; }; @@ -275,22 +347,21 @@ impl CompactionStrategy for Strategy { } if !busy_levels.contains(&1) { - let mut level = first_level.clone(); + let mut level = (**first_level).clone(); level.sort_by_key_range(); - let Some(next_level) = &resolved_view.get(1) else { + let Some(next_level) = &view.get(1) else { return Choice::DoNothing; }; - let mut segment_ids: HashSet = - level.iter().map(|x| x.metadata.id).collect(); + let mut segment_ids: HashSet = level.iter().map(Segment::id).collect(); // Get overlapping segments in next level let key_range = aggregate_key_range(&level); let next_level_overlapping_segment_ids: Vec<_> = next_level .overlapping_segments(&key_range) - .map(|x| x.metadata.id) + .map(Segment::id) .collect(); segment_ids.extend(&next_level_overlapping_segment_ids); @@ -323,7 +394,7 @@ mod tests { key_range::KeyRange, level_manifest::LevelManifest, segment::{ - block_index::two_level_index::TwoLevelBlockIndex, + block_index::{two_level_index::TwoLevelBlockIndex, BlockIndexImpl}, file_offsets::FileOffsets, meta::{Metadata, SegmentId}, value_block::BlockOffset, @@ -335,9 +406,6 @@ mod tests { use std::{path::Path, sync::Arc}; use test_log::test; - #[cfg(feature = "bloom")] - use crate::bloom::BloomFilter; - fn string_key_range(a: &str, b: &str) -> KeyRange { KeyRange::new((a.as_bytes().into(), b.as_bytes().into())) } @@ -355,10 +423,13 @@ mod tests { ) -> Segment { let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); + let block_index = TwoLevelBlockIndex::new((0, id).into(), block_cache.clone()); + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); + SegmentInner { tree_id: 0, descriptor_table: Arc::new(FileDescriptorTable::new(512, 1)), - block_index: Arc::new(TwoLevelBlockIndex::new((0, id).into(), block_cache.clone())), + block_index, offsets: FileOffsets { bloom_ptr: BlockOffset(0), @@ -391,7 +462,7 @@ mod tests { block_cache, #[cfg(feature = "bloom")] - bloom_filter: Some(BloomFilter::with_fp_rate(1, 0.1)), + bloom_filter: Some(crate::bloom::BloomFilter::with_fp_rate(1, 0.1)), } .into() } @@ -581,166 +652,11 @@ mod tests { Ok(()) } - #[test] - fn leveled_deeper_level_with_overlap() -> crate::Result<()> { - let tempdir = tempfile::tempdir()?; - let compactor = Strategy { - target_size: 64 * 1_024 * 1_024, - level_ratio: 2, - ..Default::default() - }; - let config = Config::default(); - - #[rustfmt::skip] - let levels = build_levels(tempdir.path(), vec![ - vec![], - vec![(1, "a", "g", 64), (2, "h", "t", 64), (3, "x", "z", 64)], - vec![(4, "f", "l", 64)], - vec![], - ])?; - - assert_eq!( - compactor.choose(&levels, &config), - Choice::Merge(CompactionInput { - dest_level: 2, - segment_ids: set![3], - target_size: 64 * 1_024 * 1_024 - }) - ); - - Ok(()) - } - - #[test] - fn leveled_deeper_level_no_overlap() -> crate::Result<()> { - let tempdir = tempfile::tempdir()?; - let compactor = Strategy { - target_size: 64 * 1_024 * 1_024, - level_ratio: 2, - ..Default::default() - }; - let config = Config::default(); - - #[rustfmt::skip] - let levels = build_levels(tempdir.path(), vec![ - vec![], - vec![(1, "a", "g", 64), (2, "h", "j", 64), (3, "k", "t", 64)], - vec![(4, "k", "l", 64)], - vec![], - ])?; - - assert_eq!( - compactor.choose(&levels, &config), - // NOTE: We merge because segments are demoted into "cold" levels - // see https://github.com/fjall-rs/lsm-tree/issues/63 - Choice::Merge(CompactionInput { - dest_level: 2, - segment_ids: set![1], - target_size: 64 * 1_024 * 1_024 - }) - ); - - Ok(()) - } - - #[test] - fn leveled_last_level_with_overlap() -> crate::Result<()> { - let tempdir = tempfile::tempdir()?; - let compactor = Strategy { - target_size: 64 * 1_024 * 1_024, - level_ratio: 2, - ..Default::default() - }; - let config = Config::default(); - - #[rustfmt::skip] - let levels = build_levels(tempdir.path(), vec![ - vec![], - vec![], - vec![(1, "a", "g", 64), (2, "a", "g", 64), (3, "a", "g", 64), (4, "a", "g", 64), (5, "y", "z", 64)], - vec![(6, "f", "l", 64)], - ])?; - - assert_eq!( - compactor.choose(&levels, &config), - Choice::Merge(CompactionInput { - dest_level: 3, - // NOTE: 5 is the only segment that has no overlap with #3 - segment_ids: set![5], - target_size: 64 * 1_024 * 1_024 - }) - ); - - Ok(()) - } - - #[test] - fn levelled_last_level_with_overlap_invariant() -> crate::Result<()> { - let tempdir = tempfile::tempdir()?; - let compactor = Strategy { - target_size: 64 * 1_024 * 1_024, - level_ratio: 2, - ..Default::default() - }; - let config = Config::default(); - - #[rustfmt::skip] - let levels = build_levels(tempdir.path(), vec![ - vec![], - vec![], - vec![(1, "a", "g", 64), (2, "h", "j", 64), (3, "k", "l", 64), (4, "m", "n", 64), (5, "y", "z", 64)], - vec![(6, "f", "l", 64)], - ])?; - - assert_eq!( - compactor.choose(&levels, &config), - Choice::Move(CompactionInput { - dest_level: 3, - // NOTE: segment #4 is the left-most segment that has no overlap with L3 - segment_ids: set![4], - target_size: 64 * 1_024 * 1_024 - }) - ); - - Ok(()) - } - - #[test] - fn levelled_last_level_without_overlap_invariant() -> crate::Result<()> { - let tempdir = tempfile::tempdir()?; - let compactor = Strategy { - target_size: 64 * 1_024 * 1_024, - level_ratio: 2, - ..Default::default() - }; - let config = Config::default(); - - #[rustfmt::skip] - let levels = build_levels(tempdir.path(), vec![ - vec![], - vec![], - vec![(1, "a", "g", 64), (2, "h", "j", 64), (3, "k", "l", 64), (4, "m", "n", 64), (5, "y", "z", 64)], - vec![(6, "w", "x", 64)], - ])?; - - assert_eq!( - compactor.choose(&levels, &config), - Choice::Move(CompactionInput { - dest_level: 3, - segment_ids: set![1], - target_size: 64 * 1_024 * 1_024 - }) - ); - - Ok(()) - } - #[test] fn levelled_from_tiered() -> crate::Result<()> { let tempdir = tempfile::tempdir()?; let compactor = Strategy { target_size: 64 * 1_024 * 1_024, - level_ratio: 2, ..Default::default() }; let config = Config::default(); @@ -748,7 +664,7 @@ mod tests { #[rustfmt::skip] let levels = build_levels(tempdir.path(), vec![ vec![], - vec![(1, "a", "z", 64), (2, "a", "z", 64), (3, "g", "z", 64)], + vec![(1, "a", "z", 64), (2, "a", "z", 64), (3, "g", "z", 64), (5, "g", "z", 64), (6, "g", "z", 64)], vec![(4, "a", "g", 64)], vec![], ])?; @@ -757,7 +673,7 @@ mod tests { compactor.choose(&levels, &config), Choice::Merge(CompactionInput { dest_level: 2, - segment_ids: [1, 2, 3, 4].into_iter().collect::>(), + segment_ids: [1, 2, 3, 4, 5, 6].into_iter().collect::>(), target_size: 64 * 1_024 * 1_024 }) ); diff --git a/src/compaction/maintenance.rs b/src/compaction/maintenance.rs index 11a86f2e..8f8deb49 100644 --- a/src/compaction/maintenance.rs +++ b/src/compaction/maintenance.rs @@ -39,10 +39,14 @@ pub fn choose_least_effort_compaction(segments: &[Segment], n: usize) -> HashSet .min_by_key(|window| window.iter().map(|s| s.metadata.file_size).sum::()) .expect("should have at least one window"); - window.iter().map(|x| x.metadata.id).collect() + window.iter().map(Segment::id).collect() } impl CompactionStrategy for Strategy { + fn get_name(&self) -> &'static str { + "MaintenanceStrategy" + } + fn choose(&self, levels: &LevelManifest, _: &Config) -> Choice { let resolved_view = levels.resolved_view(); @@ -86,24 +90,27 @@ mod tests { key_range::KeyRange, level_manifest::LevelManifest, segment::{ - block_index::two_level_index::TwoLevelBlockIndex, file_offsets::FileOffsets, - meta::Metadata, value_block::BlockOffset, Segment, SegmentInner, + block_index::{two_level_index::TwoLevelBlockIndex, BlockIndexImpl}, + file_offsets::FileOffsets, + meta::Metadata, + value_block::BlockOffset, + Segment, SegmentInner, }, }; use std::sync::Arc; use test_log::test; - #[cfg(feature = "bloom")] - use crate::bloom::BloomFilter; - #[allow(clippy::expect_used)] fn fixture_segment(id: SegmentId, created_at: u128) -> Segment { let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); + let block_index = TwoLevelBlockIndex::new((0, id).into(), block_cache.clone()); + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); + SegmentInner { tree_id: 0, descriptor_table: Arc::new(FileDescriptorTable::new(512, 1)), - block_index: Arc::new(TwoLevelBlockIndex::new((0, id).into(), block_cache.clone())), + block_index, offsets: FileOffsets { bloom_ptr: BlockOffset(0), @@ -136,7 +143,7 @@ mod tests { block_cache, #[cfg(feature = "bloom")] - bloom_filter: Some(BloomFilter::with_fp_rate(1, 0.1)), + bloom_filter: Some(crate::bloom::BloomFilter::with_fp_rate(1, 0.1)), } .into() } diff --git a/src/compaction/major.rs b/src/compaction/major.rs index 995d7ae2..002404ec 100644 --- a/src/compaction/major.rs +++ b/src/compaction/major.rs @@ -3,21 +3,21 @@ // (found in the LICENSE-* files in the repository) use super::{Choice, CompactionStrategy, Input as CompactionInput}; -use crate::{config::Config, level_manifest::LevelManifest}; +use crate::{config::Config, level_manifest::LevelManifest, Segment}; /// Major compaction /// -/// Compacts all segments into the last level +/// Compacts all segments into the last level. pub struct Strategy { target_size: u64, } impl Strategy { - /// Configures a new `SizeTiered` compaction strategy + /// Configures a new `SizeTiered` compaction strategy. /// /// # Panics /// - /// Panics, if `target_size` is below 1024 bytes + /// Panics, if `target_size` is below 1024 bytes. #[must_use] #[allow(dead_code)] pub fn new(target_size: u64) -> Self { @@ -35,8 +35,12 @@ impl Default for Strategy { } impl CompactionStrategy for Strategy { + fn get_name(&self) -> &'static str { + "MajorCompaction" + } + fn choose(&self, levels: &LevelManifest, _: &Config) -> Choice { - let segment_ids = levels.iter().map(|x| x.metadata.id).collect(); + let segment_ids = levels.iter().map(Segment::id).collect(); Choice::Merge(CompactionInput { segment_ids, diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index 9222fdd9..a25a4291 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -69,6 +69,10 @@ pub enum Choice { /// and emits a choice on what to do. #[allow(clippy::module_name_repetitions)] pub trait CompactionStrategy { + // TODO: could be : Display instead + /// Gets the compaction strategy name. + fn get_name(&self) -> &'static str; + /// Decides on what to do based on the current state of the LSM-tree's levels fn choose(&self, _: &LevelManifest, config: &Config) -> Choice; } diff --git a/src/compaction/pulldown.rs b/src/compaction/pulldown.rs index 826a16fe..9698e275 100644 --- a/src/compaction/pulldown.rs +++ b/src/compaction/pulldown.rs @@ -3,7 +3,7 @@ // (found in the LICENSE-* files in the repository) use super::{Choice, CompactionStrategy, Input}; -use crate::{level_manifest::LevelManifest, Config, HashSet}; +use crate::{level_manifest::LevelManifest, Config, HashSet, Segment}; /// Pulls down and merges a level into the destination level. /// @@ -11,6 +11,10 @@ use crate::{level_manifest::LevelManifest, Config, HashSet}; pub struct Strategy(pub u8, pub u8); impl CompactionStrategy for Strategy { + fn get_name(&self) -> &'static str { + "PullDownCompaction" + } + #[allow(clippy::expect_used)] fn choose(&self, levels: &LevelManifest, _: &Config) -> Choice { let resolved_view = levels.resolved_view(); @@ -23,9 +27,9 @@ impl CompactionStrategy for Strategy { .get(usize::from(self.1)) .expect("next level should exist"); - let mut segment_ids: HashSet<_> = level.segments.iter().map(|x| x.metadata.id).collect(); + let mut segment_ids: HashSet<_> = level.segments.iter().map(Segment::id).collect(); - segment_ids.extend(next_level.segments.iter().map(|x| x.metadata.id)); + segment_ids.extend(next_level.segments.iter().map(Segment::id)); Choice::Merge(Input { segment_ids, diff --git a/src/compaction/stream.rs b/src/compaction/stream.rs index d7f86db6..4314009d 100644 --- a/src/compaction/stream.rs +++ b/src/compaction/stream.rs @@ -102,6 +102,7 @@ impl>> Iterator for CompactionSt mod tests { use super::*; use crate::value::{InternalValue, ValueType}; + use test_log::test; macro_rules! stream { ($($key:expr, $sub_key:expr, $value_type:expr),* $(,)?) => {{ @@ -136,7 +137,7 @@ mod tests { }; } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_queue_weak_tombstones() { #[rustfmt::skip] @@ -156,7 +157,7 @@ mod tests { } /// GC should not evict tombstones, unless they are covered up - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_tombstone_no_gc() -> crate::Result<()> { #[rustfmt::skip] @@ -186,7 +187,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_old_tombstone() -> crate::Result<()> { #[rustfmt::skip] @@ -231,7 +232,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_tombstone_overwrite_gc() -> crate::Result<()> { #[rustfmt::skip] @@ -252,7 +253,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_weak_tombstone_simple() -> crate::Result<()> { #[rustfmt::skip] @@ -277,7 +278,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_weak_tombstone_no_gc() -> crate::Result<()> { #[rustfmt::skip] @@ -302,7 +303,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_weak_tombstone_evict() { #[rustfmt::skip] @@ -319,7 +320,7 @@ mod tests { iter_closed!(iter); } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_weak_tombstone_evict_next_value() -> crate::Result<()> { #[rustfmt::skip] @@ -349,7 +350,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_no_evict_simple() -> crate::Result<()> { #[rustfmt::skip] @@ -379,7 +380,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn compaction_stream_no_evict_simple_multi_keys() -> crate::Result<()> { #[rustfmt::skip] diff --git a/src/compaction/tiered.rs b/src/compaction/tiered.rs index 367c2548..a7a70bc7 100644 --- a/src/compaction/tiered.rs +++ b/src/compaction/tiered.rs @@ -3,7 +3,7 @@ // (found in the LICENSE-* files in the repository) use super::{Choice, CompactionStrategy, Input as CompactionInput}; -use crate::{level_manifest::LevelManifest, Config}; +use crate::{level_manifest::LevelManifest, Config, Segment}; fn desired_level_size_in_bytes(level_idx: u8, ratio: u8, base_size: u32) -> usize { (ratio as usize).pow(u32::from(level_idx + 1)) * (base_size as usize) @@ -50,6 +50,10 @@ impl Default for Strategy { } impl CompactionStrategy for Strategy { + fn get_name(&self) -> &'static str { + "TieredStrategy" + } + fn choose(&self, levels: &LevelManifest, config: &Config) -> Choice { let resolved_view = levels.resolved_view(); @@ -69,13 +73,20 @@ impl CompactionStrategy for Strategy { continue; } - let curr_level_bytes = level.size(); + let level_size: u64 = level + .segments + .iter() + // NOTE: Take bytes that are already being compacted into account, + // otherwise we may be overcompensating + .filter(|x| !levels.hidden_set().is_hidden(x.id())) + .map(|x| x.metadata.file_size) + .sum(); let desired_bytes = desired_level_size_in_bytes(curr_level_index, self.level_ratio, self.base_size) as u64; - if curr_level_bytes >= desired_bytes { + if level_size >= desired_bytes { // NOTE: Take desired_bytes because we are in tiered mode // We want to take N segments, not just the overshoot (like in leveled) let mut overshoot = desired_bytes; @@ -91,11 +102,7 @@ impl CompactionStrategy for Strategy { segments_to_compact.push(segment); } - let segment_ids = segments_to_compact - .iter() - .map(|x| &x.metadata.id) - .copied() - .collect(); + let segment_ids = segments_to_compact.iter().map(Segment::id).collect(); return Choice::Merge(CompactionInput { segment_ids, @@ -131,7 +138,7 @@ mod tests { key_range::KeyRange, level_manifest::LevelManifest, segment::{ - block_index::two_level_index::TwoLevelBlockIndex, + block_index::{two_level_index::TwoLevelBlockIndex, BlockIndexImpl}, file_offsets::FileOffsets, meta::{Metadata, SegmentId}, value_block::BlockOffset, @@ -149,10 +156,13 @@ mod tests { fn fixture_segment(id: SegmentId, size_mib: u64, max_seqno: SeqNo) -> Segment { let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); + let block_index = TwoLevelBlockIndex::new((0, id).into(), block_cache.clone()); + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); + SegmentInner { tree_id: 0, descriptor_table: Arc::new(FileDescriptorTable::new(512, 1)), - block_index: Arc::new(TwoLevelBlockIndex::new((0, id).into(), block_cache.clone())), + block_index, offsets: FileOffsets { bloom_ptr: BlockOffset(0), diff --git a/src/compaction/worker.rs b/src/compaction/worker.rs index 5e3e634f..a45aedf7 100644 --- a/src/compaction/worker.rs +++ b/src/compaction/worker.rs @@ -7,10 +7,15 @@ use crate::{ compaction::{stream::CompactionStream, Choice}, file::SEGMENTS_FOLDER, level_manifest::LevelManifest, + level_reader::LevelReader, merge::{BoxedIterator, Merger}, segment::{ - block_index::two_level_index::TwoLevelBlockIndex, id::GlobalSegmentId, - level_reader::LevelReader, multi_writer::MultiWriter, Segment, SegmentInner, + block_index::{ + full_index::FullBlockIndex, two_level_index::TwoLevelBlockIndex, BlockIndexImpl, + }, + id::GlobalSegmentId, + multi_writer::MultiWriter, + Segment, SegmentInner, }, stop_signal::StopSignal, tree::inner::{SealedMemtables, TreeId}, @@ -36,15 +41,14 @@ pub struct Options { /// Sealed memtables (required for temporarily locking). pub sealed_memtables: Arc>, - /// Compaction strategy. - /// - /// The one inside `config` is NOT used. + /// Compaction strategy to use. pub strategy: Arc, - /// Stop signal + /// Stop signal to interrupt a compaction worker in case + /// the tree is dropped. pub stop_signal: StopSignal, - /// Evicts items that are older than this seqno + /// Evicts items that are older than this seqno (MVCC GC). pub eviction_seqno: u64, } @@ -68,44 +72,27 @@ impl Options { /// This will block until the compactor is fully finished. pub fn do_compaction(opts: &Options) -> crate::Result<()> { log::trace!("compactor: acquiring levels manifest lock"); - let mut original_levels = opts.levels.write().expect("lock is poisoned"); + let original_levels = opts.levels.write().expect("lock is poisoned"); - log::trace!("compactor: consulting compaction strategy"); + log::trace!( + "compactor: consulting compaction strategy {:?}", + opts.strategy.get_name(), + ); let choice = opts.strategy.choose(&original_levels, &opts.config); log::debug!("compactor: choice: {choice:?}"); match choice { Choice::Merge(payload) => merge_segments(original_levels, opts, &payload), - Choice::Move(payload) => { - let segment_map = original_levels.get_all_segments(); - - original_levels.atomic_swap(|recipe| { - for segment_id in payload.segment_ids { - if let Some(segment) = segment_map.get(&segment_id).cloned() { - for level in recipe.iter_mut() { - level.remove(segment_id); - } - - recipe - .get_mut(payload.dest_level as usize) - .expect("destination level should exist") - .insert(segment); - } - } - }) - } - Choice::Drop(payload) => { - drop_segments( - original_levels, - opts, - &payload - .into_iter() - .map(|x| (opts.tree_id, x).into()) - .collect::>(), - )?; - Ok(()) - } + Choice::Move(payload) => move_segments(original_levels, opts, payload), + Choice::Drop(payload) => drop_segments( + original_levels, + opts, + &payload + .into_iter() + .map(|x| (opts.tree_id, x).into()) + .collect::>(), + ), Choice::DoNothing => { log::trace!("Compactor chose to do nothing"); Ok(()) @@ -133,7 +120,7 @@ fn create_compaction_stream<'a>( .segments .iter() .enumerate() - .filter(|(_, segment)| to_compact.contains(&segment.metadata.id)) + .filter(|(_, segment)| to_compact.contains(&segment.id())) .min_by(|(a, _), (b, _)| a.cmp(b)) .map(|(idx, _)| idx) else { @@ -144,7 +131,7 @@ fn create_compaction_stream<'a>( .segments .iter() .enumerate() - .filter(|(_, segment)| to_compact.contains(&segment.metadata.id)) + .filter(|(_, segment)| to_compact.contains(&segment.id())) .max_by(|(a, _), (b, _)| a.cmp(b)) .map(|(idx, _)| idx) else { @@ -161,7 +148,7 @@ fn create_compaction_stream<'a>( found += hi - lo + 1; } else { for &id in to_compact { - if let Some(segment) = level.segments.iter().find(|x| x.metadata.id == id) { + if let Some(segment) = level.segments.iter().find(|x| x.id() == id) { found += 1; readers.push(Box::new( @@ -181,6 +168,34 @@ fn create_compaction_stream<'a>( } } +fn move_segments( + mut levels: RwLockWriteGuard<'_, LevelManifest>, + opts: &Options, + payload: CompactionPayload, +) -> crate::Result<()> { + // Fail-safe for buggy compaction strategies + if levels.should_decline_compaction(payload.segment_ids.iter().copied()) { + log::warn!( + "Compaction task created by {:?} contained hidden segments, declining to run it - please report this at https://github.com/fjall-rs/lsm-tree/issues/new?template=bug_report.md", + opts.strategy.get_name(), + ); + return Ok(()); + } + + levels.atomic_swap(|recipe| { + for segment_id in payload.segment_ids { + if let Some(segment) = recipe.iter_mut().find_map(|x| x.remove(segment_id)) { + // NOTE: Destination level should definitely exist + #[allow(clippy::expect_used)] + recipe + .get_mut(payload.dest_level as usize) + .expect("should exist") + .insert(segment); + } + } + }) +} + #[allow(clippy::too_many_lines)] fn merge_segments( mut levels: RwLockWriteGuard<'_, LevelManifest>, @@ -189,6 +204,16 @@ fn merge_segments( ) -> crate::Result<()> { if opts.stop_signal.is_stopped() { log::debug!("compactor: stopping before compaction because of stop signal"); + return Ok(()); + } + + // Fail-safe for buggy compaction strategies + if levels.should_decline_compaction(payload.segment_ids.iter().copied()) { + log::warn!( + "Compaction task created by {:?} contained hidden segments, declining to run it - please report this at https://github.com/fjall-rs/lsm-tree/issues/new?template=bug_report.md", + opts.strategy.get_name(), + ); + return Ok(()); } let segments_base_folder = opts.config.path.join(SEGMENTS_FOLDER); @@ -208,6 +233,8 @@ fn merge_segments( levels.hide_segments(payload.segment_ids.iter().copied()); + // IMPORTANT: Free lock so the compaction (which may go on for a while) + // does not block possible other compactions and reads drop(levels); // NOTE: Only evict tombstones when reaching the last level, @@ -216,7 +243,7 @@ fn merge_segments( let start = Instant::now(); - let mut segment_writer = MultiWriter::new( + let Ok(segment_writer) = MultiWriter::new( opts.segment_id_generator.clone(), payload.target_size, crate::segment::writer::Options { @@ -225,8 +252,19 @@ fn merge_segments( data_block_size: opts.config.data_block_size, index_block_size: opts.config.index_block_size, }, - )? - .use_compression(opts.config.compression); + ) else { + log::error!("Compaction failed"); + + // IMPORTANT: Show the segments again, because compaction failed + opts.levels + .write() + .expect("lock is poisoned") + .show_segments(payload.segment_ids.iter().copied()); + + return Ok(()); + }; + + let mut segment_writer = segment_writer.use_compression(opts.config.compression); #[cfg(feature = "bloom")] { @@ -235,10 +273,11 @@ fn merge_segments( if opts.config.bloom_bits_per_key >= 0 { // NOTE: Apply some MONKEY to have very high FPR on small levels // because it's cheap + // + // See https://nivdayan.github.io/monkeykeyvaluestore.pdf let bloom_policy = match payload.dest_level { - // TODO: increase to 0.00001 when https://github.com/fjall-rs/lsm-tree/issues/63 is fixed - 0 => BloomConstructionPolicy::FpRate(0.0001), - 1 => BloomConstructionPolicy::FpRate(0.001), + 0 => BloomConstructionPolicy::FpRate(0.00001), + 1 => BloomConstructionPolicy::FpRate(0.0005), _ => BloomConstructionPolicy::BitsPerKey( opts.config.bloom_bits_per_key.unsigned_abs(), ), @@ -252,14 +291,34 @@ fn merge_segments( } for (idx, item) in merge_iter.enumerate() { - let item = item?; + let Ok(item) = item else { + log::error!("Compaction failed"); + + // IMPORTANT: Show the segments again, because compaction failed + opts.levels + .write() + .expect("lock is poisoned") + .show_segments(payload.segment_ids.iter().copied()); + + return Ok(()); + }; // IMPORTANT: We can only drop tombstones when writing into last level if is_last_level && item.is_tombstone() { continue; } - segment_writer.write(item)?; + if segment_writer.write(item).is_err() { + log::error!("Compaction failed"); + + // IMPORTANT: Show the segments again, because compaction failed + opts.levels + .write() + .expect("lock is poisoned") + .show_segments(payload.segment_ids.iter().copied()); + + return Ok(()); + }; if idx % 100_000 == 0 && opts.stop_signal.is_stopped() { log::debug!("compactor: stopping amidst compaction because of stop signal"); @@ -267,35 +326,55 @@ fn merge_segments( } } - let writer_results = segment_writer.finish()?; + let Ok(writer_results) = segment_writer.finish() else { + log::error!("Compaction failed"); + + // IMPORTANT: Show the segments again, because compaction failed + opts.levels + .write() + .expect("lock is poisoned") + .show_segments(payload.segment_ids.iter().copied()); + + return Ok(()); + }; log::debug!( "Compacted in {}ms ({} segments created)", start.elapsed().as_millis(), - writer_results.len() + writer_results.len(), ); - let created_segments = writer_results + let Ok(created_segments) = writer_results .into_iter() .map(|trailer| -> crate::Result { let segment_id = trailer.metadata.id; let segment_file_path = segments_base_folder.join(segment_id.to_string()); - let tli_ptr = trailer.offsets.tli_ptr; - - #[cfg(feature = "bloom")] - let bloom_ptr = trailer.offsets.bloom_ptr; - - // NOTE: Need to allow because of false positive in Clippy - // because of "bloom" feature - #[allow(clippy::needless_borrows_for_generic_args)] - let block_index = Arc::new(TwoLevelBlockIndex::from_file( - &segment_file_path, - tli_ptr, - (opts.tree_id, segment_id).into(), - opts.config.descriptor_table.clone(), - opts.config.block_cache.clone(), - )?); + let block_index = match payload.dest_level { + 0 | 1 => { + let block_index = FullBlockIndex::from_file( + &segment_file_path, + &trailer.metadata, + &trailer.offsets, + )?; + BlockIndexImpl::Full(block_index) + } + _ => { + // NOTE: Need to allow because of false positive in Clippy + // because of "bloom" feature + #[allow(clippy::needless_borrows_for_generic_args)] + let block_index = TwoLevelBlockIndex::from_file( + &segment_file_path, + &trailer.metadata, + trailer.offsets.tli_ptr, + (opts.tree_id, segment_id).into(), + opts.config.descriptor_table.clone(), + opts.config.block_cache.clone(), + )?; + BlockIndexImpl::TwoLevel(block_index) + } + }; + let block_index = Arc::new(block_index); Ok(SegmentInner { tree_id: opts.tree_id, @@ -310,24 +389,40 @@ fn merge_segments( block_index, #[cfg(feature = "bloom")] - bloom_filter: Segment::load_bloom(&segment_file_path, bloom_ptr)?, + bloom_filter: { + match Segment::load_bloom(&segment_file_path, trailer.offsets.bloom_ptr) { + Ok(filter) => filter, + Err(e) => return Err(e), + } + }, } .into()) }) - .collect::>>()?; + .collect::>>() + else { + log::error!("Compaction failed"); + + // IMPORTANT: Show the segments again, because compaction failed + opts.levels + .write() + .expect("lock is poisoned") + .show_segments(payload.segment_ids.iter().copied()); + + return Ok(()); + }; // NOTE: Mind lock order L -> M -> S log::trace!("compactor: acquiring levels manifest write lock"); - let mut original_levels = opts.levels.write().expect("lock is poisoned"); + let mut levels = opts.levels.write().expect("lock is poisoned"); // IMPORTANT: Write lock memtable(s), otherwise segments may get deleted while a range read is happening // NOTE: Mind lock order L -> M -> S log::trace!("compactor: acquiring sealed memtables write lock"); let sealed_memtables_guard = opts.sealed_memtables.write().expect("lock is poisoned"); - let swap_result = original_levels.atomic_swap(|recipe| { + let swap_result = levels.atomic_swap(|recipe| { for segment in created_segments.iter().cloned() { - log::trace!("Persisting segment {}", segment.metadata.id); + log::trace!("Persisting segment {}", segment.id()); recipe .get_mut(payload.dest_level as usize) @@ -346,17 +441,16 @@ fn merge_segments( if let Err(e) = swap_result { // IMPORTANT: Show the segments again, because compaction failed - original_levels.show_segments(payload.segment_ids.iter().copied()); + levels.show_segments(payload.segment_ids.iter().copied()); return Err(e); }; for segment in &created_segments { - let segment_file_path = segments_base_folder.join(segment.metadata.id.to_string()); + let segment_file_path = segments_base_folder.join(segment.id().to_string()); - opts.config.descriptor_table.insert( - &segment_file_path, - (opts.tree_id, segment.metadata.id).into(), - ); + opts.config + .descriptor_table + .insert(&segment_file_path, segment.global_id()); } // NOTE: Segments are registered, we can unlock the memtable(s) safely @@ -382,9 +476,9 @@ fn merge_segments( .remove((opts.tree_id, *segment_id).into()); } - original_levels.show_segments(payload.segment_ids.iter().copied()); + levels.show_segments(payload.segment_ids.iter().copied()); - drop(original_levels); + drop(levels); log::debug!("compactor: done"); @@ -392,10 +486,19 @@ fn merge_segments( } fn drop_segments( - mut original_levels: RwLockWriteGuard<'_, LevelManifest>, + mut levels: RwLockWriteGuard<'_, LevelManifest>, opts: &Options, segment_ids: &[GlobalSegmentId], ) -> crate::Result<()> { + // Fail-safe for buggy compaction strategies + if levels.should_decline_compaction(segment_ids.iter().map(GlobalSegmentId::segment_id)) { + log::warn!( + "Compaction task created by {:?} contained hidden segments, declining to run it - please report this at https://github.com/fjall-rs/lsm-tree/issues/new?template=bug_report.md", + opts.strategy.get_name(), + ); + return Ok(()); + } + let segments_base_folder = opts.config.path.join(SEGMENTS_FOLDER); // IMPORTANT: Write lock memtable, otherwise segments may get deleted while a range read is happening @@ -404,7 +507,7 @@ fn drop_segments( // IMPORTANT: Write the segment with the removed segments first // Otherwise the folder is deleted, but the segment is still referenced! - original_levels.atomic_swap(|recipe| { + levels.atomic_swap(|recipe| { for key in segment_ids { let segment_id = key.segment_id(); log::trace!("Removing segment {segment_id}"); @@ -416,7 +519,7 @@ fn drop_segments( })?; drop(memtable_lock); - drop(original_levels); + drop(levels); // NOTE: If the application were to crash >here< it's fine // The segments are not referenced anymore, and will be diff --git a/src/descriptor_table/mod.rs b/src/descriptor_table/mod.rs index 7715b22b..1f1c6a10 100644 --- a/src/descriptor_table/mod.rs +++ b/src/descriptor_table/mod.rs @@ -53,6 +53,9 @@ pub struct FileDescriptorTableInner { size: AtomicUsize, } +/// The descriptor table caches file descriptors to avoid `fopen()` calls +/// +/// See `TableCache` in `RocksDB`. #[doc(alias("table cache"))] #[allow(clippy::module_name_repetitions)] pub struct FileDescriptorTable { diff --git a/src/key.rs b/src/key.rs index 6693be0a..45ed596f 100644 --- a/src/key.rs +++ b/src/key.rs @@ -104,3 +104,55 @@ impl Ord for InternalKey { (&self.user_key, Reverse(self.seqno)).cmp(&(&other.user_key, Reverse(other.seqno))) } } + +// TODO: wait for new crossbeam-skiplist +// TODO: https://github.com/crossbeam-rs/crossbeam/pull/1162 +// +// impl Equivalent> for InternalKey { +// fn equivalent(&self, other: &InternalKeyRef<'_>) -> bool { +// self.user_key == other.user_key && self.seqno == other.seqno +// } +// } + +// impl Comparable> for InternalKey { +// fn compare(&self, other: &InternalKeyRef<'_>) -> std::cmp::Ordering { +// (&*self.user_key, Reverse(self.seqno)).cmp(&(other.user_key, Reverse(other.seqno))) +// } +// } + +// Temporary internal key without heap allocation +// #[derive(Debug, Eq)] +// pub struct InternalKeyRef<'a> { +// pub user_key: &'a [u8], +// pub seqno: SeqNo, +// pub value_type: ValueType, +// } + +// impl<'a> InternalKeyRef<'a> { +// // Constructor for InternalKeyRef +// pub fn new(user_key: &'a [u8], seqno: u64, value_type: ValueType) -> Self { +// InternalKeyRef { +// user_key, +// seqno, +// value_type, +// } +// } +// } + +// impl<'a> PartialEq for InternalKeyRef<'a> { +// fn eq(&self, other: &Self) -> bool { +// self.user_key == other.user_key && self.seqno == other.seqno +// } +// } + +// impl<'a> PartialOrd for InternalKeyRef<'a> { +// fn partial_cmp(&self, other: &Self) -> Option { +// Some(self.cmp(other)) +// } +// } + +// impl<'a> Ord for InternalKeyRef<'a> { +// fn cmp(&self, other: &Self) -> std::cmp::Ordering { +// (&self.user_key, Reverse(self.seqno)).cmp(&(&other.user_key, Reverse(other.seqno))) +// } +// } diff --git a/src/key_range.rs b/src/key_range.rs index d2b9aeea..e8c2e583 100644 --- a/src/key_range.rs +++ b/src/key_range.rs @@ -65,12 +65,21 @@ impl KeyRange { true } + /// Returns `true` if the key falls within this key range. pub fn contains_key>(&self, key: K) -> bool { let key = key.as_ref(); let (start, end) = &self.0; key >= *start && key <= *end } + /// Returns `true` if the `other` is fully contained in this range. + pub fn contains_range(&self, other: &Self) -> bool { + let (start1, end1) = &self.0; + let (start2, end2) = &other.0; + start1 <= start2 && end1 >= end2 + } + + /// Returns `true` if the `other` overlaps at least partially with this range. pub fn overlaps_with_key_range(&self, other: &Self) -> bool { let (start1, end1) = &self.0; let (start2, end2) = &other.0; @@ -183,7 +192,7 @@ mod tests { } #[test] - fn key_range_aggregate() { + fn key_range_aggregate_1() { let ranges = [ int_key_range(2, 4), int_key_range(0, 4), @@ -195,6 +204,19 @@ mod tests { assert_eq!([0, 0, 0, 0, 0, 0, 0, 10], &*max); } + #[test] + fn key_range_aggregate_2() { + let ranges = [ + int_key_range(6, 7), + int_key_range(0, 2), + int_key_range(0, 10), + ]; + let aggregated = KeyRange::aggregate(ranges.iter()); + let (min, max) = aggregated.0; + assert_eq!([0, 0, 0, 0, 0, 0, 0, 0], &*min); + assert_eq!([0, 0, 0, 0, 0, 0, 0, 10], &*max); + } + mod is_disjoint { use super::*; use test_log::test; diff --git a/src/level_manifest/hidden_set.rs b/src/level_manifest/hidden_set.rs new file mode 100644 index 00000000..a37f7df1 --- /dev/null +++ b/src/level_manifest/hidden_set.rs @@ -0,0 +1,45 @@ +use crate::segment::meta::SegmentId; +use crate::HashSet; + +/// The hidden set keeps track of which segments are currently being compacted +/// +/// When a segment is hidden (being compacted), no other compaction task can include that +/// segment, or it will be declined to be run. +/// +/// If a compaction task fails, the segments are shown again (removed from the hidden set). +#[derive(Clone)] +pub struct HiddenSet { + pub(crate) set: HashSet, +} + +impl Default for HiddenSet { + fn default() -> Self { + Self { + set: HashSet::with_capacity_and_hasher(10, xxhash_rust::xxh3::Xxh3Builder::new()), + } + } +} + +impl HiddenSet { + pub(crate) fn hide>(&mut self, keys: T) { + self.set.extend(keys); + } + + pub(crate) fn show>(&mut self, keys: T) { + for key in keys { + self.set.remove(&key); + } + } + + pub(crate) fn is_blocked>(&self, ids: T) -> bool { + ids.into_iter().any(|id| self.is_hidden(id)) + } + + pub(crate) fn is_hidden(&self, key: SegmentId) -> bool { + self.set.contains(&key) + } + + pub(crate) fn is_empty(&self) -> bool { + self.set.is_empty() + } +} diff --git a/src/level_manifest/iter.rs b/src/level_manifest/iter.rs deleted file mode 100644 index a1727186..00000000 --- a/src/level_manifest/iter.rs +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) 2024-present, fjall-rs -// This source code is licensed under both the Apache 2.0 and MIT License -// (found in the LICENSE-* files in the repository) - -use super::LevelManifest; -use crate::Segment; - -/// Iterates through all levels -pub struct LevelManifestIterator<'a> { - level_manifest: &'a LevelManifest, - current_level: usize, - current_idx: usize, -} - -impl<'a> LevelManifestIterator<'a> { - #[must_use] - pub fn new(level_manifest: &'a LevelManifest) -> Self { - Self { - level_manifest, - current_idx: 0, - current_level: 0, - } - } -} - -impl<'a> Iterator for LevelManifestIterator<'a> { - type Item = &'a Segment; - - fn next(&mut self) -> Option { - loop { - let segment = self - .level_manifest - .levels - .get(self.current_level)? - .segments - .get(self.current_idx); - - if let Some(segment) = segment { - self.current_idx += 1; - return Some(segment); - } - - self.current_level += 1; - self.current_idx = 0; - } - } -} diff --git a/src/level_manifest/level.rs b/src/level_manifest/level.rs index 32199ed2..481c4485 100644 --- a/src/level_manifest/level.rs +++ b/src/level_manifest/level.rs @@ -17,13 +17,13 @@ pub struct Level { /// is only recomputed when the level is changed /// to avoid unnecessary CPU work pub is_disjoint: bool, + // pub key_range: KeyRange, } impl std::fmt::Display for Level { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { for segment in self.segments.iter().rev().take(2).rev() { - let id = segment.metadata.id; - write!(f, "[{id}]")?; + write!(f, "[{}]", segment.id())?; } Ok(()) } @@ -41,26 +41,35 @@ impl Default for Level { fn default() -> Self { Self { is_disjoint: true, - segments: Vec::with_capacity(10), + segments: Vec::new(), + // key_range: KeyRange::empty(), } } } impl Level { pub fn list_ids(&self) -> HashSet { - self.segments.iter().map(|x| x.metadata.id).collect() + self.segments.iter().map(Segment::id).collect() } - pub fn insert(&mut self, segment: Segment) { - self.segments.push(segment); + pub fn update_metadata(&mut self) { self.set_disjoint_flag(); self.sort(); } - pub fn remove(&mut self, segment_id: SegmentId) { - self.segments.retain(|x| segment_id != x.metadata.id); - self.set_disjoint_flag(); - self.sort(); + pub fn insert(&mut self, segment: Segment) { + self.segments.push(segment); + self.update_metadata(); + } + + pub fn remove(&mut self, segment_id: SegmentId) -> Option { + if let Some(idx) = self.segments.iter().position(|x| x.id() == segment_id) { + let segment = self.segments.remove(idx); + self.update_metadata(); + Some(segment) + } else { + None + } } pub(crate) fn sort(&mut self) { @@ -96,7 +105,7 @@ impl Level { /// Returns an iterator over the level's segment IDs. pub fn ids(&self) -> impl Iterator + '_ { - self.segments.iter().map(|x| x.metadata.id) + self.segments.iter().map(Segment::id) } /// Returns `true` if the level contains no segments. @@ -114,15 +123,19 @@ impl Level { self.segments.iter().map(|x| x.metadata.file_size).sum() } - /// Checks if the level is disjoint and caches the result in `is_disjoint`. - fn set_disjoint_flag(&mut self) { + pub(crate) fn compute_is_disjoint(&self) -> bool { let ranges = self .segments .iter() .map(|x| &x.metadata.key_range) .collect::>(); - self.is_disjoint = KeyRange::is_disjoint(&ranges); + KeyRange::is_disjoint(&ranges) + } + + /// Checks if the level is disjoint and caches the result in `is_disjoint`. + fn set_disjoint_flag(&mut self) { + self.is_disjoint = self.compute_is_disjoint(); } /// Returns an iterator over segments in the level that have a key range @@ -136,6 +149,17 @@ impl Level { .filter(|x| x.metadata.key_range.overlaps_with_key_range(key_range)) } + /// Returns an iterator over segments in the level that have a key range + /// fully contained in the input key range. + pub fn contained_segments<'a>( + &'a self, + key_range: &'a KeyRange, + ) -> impl Iterator { + self.segments + .iter() + .filter(|x| key_range.contains_range(&x.metadata.key_range)) + } + pub fn as_disjoint(&self) -> Option> { if self.is_disjoint { Some(DisjointLevel(self)) @@ -223,7 +247,7 @@ mod tests { descriptor_table::FileDescriptorTable, key_range::KeyRange, segment::{ - block_index::two_level_index::TwoLevelBlockIndex, + block_index::{two_level_index::TwoLevelBlockIndex, BlockIndexImpl}, file_offsets::FileOffsets, meta::{Metadata, SegmentId}, value_block::BlockOffset, @@ -234,17 +258,17 @@ mod tests { use std::sync::Arc; use test_log::test; - #[cfg(feature = "bloom")] - use crate::bloom::BloomFilter; - #[allow(clippy::expect_used)] fn fixture_segment(id: SegmentId, key_range: KeyRange) -> Segment { let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); + let block_index = TwoLevelBlockIndex::new((0, id).into(), block_cache.clone()); + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); + SegmentInner { tree_id: 0, descriptor_table: Arc::new(FileDescriptorTable::new(512, 1)), - block_index: Arc::new(TwoLevelBlockIndex::new((0, id).into(), block_cache.clone())), + block_index, offsets: FileOffsets { bloom_ptr: BlockOffset(0), @@ -277,7 +301,7 @@ mod tests { block_cache, #[cfg(feature = "bloom")] - bloom_filter: Some(BloomFilter::with_fp_rate(1, 0.1)), + bloom_filter: Some(crate::bloom::BloomFilter::with_fp_rate(1, 0.1)), } .into() } @@ -287,6 +311,7 @@ mod tests { fn level_disjoint_cull() { let level = Level { is_disjoint: true, + // key_range: KeyRange::empty(), segments: vec![ fixture_segment(0, KeyRange::new((Slice::from("a"), Slice::from("c")))), fixture_segment(1, KeyRange::new((Slice::from("d"), Slice::from("g")))), @@ -481,7 +506,7 @@ mod tests { Vec::::new(), level .overlapping_segments(&KeyRange::new((b"a".to_vec().into(), b"b".to_vec().into()))) - .map(|x| x.metadata.id) + .map(Segment::id) .collect::>(), ); @@ -489,7 +514,7 @@ mod tests { vec![1], level .overlapping_segments(&KeyRange::new((b"d".to_vec().into(), b"k".to_vec().into()))) - .map(|x| x.metadata.id) + .map(Segment::id) .collect::>(), ); @@ -497,7 +522,7 @@ mod tests { vec![1, 2], level .overlapping_segments(&KeyRange::new((b"f".to_vec().into(), b"x".to_vec().into()))) - .map(|x| x.metadata.id) + .map(Segment::id) .collect::>(), ); } diff --git a/src/level_manifest/mod.rs b/src/level_manifest/mod.rs index 30395ac1..c89e2b0a 100644 --- a/src/level_manifest/mod.rs +++ b/src/level_manifest/mod.rs @@ -2,7 +2,7 @@ // This source code is licensed under both the Apache 2.0 and MIT License // (found in the LICENSE-* files in the repository) -pub mod iter; +pub(crate) mod hidden_set; pub(crate) mod level; use crate::{ @@ -13,7 +13,7 @@ use crate::{ HashMap, HashSet, }; use byteorder::{BigEndian, ReadBytesExt, WriteBytesExt}; -use iter::LevelManifestIterator; +use hidden_set::HiddenSet; use level::Level; use std::{ io::{Cursor, Read, Write}, @@ -21,23 +21,21 @@ use std::{ sync::Arc, }; -pub type HiddenSet = HashSet; - type Levels = Vec>; -/// Represents the levels of a log-structured merge tree. +/// Represents the levels of a log-structured merge tree pub struct LevelManifest { - /// Path of level manifest file + /// Path of level manifest file. path: PathBuf, - /// Actual levels containing segments + /// Actual levels containing segments. #[doc(hidden)] pub levels: Levels, - /// Set of segment IDs that are masked + /// Set of segment IDs that are masked. /// /// While consuming segments (because of compaction) they will not appear in the list of segments - /// as to not cause conflicts between multiple compaction threads (compacting the same segments) + /// as to not cause conflicts between multiple compaction threads (compacting the same segments). hidden_set: HiddenSet, is_disjoint: bool, @@ -46,15 +44,23 @@ pub struct LevelManifest { impl std::fmt::Display for LevelManifest { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { for (idx, level) in self.levels.iter().enumerate() { - write!(f, "{idx}: ")?; + write!( + f, + "{idx} [{}]: ", + match (level.is_empty(), level.compute_is_disjoint()) { + (true, _) => ".", + (false, true) => "D", + (false, false) => "_", + } + )?; if level.segments.is_empty() { write!(f, "")?; - } else if level.segments.len() >= 10 { + } else if level.segments.len() >= 30 { #[allow(clippy::indexing_slicing)] for segment in level.segments.iter().take(2) { - let id = segment.metadata.id; - let is_hidden = self.hidden_set.contains(&id); + let id = segment.id(); + let is_hidden = self.hidden_set.is_hidden(id); write!( f, @@ -67,8 +73,8 @@ impl std::fmt::Display for LevelManifest { #[allow(clippy::indexing_slicing)] for segment in level.segments.iter().rev().take(2).rev() { - let id = segment.metadata.id; - let is_hidden = self.hidden_set.contains(&id); + let id = segment.id(); + let is_hidden = self.hidden_set.is_hidden(id); write!( f, @@ -79,8 +85,8 @@ impl std::fmt::Display for LevelManifest { } } else { for segment in &level.segments { - let id = segment.metadata.id; - let is_hidden = self.hidden_set.contains(&id); + let id = segment.id(); + let is_hidden = self.hidden_set.is_hidden(id); write!( f, @@ -96,7 +102,7 @@ impl std::fmt::Display for LevelManifest { f, " | # = {}, {} MiB", level.len(), - level.size() / 1_024 / 1_024 + level.size() / 1_024 / 1_024, )?; } @@ -118,10 +124,7 @@ impl LevelManifest { let mut manifest = Self { path: path.as_ref().to_path_buf(), levels, - hidden_set: HashSet::with_capacity_and_hasher( - 10, - xxhash_rust::xxh3::Xxh3Builder::new(), - ), + hidden_set: Default::default(), is_disjoint: true, }; Self::write_to_disk(path, &manifest.deep_clone())?; @@ -175,11 +178,24 @@ impl LevelManifest { Ok(levels) } - pub(crate) fn recover_ids>(path: P) -> crate::Result> { - Ok(Self::load_level_manifest(path)? - .into_iter() - .flatten() - .collect()) + pub(crate) fn recover_ids>( + path: P, + ) -> crate::Result> { + let manifest = Self::load_level_manifest(path)?; + let mut result = crate::HashMap::default(); + + for (level_idx, segment_ids) in manifest.into_iter().enumerate() { + for segment_id in segment_ids { + result.insert( + segment_id, + level_idx + .try_into() + .expect("there are less than 256 levels"), + ); + } + } + + Ok(result) } fn resolve_levels( @@ -205,19 +221,13 @@ impl LevelManifest { pub(crate) fn recover>(path: P, segments: Vec) -> crate::Result { let level_manifest = Self::load_level_manifest(&path)?; - let segments: HashMap<_, _> = segments - .into_iter() - .map(|seg| (seg.metadata.id, seg)) - .collect(); + let segments: HashMap<_, _> = segments.into_iter().map(|seg| (seg.id(), seg)).collect(); let levels = Self::resolve_levels(level_manifest, &segments); let mut manifest = Self { levels, - hidden_set: HashSet::with_capacity_and_hasher( - 10, - xxhash_rust::xxh3::Xxh3Builder::new(), - ), + hidden_set: HiddenSet::default(), path: path.as_ref().to_path_buf(), is_disjoint: false, }; @@ -270,7 +280,7 @@ impl LevelManifest { Self::write_to_disk(&self.path, &working_copy)?; self.levels = working_copy.into_iter().map(Arc::new).collect(); - self.sort_levels(); + self.update_metadata(); self.set_disjoint_flag(); log::trace!("Swapped level manifest to:\n{self}"); @@ -284,11 +294,11 @@ impl LevelManifest { self.insert_into_level(0, segment); } - pub(crate) fn sort_levels(&mut self) { + pub fn update_metadata(&mut self) { for level in &mut self.levels { Arc::get_mut(level) .expect("could not get mutable Arc - this is a bug") - .sort(); + .update_metadata(); } } @@ -358,14 +368,10 @@ impl LevelManifest { HashSet::with_capacity_and_hasher(self.len(), xxhash_rust::xxh3::Xxh3Builder::new()); for (idx, level) in self.levels.iter().enumerate() { - for segment_id in level.ids() { - if self.hidden_set.contains(&segment_id) { - // NOTE: Level count is u8 - #[allow(clippy::cast_possible_truncation)] - let idx = idx as u8; - - output.insert(idx); - } + if level.ids().any(|id| self.hidden_set.is_hidden(id)) { + // NOTE: Level count is u8 + #[allow(clippy::cast_possible_truncation)] + output.insert(idx as u8); } } @@ -379,7 +385,7 @@ impl LevelManifest { for raw_level in &self.levels { let mut level = raw_level.iter().cloned().collect::>(); - level.retain(|x| !self.hidden_set.contains(&x.metadata.id)); + level.retain(|x| !self.hidden_set.is_hidden(x.id())); output.push(Level { segments: level, @@ -391,29 +397,26 @@ impl LevelManifest { } pub fn iter(&self) -> impl Iterator + '_ { - LevelManifestIterator::new(self) + self.levels.iter().flat_map(|x| &x.segments) } - pub(crate) fn get_all_segments(&self) -> HashMap { - let mut output = HashMap::with_hasher(xxhash_rust::xxh3::Xxh3Builder::new()); - - for segment in self.iter().cloned() { - output.insert(segment.metadata.id, segment); - } + pub(crate) fn should_decline_compaction>( + &self, + ids: T, + ) -> bool { + self.hidden_set().is_blocked(ids) + } - output + pub(crate) fn hidden_set(&self) -> &HiddenSet { + &self.hidden_set } - pub(crate) fn show_segments(&mut self, keys: impl Iterator) { - for key in keys { - self.hidden_set.remove(&key); - } + pub(crate) fn hide_segments>(&mut self, keys: T) { + self.hidden_set.hide(keys); } - pub(crate) fn hide_segments(&mut self, keys: impl Iterator) { - for key in keys { - self.hidden_set.insert(key); - } + pub(crate) fn show_segments>(&mut self, keys: T) { + self.hidden_set.show(keys); } } @@ -432,7 +435,7 @@ impl Encode for Vec { writer.write_u32::(level.segments.len() as u32)?; for segment in &level.segments { - writer.write_u64::(segment.metadata.id)?; + writer.write_u64::(segment.id())?; } } @@ -443,8 +446,11 @@ impl Encode for Vec { #[cfg(test)] #[allow(clippy::expect_used)] mod tests { - use crate::{coding::Encode, level_manifest::LevelManifest, AbstractTree}; - use std::collections::HashSet; + use crate::{ + coding::Encode, + level_manifest::{hidden_set::HiddenSet, LevelManifest}, + AbstractTree, + }; use test_log::test; #[test] @@ -492,7 +498,7 @@ mod tests { #[test] fn level_manifest_raw_empty() -> crate::Result<()> { let manifest = LevelManifest { - hidden_set: HashSet::default(), + hidden_set: HiddenSet::default(), levels: Vec::default(), path: "a".into(), is_disjoint: false, diff --git a/src/segment/level_reader.rs b/src/level_reader.rs similarity index 98% rename from src/segment/level_reader.rs rename to src/level_reader.rs index 792ca4f1..334c6256 100644 --- a/src/segment/level_reader.rs +++ b/src/level_reader.rs @@ -2,8 +2,11 @@ // This source code is licensed under both the Apache 2.0 and MIT License // (found in the LICENSE-* files in the repository) -use super::{range::Range, value_block::CachePolicy}; -use crate::{level_manifest::level::Level, InternalValue, UserKey}; +use crate::{ + level_manifest::level::Level, + segment::{range::Range, value_block::CachePolicy}, + InternalValue, UserKey, +}; use std::{ops::Bound, sync::Arc}; /// Reads through a disjoint level diff --git a/src/lib.rs b/src/lib.rs index 73ef002e..69b2b63c 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -154,6 +154,8 @@ mod key_range; #[doc(hidden)] pub mod level_manifest; +mod level_reader; + mod manifest; mod memtable; @@ -161,7 +163,10 @@ mod memtable; pub mod merge; mod multi_reader; -mod mvcc_stream; + +#[doc(hidden)] +pub mod mvcc_stream; + mod path; #[doc(hidden)] diff --git a/src/memtable/mod.rs b/src/memtable/mod.rs index d8f7b8e0..a5b390f1 100644 --- a/src/memtable/mod.rs +++ b/src/memtable/mod.rs @@ -7,24 +7,33 @@ use crate::segment::block::ItemSize; use crate::value::{InternalValue, SeqNo, UserValue, ValueType}; use crossbeam_skiplist::SkipMap; use std::ops::RangeBounds; -use std::sync::atomic::AtomicU32; +use std::sync::atomic::{AtomicU32, AtomicU64}; -/// The memtable serves as an intermediary storage for new items +/// The memtable serves as an intermediary, ephemeral, sorted storage for new items +/// +/// When the Memtable exceeds some size, it should be flushed to a disk segment. #[derive(Default)] pub struct Memtable { + /// The actual content, stored in a lock-free skiplist. #[doc(hidden)] pub items: SkipMap, - /// Approximate active memtable size + /// Approximate active memtable size. /// - /// If this grows too large, a flush is triggered + /// If this grows too large, a flush is triggered. pub(crate) approximate_size: AtomicU32, + + /// Highest encountered sequence number. + /// + /// This is used so that `get_highest_seqno` has O(1) complexity. + pub(crate) highest_seqno: AtomicU64, } impl Memtable { /// Clears the memtable. pub fn clear(&mut self) { self.items.clear(); + self.highest_seqno = AtomicU64::new(0); self.approximate_size .store(0, std::sync::atomic::Ordering::Release); } @@ -41,7 +50,7 @@ impl Memtable { pub(crate) fn range<'a, R: RangeBounds + 'a>( &'a self, range: R, - ) -> impl DoubleEndedIterator + '_ { + ) -> impl DoubleEndedIterator + 'a { self.items.range(range).map(|entry| InternalValue { key: entry.key().clone(), value: entry.value().clone(), @@ -126,18 +135,22 @@ impl Memtable { let key = InternalKey::new(item.key.user_key, item.key.seqno, item.key.value_type); self.items.insert(key, item.value); + self.highest_seqno + .fetch_max(item.key.seqno, std::sync::atomic::Ordering::AcqRel); + (item_size, size_before + item_size) } /// Returns the highest sequence number in the memtable. pub fn get_highest_seqno(&self) -> Option { - self.items - .iter() - .map(|x| { - let key = x.key(); - key.seqno - }) - .max() + if self.is_empty() { + None + } else { + Some( + self.highest_seqno + .load(std::sync::atomic::Ordering::Acquire), + ) + } } } diff --git a/src/mvcc_stream.rs b/src/mvcc_stream.rs index 27eac255..258d3225 100644 --- a/src/mvcc_stream.rs +++ b/src/mvcc_stream.rs @@ -97,6 +97,7 @@ impl>> DoubleEndedIte mod tests { use super::*; use crate::value::{InternalValue, ValueType}; + use test_log::test; macro_rules! stream { ($($key:expr, $sub_key:expr, $value_type:expr),* $(,)?) => {{ @@ -147,7 +148,7 @@ mod tests { }; } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_queue_reverse_almost_gone() -> crate::Result<()> { let vec = [ @@ -193,7 +194,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_queue_almost_gone_2() -> crate::Result<()> { let vec = [ @@ -235,7 +236,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_queue() -> crate::Result<()> { let vec = [ @@ -278,7 +279,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_queue_weak_almost_gone() -> crate::Result<()> { let vec = [ @@ -324,7 +325,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_queue_weak_almost_gone_2() -> crate::Result<()> { let vec = [ @@ -366,7 +367,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_queue_weak_reverse() -> crate::Result<()> { let vec = [ @@ -409,7 +410,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_stream_simple() -> crate::Result<()> { #[rustfmt::skip] @@ -433,7 +434,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_stream_simple_multi_keys() -> crate::Result<()> { #[rustfmt::skip] @@ -470,7 +471,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_stream_tombstone() -> crate::Result<()> { #[rustfmt::skip] @@ -494,7 +495,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_stream_tombstone_multi_keys() -> crate::Result<()> { #[rustfmt::skip] @@ -531,7 +532,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_stream_weak_tombstone_simple() -> crate::Result<()> { #[rustfmt::skip] @@ -555,7 +556,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_stream_weak_tombstone_resurrection() -> crate::Result<()> { #[rustfmt::skip] @@ -580,7 +581,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_stream_weak_tombstone_priority() -> crate::Result<()> { #[rustfmt::skip] @@ -606,7 +607,7 @@ mod tests { Ok(()) } - #[test_log::test] + #[test] #[allow(clippy::unwrap_used)] fn mvcc_stream_weak_tombstone_multi_keys() -> crate::Result<()> { #[rustfmt::skip] diff --git a/src/range.rs b/src/range.rs index 829bc7b1..404550b5 100644 --- a/src/range.rs +++ b/src/range.rs @@ -5,11 +5,12 @@ use crate::{ key::InternalKey, level_manifest::LevelManifest, + level_reader::LevelReader, memtable::Memtable, merge::{BoxedIterator, Merger}, multi_reader::MultiReader, mvcc_stream::MvccStream, - segment::{level_reader::LevelReader, value_block::CachePolicy}, + segment::value_block::CachePolicy, tree::inner::SealedMemtables, value::{SeqNo, UserKey}, InternalValue, diff --git a/src/segment/block_index/block_handle.rs b/src/segment/block_index/block_handle.rs index 6845c231..e1be126f 100644 --- a/src/segment/block_index/block_handle.rs +++ b/src/segment/block_index/block_handle.rs @@ -97,6 +97,7 @@ impl Decode for KeyedBlockHandle { #[cfg(test)] mod tests { use super::*; + use test_log::test; #[test] fn index_block_size() { diff --git a/src/segment/block_index/full_index.rs b/src/segment/block_index/full_index.rs new file mode 100644 index 00000000..1268465d --- /dev/null +++ b/src/segment/block_index/full_index.rs @@ -0,0 +1,87 @@ +use super::{block_handle::KeyedBlockHandle, BlockIndex}; +use crate::segment::{ + block_index::IndexBlock, + value_block::{BlockOffset, CachePolicy}, +}; +use std::{fs::File, io::Seek, path::Path}; + +/// Index that translates item keys to data block handles +/// +/// The index is fully loaded into memory. +/// +/// Currently, a full block index is used for L0 & L1 segments. +pub struct FullBlockIndex(Box<[KeyedBlockHandle]>); + +impl std::ops::Deref for FullBlockIndex { + type Target = Box<[KeyedBlockHandle]>; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl FullBlockIndex { + pub fn from_file>( + path: P, + metadata: &crate::segment::meta::Metadata, + offsets: &crate::segment::file_offsets::FileOffsets, + ) -> crate::Result { + let path = path.as_ref(); + let cnt = metadata.index_block_count as usize; + + log::trace!( + "reading full block index from {path:?} at idx_ptr={} ({cnt} index blocks)", + offsets.index_block_ptr, + ); + + let mut file = File::open(path)?; + file.seek(std::io::SeekFrom::Start(*offsets.index_block_ptr))?; + + let mut block_handles = Vec::with_capacity(cnt); + + for _ in 0..cnt { + let idx_block = IndexBlock::from_reader(&mut file)?.items; + // TODO: 1.80? IntoIter impl for Box<[T]> + block_handles.extend(idx_block.into_vec()); + } + + debug_assert!(!block_handles.is_empty()); + + Ok(Self(block_handles.into_boxed_slice())) + } +} + +impl BlockIndex for FullBlockIndex { + fn get_lowest_block_containing_key( + &self, + key: &[u8], + _: CachePolicy, + ) -> crate::Result> { + use super::KeyedBlockIndex; + + self.0 + .get_lowest_block_containing_key(key, CachePolicy::Read) + .map(|x| x.map(|x| x.offset)) + } + + /// Gets the last block handle that may contain the given item + fn get_last_block_containing_key( + &self, + key: &[u8], + cache_policy: CachePolicy, + ) -> crate::Result> { + use super::KeyedBlockIndex; + + self.0 + .get_last_block_containing_key(key, cache_policy) + .map(|x| x.map(|x| x.offset)) + } + + fn get_last_block_handle(&self, _: CachePolicy) -> crate::Result { + use super::KeyedBlockIndex; + + self.0 + .get_last_block_handle(CachePolicy::Read) + .map(|x| x.offset) + } +} diff --git a/src/segment/block_index/mod.rs b/src/segment/block_index/mod.rs index 8465a12a..a539fcc5 100644 --- a/src/segment/block_index/mod.rs +++ b/src/segment/block_index/mod.rs @@ -3,16 +3,42 @@ // (found in the LICENSE-* files in the repository) pub mod block_handle; +pub mod full_index; pub mod top_level; pub mod two_level_index; pub mod writer; -use super::{block::Block, value_block::CachePolicy}; +use super::{ + block::Block, + value_block::{BlockOffset, CachePolicy}, +}; use block_handle::KeyedBlockHandle; +use full_index::FullBlockIndex; +use two_level_index::TwoLevelBlockIndex; pub type IndexBlock = Block; -impl BlockIndex for [KeyedBlockHandle] { +#[allow(clippy::module_name_repetitions)] +pub trait KeyedBlockIndex { + /// Gets the lowest block handle that may contain the given item + fn get_lowest_block_containing_key( + &self, + key: &[u8], + cache_policy: CachePolicy, + ) -> crate::Result>; + + /// Gets the last block handle that may contain the given item + fn get_last_block_containing_key( + &self, + key: &[u8], + cache_policy: CachePolicy, + ) -> crate::Result>; + + /// Returns a handle to the last block + fn get_last_block_handle(&self, cache_policy: CachePolicy) -> crate::Result<&KeyedBlockHandle>; +} + +impl KeyedBlockIndex for [KeyedBlockHandle] { fn get_lowest_block_containing_key( &self, key: &[u8], @@ -54,33 +80,56 @@ impl BlockIndex for [KeyedBlockHandle] { } } +#[enum_dispatch::enum_dispatch] pub trait BlockIndex { /// Gets the lowest block handle that may contain the given item fn get_lowest_block_containing_key( &self, key: &[u8], cache_policy: CachePolicy, - ) -> crate::Result>; + ) -> crate::Result>; /// Gets the last block handle that may contain the given item fn get_last_block_containing_key( &self, key: &[u8], cache_policy: CachePolicy, - ) -> crate::Result>; + ) -> crate::Result>; /// Returns a handle to the last block - fn get_last_block_handle(&self, cache_policy: CachePolicy) -> crate::Result<&KeyedBlockHandle>; + fn get_last_block_handle(&self, cache_policy: CachePolicy) -> crate::Result; +} + +/// The block index stores references to the positions of blocks on a file and their size +/// +/// __________________ +/// | | +/// | BLOCK0 | +/// |________________| <- 'G': 0x0 +/// | | +/// | BLOCK1 | +/// |________________| <- 'M': 0x... +/// | | +/// | BLOCK2 | +/// |________________| <- 'Z': 0x... +/// +/// The block information can be accessed by key. +/// Because the blocks are sorted, any entries not covered by the index (it is sparse) can be +/// found by finding the highest block that has a lower or equal end key than the searched key (by performing in-memory binary search). +/// In the diagram above, searching for 'J' yields the block starting with 'G'. +/// 'J' must be in that block, because the next block starts with 'M'). +#[enum_dispatch::enum_dispatch(BlockIndex)] +#[allow(clippy::module_name_repetitions)] +pub enum BlockIndexImpl { + Full(FullBlockIndex), + TwoLevel(TwoLevelBlockIndex), } #[cfg(test)] #[allow(clippy::expect_used)] mod tests { use super::*; - use crate::{ - segment::{block_index::BlockIndex, value_block::BlockOffset}, - Slice, - }; + use crate::{segment::value_block::BlockOffset, Slice}; use test_log::test; fn bh>(end_key: K, offset: BlockOffset) -> KeyedBlockHandle { diff --git a/src/segment/block_index/top_level.rs b/src/segment/block_index/top_level.rs index 71672bff..4395e8f7 100644 --- a/src/segment/block_index/top_level.rs +++ b/src/segment/block_index/top_level.rs @@ -2,57 +2,45 @@ // This source code is licensed under both the Apache 2.0 and MIT License // (found in the LICENSE-* files in the repository) -use super::{block_handle::KeyedBlockHandle, BlockIndex}; +use super::{block_handle::KeyedBlockHandle, KeyedBlockIndex}; use crate::segment::{ block_index::IndexBlock, value_block::{BlockOffset, CachePolicy}, }; use std::{fs::File, path::Path}; -/// The block index stores references to the positions of blocks on a file and their size +/// The top-level index (TLI) is the level-0 index in a partitioned (two-level) block index /// -/// __________________ -/// | | -/// | BLOCK0 | -/// |________________| <- 'G': 0x0 -/// | | -/// | BLOCK1 | -/// |________________| <- 'M': 0x... -/// | | -/// | BLOCK2 | -/// |________________| <- 'Z': 0x... -/// -/// The block information can be accessed by key. -/// Because the blocks are sorted, any entries not covered by the index (it is sparse) can be -/// found by finding the highest block that has a lower or equal end key than the searched key (by performing in-memory binary search). -/// In the diagram above, searching for 'J' yields the block starting with 'G'. -/// 'J' must be in that block, because the next block starts with 'M'). +/// See `top_level_index.rs` for more info. #[allow(clippy::module_name_repetitions)] #[derive(Debug)] pub struct TopLevelIndex(Box<[KeyedBlockHandle]>); impl TopLevelIndex { - /// Creates a top-level block index - #[must_use] - pub fn from_boxed_slice(handles: Box<[KeyedBlockHandle]>) -> Self { - Self(handles) - } - - /// Loads a top-level index from disk - pub fn from_file>(path: P, offset: BlockOffset) -> crate::Result { + pub fn from_file>( + path: P, + _: &crate::segment::meta::Metadata, + tli_ptr: BlockOffset, + ) -> crate::Result { let path = path.as_ref(); - log::trace!("reading TLI from {path:?}, offset={offset}"); + + log::trace!("reading TLI from {path:?} at tli_ptr={tli_ptr}"); let mut file = File::open(path)?; + let items = IndexBlock::from_file(&mut file, tli_ptr)?.items; - let items = IndexBlock::from_file(&mut file, offset)?.items; log::trace!("loaded TLI ({path:?}): {items:?}"); - debug_assert!(!items.is_empty()); Ok(Self::from_boxed_slice(items)) } + /// Creates a top-level block index + #[must_use] + pub fn from_boxed_slice(handles: Box<[KeyedBlockHandle]>) -> Self { + Self(handles) + } + #[must_use] pub fn len(&self) -> usize { self.0.len() @@ -68,7 +56,7 @@ impl TopLevelIndex { } } -impl BlockIndex for TopLevelIndex { +impl KeyedBlockIndex for TopLevelIndex { fn get_lowest_block_containing_key( &self, key: &[u8], diff --git a/src/segment/block_index/two_level_index.rs b/src/segment/block_index/two_level_index.rs index c66649e7..1d454548 100644 --- a/src/segment/block_index/two_level_index.rs +++ b/src/segment/block_index/two_level_index.rs @@ -4,13 +4,13 @@ use super::{ super::{id::GlobalSegmentId, value_block::CachePolicy}, - block_handle::KeyedBlockHandle, top_level::TopLevelIndex, BlockIndex, IndexBlock, }; use crate::{ - block_cache::BlockCache, descriptor_table::FileDescriptorTable, - segment::value_block::BlockOffset, + block_cache::BlockCache, + descriptor_table::FileDescriptorTable, + segment::{meta::Metadata, value_block::BlockOffset}, }; use std::{path::Path, sync::Arc}; @@ -29,24 +29,25 @@ impl IndexBlockFetcher { } } -/// Index that translates item keys to block handles +/// Index that translates item keys to data block handles /// /// The index is only partially loaded into memory. /// /// See #[allow(clippy::module_name_repetitions)] pub struct TwoLevelBlockIndex { - descriptor_table: Arc, - - /// Segment ID segment_id: GlobalSegmentId, + descriptor_table: Arc, + /// Level-0 index. Is read-only and always fully loaded. /// /// This index points to index blocks inside the level-1 index. pub(crate) top_level_index: TopLevelIndex, - /// Level-1 index. This index is only partially loaded into memory, decreasing memory usage, compared to a fully loaded one. + /// Level-1 index. + /// + /// This index is only partially loaded into memory, decreasing memory usage, compared to a fully loaded one. /// /// However to find a disk block, one layer of indirection is required: /// @@ -55,13 +56,37 @@ pub struct TwoLevelBlockIndex { index_block_fetcher: IndexBlockFetcher, } +impl BlockIndex for TwoLevelBlockIndex { + fn get_lowest_block_containing_key( + &self, + key: &[u8], + cache_policy: CachePolicy, + ) -> crate::Result> { + self.get_lowest_data_block_handle_containing_item(key, cache_policy) + } + + fn get_last_block_handle(&self, cache_policy: CachePolicy) -> crate::Result { + self.get_last_data_block_handle(cache_policy) + } + + fn get_last_block_containing_key( + &self, + key: &[u8], + cache_policy: CachePolicy, + ) -> crate::Result> { + self.get_last_data_block_handle_containing_item(key, cache_policy) + } +} + impl TwoLevelBlockIndex { /// Gets the lowest block handle that may contain the given item pub fn get_lowest_data_block_handle_containing_item( &self, key: &[u8], cache_policy: CachePolicy, - ) -> crate::Result> { + ) -> crate::Result> { + use super::KeyedBlockIndex; + let Some(index_block_handle) = self .top_level_index .get_lowest_block_containing_key(key, cache_policy) @@ -70,13 +95,17 @@ impl TwoLevelBlockIndex { return Ok(None); }; - let index_block = self.load_index_block(index_block_handle, cache_policy)?; + let index_block = self.load_index_block(index_block_handle.offset, cache_policy)?; - Ok(index_block - .items - .get_lowest_block_containing_key(key, cache_policy) - .expect("cannot fail") - .cloned()) + Ok({ + use super::KeyedBlockIndex; + + index_block + .items + .get_lowest_block_containing_key(key, cache_policy) + .expect("cannot fail") + .map(|x| x.offset) + }) } /// Gets the last block handle that may contain the given item @@ -84,7 +113,9 @@ impl TwoLevelBlockIndex { &self, key: &[u8], cache_policy: CachePolicy, - ) -> crate::Result> { + ) -> crate::Result> { + use super::KeyedBlockIndex; + let Some(index_block_handle) = self .top_level_index .get_last_block_containing_key(key, cache_policy) @@ -93,45 +124,48 @@ impl TwoLevelBlockIndex { return Ok(Some(self.get_last_data_block_handle(cache_policy)?)); }; - let index_block = self.load_index_block(index_block_handle, cache_policy)?; + let index_block = self.load_index_block(index_block_handle.offset, cache_policy)?; - Ok(index_block - .items - .get_last_block_containing_key(key, cache_policy) - .expect("cannot fail") - .cloned()) + Ok({ + use super::KeyedBlockIndex; + + index_block + .items + .get_last_block_containing_key(key, cache_policy) + .expect("cannot fail") + .map(|x| x.offset) + }) } pub fn get_last_data_block_handle( &self, cache_policy: CachePolicy, - ) -> crate::Result { + ) -> crate::Result { + use super::KeyedBlockIndex; + let index_block_handle = self .top_level_index .get_last_block_handle(cache_policy) .expect("cannot fail"); - let index_block = self.load_index_block(index_block_handle, cache_policy)?; + let index_block = self.load_index_block(index_block_handle.offset, cache_policy)?; Ok(index_block .items .last() .expect("index block should not be empty") - .clone()) + .offset) } /// Loads an index block from disk pub fn load_index_block( &self, - block_handle: &KeyedBlockHandle, + offset: BlockOffset, cache_policy: CachePolicy, ) -> crate::Result> { - log::trace!("loading index block {:?}/{block_handle:?}", self.segment_id); + log::trace!("loading index block {:?}/{offset:?}", self.segment_id); - if let Some(block) = self - .index_block_fetcher - .get(self.segment_id, block_handle.offset) - { + if let Some(block) = self.index_block_fetcher.get(self.segment_id, offset) { // Cache hit: Copy from block Ok(block) @@ -145,13 +179,13 @@ impl TwoLevelBlockIndex { let block = IndexBlock::from_file( &mut *file_guard.file.lock().expect("lock is poisoned"), - block_handle.offset, + offset, ) .map_err(|e| { log::error!( "Failed to load index block {:?}/{:?}: {e:?}", self.segment_id, - block_handle.offset + offset ); e })?; @@ -162,11 +196,8 @@ impl TwoLevelBlockIndex { let block = Arc::new(block); if cache_policy == CachePolicy::Write { - self.index_block_fetcher.insert( - self.segment_id, - block_handle.offset, - block.clone(), - ); + self.index_block_fetcher + .insert(self.segment_id, offset, block.clone()); } Ok(block) @@ -187,16 +218,17 @@ impl TwoLevelBlockIndex { } pub fn from_file>( - file_path: P, - offset: BlockOffset, + path: P, + metadata: &Metadata, + tli_ptr: BlockOffset, segment_id: GlobalSegmentId, descriptor_table: Arc, block_cache: Arc, ) -> crate::Result { - let file_path = file_path.as_ref(); + let file_path = path.as_ref(); log::trace!("Reading block index from {file_path:?}"); - let top_level_index = TopLevelIndex::from_file(file_path, offset)?; + let top_level_index = TopLevelIndex::from_file(file_path, metadata, tli_ptr)?; Ok(Self { descriptor_table, diff --git a/src/segment/file_offsets.rs b/src/segment/file_offsets.rs index f25a1910..e96fcc34 100644 --- a/src/segment/file_offsets.rs +++ b/src/segment/file_offsets.rs @@ -7,7 +7,7 @@ use crate::coding::{Decode, DecodeError, Encode, EncodeError}; use byteorder::{BigEndian, ReadBytesExt, WriteBytesExt}; use std::io::{Read, Write}; -#[derive(Debug, Default, PartialEq, Eq)] +#[derive(Copy, Clone, Debug, Default, PartialEq, Eq)] pub struct FileOffsets { pub metadata_ptr: BlockOffset, pub index_block_ptr: BlockOffset, diff --git a/src/segment/forward_reader.rs b/src/segment/forward_reader.rs new file mode 100644 index 00000000..ba203c7c --- /dev/null +++ b/src/segment/forward_reader.rs @@ -0,0 +1,132 @@ +// Copyright (c) 2024-present, fjall-rs +// This source code is licensed under both the Apache 2.0 and MIT License +// (found in the LICENSE-* files in the repository) + +use super::{ + value_block::{BlockOffset, CachePolicy, ValueBlock}, + value_block_consumer::ValueBlockConsumer, +}; +use crate::{ + descriptor_table::FileDescriptorTable, segment::block::header::Header, value::InternalValue, + BlockCache, GlobalSegmentId, +}; + +/// Segment forward reader specialized for point reads +pub struct ForwardReader<'a> { + segment_id: GlobalSegmentId, + + descriptor_table: &'a FileDescriptorTable, + block_cache: &'a BlockCache, + + data_block_boundary: BlockOffset, + + pub lo_block_offset: BlockOffset, + pub(crate) lo_block_size: u64, + pub(crate) lo_block_items: Option, + pub(crate) lo_initialized: bool, + + cache_policy: CachePolicy, +} + +impl<'a> ForwardReader<'a> { + #[must_use] + pub fn new( + data_block_boundary: BlockOffset, + descriptor_table: &'a FileDescriptorTable, + segment_id: GlobalSegmentId, + block_cache: &'a BlockCache, + lo_block_offset: BlockOffset, + ) -> Self { + Self { + descriptor_table, + segment_id, + block_cache, + + data_block_boundary, + + lo_block_offset, + lo_block_size: 0, + lo_block_items: None, + lo_initialized: false, + + cache_policy: CachePolicy::Write, + } + } + + fn load_data_block( + &self, + offset: BlockOffset, + ) -> crate::Result> { + let block = ValueBlock::load_by_block_handle( + self.descriptor_table, + self.block_cache, + self.segment_id, + offset, + self.cache_policy, + )?; + + // Truncate as many items as possible + block.map_or(Ok(None), |block| { + Ok(Some(( + block.header.data_length.into(), + block.header.previous_block_offset, + ValueBlockConsumer::with_bounds(block, None, None), + ))) + }) + } + + fn initialize_lo(&mut self) -> crate::Result<()> { + if let Some((size, _, items)) = self.load_data_block(self.lo_block_offset)? { + self.lo_block_items = Some(items); + self.lo_block_size = size; + } + + self.lo_initialized = true; + + Ok(()) + } +} + +impl<'a> Iterator for ForwardReader<'a> { + type Item = crate::Result; + + fn next(&mut self) -> Option { + if !self.lo_initialized { + fail_iter!(self.initialize_lo()); + } + + if let Some(head) = self.lo_block_items.as_mut()?.next() { + // Just consume item + return Some(Ok(head)); + } + + // Load next block + let next_block_offset = BlockOffset( + *self.lo_block_offset + Header::serialized_len() as u64 + self.lo_block_size, + ); + + if next_block_offset >= self.data_block_boundary { + // We are done + return None; + } + + assert_ne!( + self.lo_block_offset, next_block_offset, + "invalid next block offset", + ); + + match fail_iter!(self.load_data_block(next_block_offset)) { + Some((size, _, items)) => { + self.lo_block_items = Some(items); + self.lo_block_size = size; + self.lo_block_offset = next_block_offset; + + // We just loaded the block + self.lo_block_items.as_mut()?.next().map(Ok) + } + None => { + panic!("searched for invalid data block"); + } + } + } +} diff --git a/src/segment/inner.rs b/src/segment/inner.rs index 805c2f59..01cacdf7 100644 --- a/src/segment/inner.rs +++ b/src/segment/inner.rs @@ -2,9 +2,7 @@ // This source code is licensed under both the Apache 2.0 and MIT License // (found in the LICENSE-* files in the repository) -use super::{ - block_index::two_level_index::TwoLevelBlockIndex, file_offsets::FileOffsets, meta::Metadata, -}; +use super::{block_index::BlockIndexImpl, file_offsets::FileOffsets, meta::Metadata}; use crate::{block_cache::BlockCache, descriptor_table::FileDescriptorTable, tree::inner::TreeId}; use std::sync::Arc; @@ -22,7 +20,7 @@ pub struct Inner { /// Translates key (first item of a block) to block offset (address inside file) and (compressed) size #[doc(hidden)] - pub block_index: Arc, + pub block_index: Arc, /// Block cache /// diff --git a/src/segment/meta/compression.rs b/src/segment/meta/compression.rs index 00413ceb..eac0358a 100644 --- a/src/segment/meta/compression.rs +++ b/src/segment/meta/compression.rs @@ -113,37 +113,37 @@ impl std::fmt::Display for CompressionType { #[cfg(test)] mod tests { use super::*; + use test_log::test; - #[test_log::test] - fn compression_serialize_none() -> crate::Result<()> { + #[test] + fn compression_serialize_none() { let serialized = CompressionType::None.encode_into_vec(); assert_eq!(2, serialized.len()); - Ok(()) } #[cfg(feature = "lz4")] mod lz4 { use super::*; + use test_log::test; - #[test_log::test] - fn compression_serialize_none() -> crate::Result<()> { + #[test] + fn compression_serialize_none() { let serialized = CompressionType::Lz4.encode_into_vec(); assert_eq!(2, serialized.len()); - Ok(()) } } #[cfg(feature = "miniz")] mod miniz { use super::*; + use test_log::test; - #[test_log::test] - fn compression_serialize_none() -> crate::Result<()> { + #[test] + fn compression_serialize_none() { for lvl in 0..10 { let serialized = CompressionType::Miniz(lvl).encode_into_vec(); assert_eq!(2, serialized.len()); } - Ok(()) } } } diff --git a/src/segment/mod.rs b/src/segment/mod.rs index 6df15bde..229fe12a 100644 --- a/src/segment/mod.rs +++ b/src/segment/mod.rs @@ -5,9 +5,9 @@ pub mod block; pub mod block_index; pub mod file_offsets; +mod forward_reader; pub mod id; pub mod inner; -pub mod level_reader; pub mod meta; pub mod multi_writer; pub mod range; @@ -20,18 +20,22 @@ pub mod writer; use crate::{ block_cache::BlockCache, descriptor_table::FileDescriptorTable, - segment::reader::Reader, + time::unix_timestamp, tree::inner::TreeId, value::{InternalValue, SeqNo, UserKey}, }; +use block_index::BlockIndexImpl; +use forward_reader::ForwardReader; use id::GlobalSegmentId; use inner::Inner; +use meta::SegmentId; use range::Range; use std::{ops::Bound, path::Path, sync::Arc}; #[cfg(feature = "bloom")] use crate::bloom::{BloomFilter, CompositeHash}; +#[allow(clippy::module_name_repetitions)] pub type SegmentInner = Inner; /// Disk segment (a.k.a. `SSTable`, `SST`, `sorted string table`) that is located on disk @@ -62,15 +66,42 @@ impl std::ops::Deref for Segment { impl std::fmt::Debug for Segment { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!( - f, - "Segment:{}({})", - self.metadata.id, self.metadata.key_range - ) + write!(f, "Segment:{}({})", self.id(), self.metadata.key_range) } } impl Segment { + // TODO: in Leveled compaction, compact segments that live very long and have + // many versions (possibly unnecessary space usage of old, stale versions) + /// Calculates how many versions per key there are on average. + #[must_use] + pub fn version_factor(&self) -> f32 { + self.metadata.item_count as f32 / self.metadata.key_count as f32 + } + + /// Gets the segment age in nanoseconds. + #[must_use] + pub fn age(&self) -> u128 { + let now = unix_timestamp().as_nanos(); + let created_at = self.metadata.created_at * 1_000; + now.saturating_sub(created_at) + } + + /// Gets the global segment ID. + #[must_use] + pub fn global_id(&self) -> GlobalSegmentId { + (self.tree_id, self.id()).into() + } + + /// Gets the segment ID. + /// + /// The segment ID is unique for this tree, but not + /// across multiple trees, use [`Segment::global_id`] for that. + #[must_use] + pub fn id(&self) -> SegmentId { + self.metadata.id + } + pub(crate) fn verify(&self) -> crate::Result { use block::checksum::Checksum; use block_index::IndexBlock; @@ -81,62 +112,103 @@ impl Segment { let guard = self .descriptor_table - .access(&(self.tree_id, self.metadata.id).into())? + .access(&self.global_id())? .expect("should have gotten file"); let mut file = guard.file.lock().expect("lock is poisoned"); - // NOTE: TODO: because of 1.74.0 - #[allow(clippy::explicit_iter_loop)] - for handle in self.block_index.top_level_index.iter() { - let block = match IndexBlock::from_file(&mut *file, handle.offset) { - Ok(v) => v, - Err(e) => { - log::error!( - "index block {handle:?} could not be loaded, it is probably corrupted: {e:?}" - ); - broken_count += 1; - continue; - } - }; - - for handle in &*block.items { - let value_block = match ValueBlock::from_file(&mut *file, handle.offset) { - Ok(v) => v, - Err(e) => { - log::error!( - "data block {handle:?} could not be loaded, it is probably corrupted: {e:?}" - ); + // TODO: maybe move to BlockIndexImpl::verify + match &*self.block_index { + BlockIndexImpl::Full(block_index) => { + for handle in block_index.iter() { + let value_block = match ValueBlock::from_file(&mut *file, handle.offset) { + Ok(v) => v, + Err(e) => { + log::error!( + "data block {handle:?} could not be loaded, it is probably corrupted: {e:?}" + ); + broken_count += 1; + data_block_count += 1; + continue; + } + }; + + let (_, data) = ValueBlock::to_bytes_compressed( + &value_block.items, + value_block.header.previous_block_offset, + value_block.header.compression, + )?; + let actual_checksum = Checksum::from_bytes(&data); + + if value_block.header.checksum != actual_checksum { + log::error!("{handle:?} is corrupted, invalid checksum value"); broken_count += 1; - data_block_count += 1; - continue; } - }; - - let (_, data) = ValueBlock::to_bytes_compressed( - &value_block.items, - value_block.header.previous_block_offset, - value_block.header.compression, - )?; - let actual_checksum = Checksum::from_bytes(&data); - - if value_block.header.checksum != actual_checksum { - log::error!("{handle:?} is corrupted, invalid checksum value"); - broken_count += 1; + + data_block_count += 1; + + if data_block_count % 1_000 == 0 { + log::debug!("Checked {data_block_count} data blocks"); + } } + } + BlockIndexImpl::TwoLevel(block_index) => { + // NOTE: TODO: because of 1.74.0 + #[allow(clippy::explicit_iter_loop)] + for handle in block_index.top_level_index.iter() { + let block = match IndexBlock::from_file(&mut *file, handle.offset) { + Ok(v) => v, + Err(e) => { + log::error!( + "index block {handle:?} could not be loaded, it is probably corrupted: {e:?}" + ); + broken_count += 1; + continue; + } + }; + + for handle in &*block.items { + let value_block = match ValueBlock::from_file(&mut *file, handle.offset) { + Ok(v) => v, + Err(e) => { + log::error!( + "data block {handle:?} could not be loaded, it is probably corrupted: {e:?}" + ); + broken_count += 1; + data_block_count += 1; + continue; + } + }; + + let (_, data) = ValueBlock::to_bytes_compressed( + &value_block.items, + value_block.header.previous_block_offset, + value_block.header.compression, + )?; + let actual_checksum = Checksum::from_bytes(&data); + + if value_block.header.checksum != actual_checksum { + log::error!("{handle:?} is corrupted, invalid checksum value"); + broken_count += 1; + } - data_block_count += 1; + data_block_count += 1; - if data_block_count % 1_000 == 0 { - log::debug!("Checked {data_block_count} data blocks"); + if data_block_count % 1_000 == 0 { + log::debug!("Checked {data_block_count} data blocks"); + } + } } } } - assert_eq!( - data_block_count, self.metadata.data_block_count, - "not all data blocks were visited" - ); + if data_block_count != self.metadata.data_block_count { + log::error!( + "Not all data blocks were visited during verification of disk segment {:?}", + self.id(), + ); + broken_count += 1; + } Ok(broken_count) } @@ -167,8 +239,9 @@ impl Segment { tree_id: TreeId, block_cache: Arc, descriptor_table: Arc, + use_full_block_index: bool, ) -> crate::Result { - use block_index::two_level_index::TwoLevelBlockIndex; + use block_index::{full_index::FullBlockIndex, two_level_index::TwoLevelBlockIndex}; use trailer::SegmentFileTrailer; let file_path = file_path.as_ref(); @@ -185,13 +258,23 @@ impl Segment { "Creating block index, with tli_ptr={}", trailer.offsets.tli_ptr ); - let block_index = TwoLevelBlockIndex::from_file( - file_path, - trailer.offsets.tli_ptr, - (tree_id, trailer.metadata.id).into(), - descriptor_table.clone(), - block_cache.clone(), - )?; + + let block_index = if use_full_block_index { + let block_index = + FullBlockIndex::from_file(file_path, &trailer.metadata, &trailer.offsets)?; + + BlockIndexImpl::Full(block_index) + } else { + let block_index = TwoLevelBlockIndex::from_file( + file_path, + &trailer.metadata, + trailer.offsets.tli_ptr, + (tree_id, trailer.metadata.id).into(), + descriptor_table.clone(), + block_cache.clone(), + )?; + BlockIndexImpl::TwoLevel(block_index) + }; #[cfg(feature = "bloom")] let bloom_ptr = trailer.offsets.bloom_ptr; @@ -252,7 +335,7 @@ impl Segment { key: K, seqno: Option, ) -> crate::Result> { - use crate::{mvcc_stream::MvccStream, ValueType}; + use block_index::BlockIndex; use value_block::{CachePolicy, ValueBlock}; use value_block_consumer::ValueBlockConsumer; @@ -260,7 +343,7 @@ impl Segment { let Some(first_block_handle) = self .block_index - .get_lowest_data_block_handle_containing_item(key.as_ref(), CachePolicy::Write)? + .get_lowest_block_containing_key(key, CachePolicy::Write)? else { return Ok(None); }; @@ -268,8 +351,8 @@ impl Segment { let Some(block) = ValueBlock::load_by_block_handle( &self.descriptor_table, &self.block_cache, - GlobalSegmentId::from((self.tree_id, self.metadata.id)), - first_block_handle.offset, + self.global_id(), + first_block_handle, CachePolicy::Write, )? else { @@ -282,33 +365,18 @@ impl Segment { // (see explanation for that below) // This only really works because sequence numbers are sorted // in descending order - let Some(latest) = block.get_latest(key.as_ref()) else { - return Ok(None); - }; - - if latest.key.value_type == ValueType::WeakTombstone { - // NOTE: Continue in slow path - } else { - return Ok(Some(latest.clone())); - } + return Ok(block.get_latest(key.as_ref()).cloned()); } - // TODO: it would be nice to have the possibility of using a lifetime'd - // reader, so we don't need to Arc::clone descriptor_table, and block_cache - let mut reader = Reader::new( + let mut reader = ForwardReader::new( self.offsets.index_block_ptr, - self.descriptor_table.clone(), - GlobalSegmentId::from((self.tree_id, self.metadata.id)), - self.block_cache.clone(), - first_block_handle.offset, - None, + &self.descriptor_table, + self.global_id(), + &self.block_cache, + first_block_handle, ); reader.lo_block_size = block.header.data_length.into(); - reader.lo_block_items = Some(ValueBlockConsumer::with_bounds( - block, - &Some(key.into()), // TODO: this may cause a heap alloc - &None, - )); + reader.lo_block_items = Some(ValueBlockConsumer::with_bounds(block, Some(key), None)); reader.lo_initialized = true; // NOTE: For finding a specific seqno, @@ -328,7 +396,7 @@ impl Segment { // unfortunately is in the next block // // Also because of weak tombstones, we may have to look further than the first item we encounter - let reader = reader.filter(|x| { + let mut reader = reader.filter(|x| { match x { Ok(entry) => { // Check for seqno if needed @@ -342,7 +410,7 @@ impl Segment { } }); - let Some(entry) = MvccStream::new(reader).next().transpose()? else { + let Some(entry) = reader.next().transpose()? else { return Ok(None); }; @@ -370,18 +438,18 @@ impl Segment { key: K, seqno: Option, ) -> crate::Result> { + let key = key.as_ref(); + if let Some(seqno) = seqno { if self.metadata.seqnos.0 >= seqno { return Ok(None); } } - if !self.is_key_in_key_range(&key) { + if !self.is_key_in_key_range(key) { return Ok(None); } - let key = key.as_ref(); - #[cfg(feature = "bloom")] if let Some(bf) = &self.bloom_filter { debug_assert!(false, "Use Segment::get_with_hash instead"); @@ -418,7 +486,7 @@ impl Segment { Range::new( self.offsets.index_block_ptr, self.descriptor_table.clone(), - GlobalSegmentId::from((self.tree_id, self.metadata.id)), + self.global_id(), self.block_cache.clone(), self.block_index.clone(), range, diff --git a/src/segment/multi_writer.rs b/src/segment/multi_writer.rs index af53da3a..c2153b33 100644 --- a/src/segment/multi_writer.rs +++ b/src/segment/multi_writer.rs @@ -126,7 +126,7 @@ impl MultiWriter { pub fn write(&mut self, item: InternalValue) -> crate::Result<()> { self.writer.write(item)?; - if *self.writer.meta.file_pos >= self.target_size { + if *self.writer.meta.file_pos >= self.target_size && self.writer.can_rotate() { self.rotate()?; } @@ -144,3 +144,32 @@ impl MultiWriter { Ok(self.results) } } + +#[cfg(test)] +mod tests { + use crate::{AbstractTree, Config}; + use test_log::test; + + // NOTE: Tests that versions of the same key stay + // in the same segment even if it needs to be rotated + // This avoids segments' key ranges overlapping + #[test] + fn segment_multi_writer_same_key_norotate() -> crate::Result<()> { + let folder = tempfile::tempdir()?; + + let tree = Config::new(&folder).open()?; + + tree.insert("a", "a1".repeat(4_000), 0); + tree.insert("a", "a2".repeat(4_000), 1); + tree.insert("a", "a3".repeat(4_000), 2); + tree.insert("a", "a4".repeat(4_000), 3); + tree.insert("a", "a5".repeat(4_000), 4); + tree.flush_active_memtable(0)?; + assert_eq!(1, tree.segment_count()); + + tree.major_compact(1_024, 0)?; + assert_eq!(1, tree.segment_count()); + + Ok(()) + } +} diff --git a/src/segment/range.rs b/src/segment/range.rs index de0b7eff..2c0af968 100644 --- a/src/segment/range.rs +++ b/src/segment/range.rs @@ -2,7 +2,8 @@ // This source code is licensed under both the Apache 2.0 and MIT License // (found in the LICENSE-* files in the repository) -use super::block_index::two_level_index::TwoLevelBlockIndex; +use super::block_index::BlockIndex; +use super::block_index::BlockIndexImpl; use super::id::GlobalSegmentId; use super::reader::Reader; use super::value_block::BlockOffset; @@ -17,9 +18,10 @@ use std::ops::RangeBounds; use std::sync::Arc; pub struct Range { - block_index: Arc, + block_index: Arc, - is_initialized: bool, + lo_initialized: bool, + hi_initialized: bool, pub(crate) range: (Bound, Bound), @@ -32,7 +34,7 @@ impl Range { descriptor_table: Arc, segment_id: GlobalSegmentId, block_cache: Arc, - block_index: Arc, + block_index: Arc, range: (Bound, Bound), ) -> Self { let reader = Reader::new( @@ -45,7 +47,8 @@ impl Range { ); Self { - is_initialized: false, + lo_initialized: false, + hi_initialized: false, block_index, @@ -67,37 +70,42 @@ impl Range { Bound::Included(start) | Bound::Excluded(start) => { if let Some(lower_bound) = self .block_index - .get_lowest_data_block_handle_containing_item(start, CachePolicy::Write)? + .get_lowest_block_containing_key(start, CachePolicy::Write)? { - self.reader.lo_block_offset = lower_bound.offset; + self.reader.lo_block_offset = lower_bound; } Some(start) } }; + if let Some(key) = start_key.cloned() { self.reader.set_lower_bound(key); } + + self.lo_initialized = true; + Ok(()) } fn initialize_hi_bound(&mut self) -> crate::Result<()> { let end_key: Option<&Slice> = match self.range.end_bound() { Bound::Unbounded => { - let upper_bound = self - .block_index - .get_last_data_block_handle(CachePolicy::Write)?; + let upper_bound = self.block_index.get_last_block_handle(CachePolicy::Write)?; - self.reader.hi_block_offset = Some(upper_bound.offset); + self.reader.hi_block_offset = Some(upper_bound); None } Bound::Included(end) | Bound::Excluded(end) => { if let Some(upper_bound) = self .block_index - .get_last_data_block_handle_containing_item(end, CachePolicy::Write)? + .get_last_block_containing_key(end, CachePolicy::Write)? { - self.reader.hi_block_offset = Some(upper_bound.offset); + self.reader.hi_block_offset = Some(upper_bound); + } else { + self.reader.hi_block_offset = + Some(self.block_index.get_last_block_handle(CachePolicy::Write)?); } Some(end) @@ -108,19 +116,7 @@ impl Range { self.reader.set_upper_bound(key); } - Ok(()) - } - - fn initialize(&mut self) -> crate::Result<()> { - // TODO: can we skip searching for lower bound until next is called at least once...? - // would make short ranges 1.5-2x faster (if cache miss) if only one direction is used - self.initialize_lo_bound()?; - - // TODO: can we skip searching for upper bound until next_back is called at least once...? - // would make short ranges 1.5-2x faster (if cache miss) if only one direction is used - self.initialize_hi_bound()?; - - self.is_initialized = true; + self.hi_initialized = true; Ok(()) } @@ -130,8 +126,8 @@ impl Iterator for Range { type Item = crate::Result; fn next(&mut self) -> Option { - if !self.is_initialized { - if let Err(e) = self.initialize() { + if !self.lo_initialized { + if let Err(e) = self.initialize_lo_bound() { return Some(Err(e)); }; } @@ -183,16 +179,14 @@ impl Iterator for Range { impl DoubleEndedIterator for Range { fn next_back(&mut self) -> Option { - if !self.is_initialized { - if let Err(e) = self.initialize() { + if !self.hi_initialized { + if let Err(e) = self.initialize_hi_bound() { return Some(Err(e)); }; } loop { - let entry_result = self.reader.next_back()?; - - match entry_result { + match self.reader.next_back()? { Ok(entry) => { match self.range.start_bound() { Bound::Included(start) => { @@ -241,7 +235,7 @@ mod tests { block_cache::BlockCache, descriptor_table::FileDescriptorTable, segment::{ - block_index::two_level_index::TwoLevelBlockIndex, + block_index::{two_level_index::TwoLevelBlockIndex, BlockIndexImpl}, range::Range, writer::{Options, Writer}, }, @@ -292,13 +286,15 @@ mod tests { table.insert(&segment_file_path, (0, 0).into()); let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); - let block_index = Arc::new(TwoLevelBlockIndex::from_file( + let block_index = TwoLevelBlockIndex::from_file( segment_file_path, + &trailer.metadata, trailer.offsets.tli_ptr, (0, 0).into(), table.clone(), block_cache.clone(), - )?); + )?; + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); let iter = Range::new( trailer.offsets.index_block_ptr, @@ -390,13 +386,15 @@ mod tests { table.insert(&segment_file_path, (0, 0).into()); let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); - let block_index = Arc::new(TwoLevelBlockIndex::from_file( + let block_index = TwoLevelBlockIndex::from_file( segment_file_path, + &trailer.metadata, trailer.offsets.tli_ptr, (0, 0).into(), table.clone(), block_cache.clone(), - )?); + )?; + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); { let mut iter = Range::new( @@ -589,13 +587,15 @@ mod tests { table.insert(&segment_file_path, (0, 0).into()); let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); - let block_index = Arc::new(TwoLevelBlockIndex::from_file( + let block_index = TwoLevelBlockIndex::from_file( segment_file_path, + &trailer.metadata, trailer.offsets.tli_ptr, (0, 0).into(), table.clone(), block_cache.clone(), - )?); + )?; + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); let ranges: Vec<(Bound, Bound)> = vec![ range_bounds_to_tuple(&(0..1_000)), @@ -691,13 +691,15 @@ mod tests { table.insert(&segment_file_path, (0, 0).into()); let block_cache = Arc::new(BlockCache::with_capacity_bytes(10 * 1_024 * 1_024)); - let block_index = Arc::new(TwoLevelBlockIndex::from_file( + let block_index = TwoLevelBlockIndex::from_file( segment_file_path, + &trailer.metadata, trailer.offsets.tli_ptr, (0, 0).into(), table.clone(), block_cache.clone(), - )?); + )?; + let block_index = Arc::new(BlockIndexImpl::TwoLevel(block_index)); for (i, &start_char) in chars.iter().enumerate() { for &end_char in chars.iter().skip(i + 1) { diff --git a/src/segment/reader.rs b/src/segment/reader.rs index f26bd687..932fabb6 100644 --- a/src/segment/reader.rs +++ b/src/segment/reader.rs @@ -13,8 +13,9 @@ use crate::{ use std::sync::Arc; pub struct Reader { - descriptor_table: Arc, segment_id: GlobalSegmentId, + + descriptor_table: Arc, block_cache: Arc, data_block_boundary: BlockOffset, @@ -103,7 +104,11 @@ impl Reader { Ok(Some(( block.header.data_length.into(), block.header.previous_block_offset, - ValueBlockConsumer::with_bounds(block, &self.start_key, &self.end_key), + ValueBlockConsumer::with_bounds( + block, + self.start_key.as_deref(), + self.end_key.as_deref(), + ), ))) }) } diff --git a/src/segment/value_block.rs b/src/segment/value_block.rs index 48832716..40c923dc 100644 --- a/src/segment/value_block.rs +++ b/src/segment/value_block.rs @@ -80,7 +80,12 @@ impl ValueBlock { let file_guard = descriptor_table .access(&segment_id)? + .ok_or(()) + .map_err(|()| { + log::error!("Failed to get file guard for segment {segment_id:?}"); + }) .expect("should acquire file handle"); + // TODO: ^ use inspect instead: 1.76 let block = Self::from_file( &mut *file_guard.file.lock().expect("lock is poisoned"), diff --git a/src/segment/value_block_consumer.rs b/src/segment/value_block_consumer.rs index 5e70efd5..0e399d28 100644 --- a/src/segment/value_block_consumer.rs +++ b/src/segment/value_block_consumer.rs @@ -3,7 +3,7 @@ // (found in the LICENSE-* files in the repository) use super::value_block::ValueBlock; -use crate::{value::InternalValue, UserKey}; +use crate::value::InternalValue; use std::sync::Arc; pub struct ValueBlockConsumer { @@ -15,14 +15,14 @@ pub struct ValueBlockConsumer { impl ValueBlockConsumer { #[must_use] pub fn new(inner: Arc) -> Self { - Self::with_bounds(inner, &None, &None) + Self::with_bounds(inner, None, None) } #[must_use] pub fn with_bounds( inner: Arc, - start_key: &Option, - end_key: &Option, + start_key: Option<&[u8]>, + end_key: Option<&[u8]>, ) -> Self { let mut lo = start_key.as_ref().map_or(0, |key| { inner.items.partition_point(|x| &*x.key.user_key < *key) @@ -90,12 +90,9 @@ impl DoubleEndedIterator for ValueBlockConsumer { #[allow(clippy::expect_used)] mod tests { use super::*; - use crate::{ - segment::{ - block::{checksum::Checksum, header::Header}, - value_block::BlockOffset, - }, - Slice, + use crate::segment::{ + block::{checksum::Checksum, header::Header}, + value_block::BlockOffset, }; use test_log::test; @@ -227,15 +224,13 @@ mod tests { InternalValue::from_components(*b"e", vec![], 0, crate::ValueType::Value), ]); - let mut iter = - ValueBlockConsumer::with_bounds(block.clone().into(), &Some(Slice::from(*b"c")), &None); + let mut iter = ValueBlockConsumer::with_bounds(block.clone().into(), Some(b"c"), None); assert_eq!(*b"c", &*iter.next().expect("should exist").key.user_key); assert_eq!(*b"d", &*iter.next().expect("should exist").key.user_key); assert_eq!(*b"e", &*iter.next().expect("should exist").key.user_key); iter_closed!(iter); - let mut iter = - ValueBlockConsumer::with_bounds(block.into(), &Some(Slice::from(*b"c")), &None); + let mut iter = ValueBlockConsumer::with_bounds(block.into(), Some(b"c"), None); assert_eq!( *b"e", &*iter.next_back().expect("should exist").key.user_key @@ -261,15 +256,13 @@ mod tests { InternalValue::from_components(*b"e", vec![], 0, crate::ValueType::Value), ]); - let mut iter = - ValueBlockConsumer::with_bounds(block.clone().into(), &None, &Some(Slice::from(*b"c"))); + let mut iter = ValueBlockConsumer::with_bounds(block.clone().into(), None, Some(b"c")); assert_eq!(*b"a", &*iter.next().expect("should exist").key.user_key); assert_eq!(*b"b", &*iter.next().expect("should exist").key.user_key); assert_eq!(*b"c", &*iter.next().expect("should exist").key.user_key); iter_closed!(iter); - let mut iter = - ValueBlockConsumer::with_bounds(block.into(), &None, &Some(Slice::from(*b"c"))); + let mut iter = ValueBlockConsumer::with_bounds(block.into(), None, Some(b"c")); assert_eq!( *b"c", &*iter.next_back().expect("should exist").key.user_key @@ -294,12 +287,10 @@ mod tests { InternalValue::from_components(*b"e", vec![], 0, crate::ValueType::Value), ]); - let mut iter = - ValueBlockConsumer::with_bounds(block.clone().into(), &None, &Some(Slice::from(*b"a"))); + let mut iter = ValueBlockConsumer::with_bounds(block.clone().into(), None, Some(b"a")); iter_closed!(iter); - let mut iter = - ValueBlockConsumer::with_bounds(block.into(), &None, &Some(Slice::from(*b"a"))).rev(); + let mut iter = ValueBlockConsumer::with_bounds(block.into(), None, Some(b"a")).rev(); iter_closed!(iter); } @@ -313,12 +304,10 @@ mod tests { InternalValue::from_components(*b"e", vec![], 0, crate::ValueType::Value), ]); - let mut iter = - ValueBlockConsumer::with_bounds(block.clone().into(), &Some(Slice::from(*b"f")), &None); + let mut iter = ValueBlockConsumer::with_bounds(block.clone().into(), Some(b"f"), None); iter_closed!(iter); - let mut iter = - ValueBlockConsumer::with_bounds(block.into(), &Some(Slice::from(*b"f")), &None).rev(); + let mut iter = ValueBlockConsumer::with_bounds(block.into(), Some(b"f"), None).rev(); iter_closed!(iter); } } diff --git a/src/segment/writer/mod.rs b/src/segment/writer/mod.rs index 6813791e..eb69a1ce 100644 --- a/src/segment/writer/mod.rs +++ b/src/segment/writer/mod.rs @@ -55,6 +55,8 @@ pub struct Writer { current_key: Option, + can_rotate: bool, + #[cfg(feature = "bloom")] bloom_policy: BloomConstructionPolicy, @@ -134,6 +136,8 @@ impl Writer { current_key: None, + can_rotate: false, + #[cfg(feature = "bloom")] bloom_policy: BloomConstructionPolicy::default(), @@ -142,6 +146,11 @@ impl Writer { }) } + #[must_use] + pub fn can_rotate(&self) -> bool { + self.can_rotate + } + #[must_use] pub(crate) fn use_compression(mut self, compression: CompressionType) -> Self { self.compression = compression; @@ -224,7 +233,13 @@ impl Writer { self.meta.tombstone_count += 1; } + // NOTE: Check if we visit a new key if Some(&item.key.user_key) != self.current_key.as_ref() { + // IMPORTANT: Check that we are not at the first key + if self.current_key.is_some() { + self.can_rotate = true; + } + self.meta.key_count += 1; self.current_key = Some(item.key.user_key.clone()); @@ -241,7 +256,7 @@ impl Writer { let seqno = item.key.seqno; if self.meta.first_key.is_none() { - self.meta.first_key = Some(item.key.clone().user_key); + self.meta.first_key = Some(item.key.user_key.clone()); } self.chunk_size += item.size(); @@ -251,13 +266,8 @@ impl Writer { self.spill_block()?; } - if self.meta.lowest_seqno > seqno { - self.meta.lowest_seqno = seqno; - } - - if self.meta.highest_seqno < seqno { - self.meta.highest_seqno = seqno; - } + self.meta.lowest_seqno = self.meta.lowest_seqno.min(seqno); + self.meta.highest_seqno = self.meta.highest_seqno.max(seqno); Ok(()) } @@ -376,6 +386,52 @@ mod tests { use std::sync::Arc; use test_log::test; + #[test] + fn segment_writer_seqnos() -> crate::Result<()> { + let folder = tempfile::tempdir()?.into_path(); + + let segment_id = 532; + + let mut writer = Writer::new(Options { + folder, + data_block_size: 4_096, + index_block_size: 4_096, + segment_id, + })?; + + writer.write(InternalValue::from_components( + "a", + nanoid::nanoid!().as_bytes(), + 7, + ValueType::Value, + ))?; + writer.write(InternalValue::from_components( + "b", + nanoid::nanoid!().as_bytes(), + 5, + ValueType::Value, + ))?; + writer.write(InternalValue::from_components( + "c", + nanoid::nanoid!().as_bytes(), + 8, + ValueType::Value, + ))?; + writer.write(InternalValue::from_components( + "d", + nanoid::nanoid!().as_bytes(), + 10, + ValueType::Value, + ))?; + + let trailer = writer.finish()?.expect("should exist"); + + assert_eq!(5, trailer.metadata.seqnos.0); + assert_eq!(10, trailer.metadata.seqnos.1); + + Ok(()) + } + #[test] #[cfg(feature = "bloom")] fn segment_writer_zero_bpk() -> crate::Result<()> { @@ -457,7 +513,12 @@ mod tests { // the TLI length fits into u32 as well #[allow(clippy::cast_possible_truncation)] { - let tli = TopLevelIndex::from_file(&segment_file_path, trailer.offsets.tli_ptr)?; + let tli = TopLevelIndex::from_file( + &segment_file_path, + &trailer.metadata, + trailer.offsets.tli_ptr, + )?; + assert_eq!(tli.len() as u32, trailer.metadata.index_block_count); } diff --git a/src/tree/mod.rs b/src/tree/mod.rs index d281e396..8916bb43 100644 --- a/src/tree/mod.rs +++ b/src/tree/mod.rs @@ -14,7 +14,9 @@ use crate::{ memtable::Memtable, range::{prefix_to_range, MemtableLockGuard, TreeIter}, segment::{ - block_index::two_level_index::TwoLevelBlockIndex, meta::TableType, Segment, SegmentInner, + block_index::{full_index::FullBlockIndex, BlockIndexImpl}, + meta::TableType, + Segment, SegmentInner, }, stop_signal::StopSignal, value::InternalValue, @@ -163,11 +165,8 @@ impl AbstractTree for Tree { use crate::segment::writer::BloomConstructionPolicy; if self.config.bloom_bits_per_key >= 0 { - segment_writer = segment_writer.use_bloom_policy( - // TODO: increase to 0.00001 when https://github.com/fjall-rs/lsm-tree/issues/63 - // is fixed - BloomConstructionPolicy::FpRate(0.0001), - ); + segment_writer = + segment_writer.use_bloom_policy(BloomConstructionPolicy::FpRate(0.00001)); } else { segment_writer = segment_writer.use_bloom_policy(BloomConstructionPolicy::BitsPerKey(0)); @@ -202,9 +201,11 @@ impl AbstractTree for Tree { } })?; + // eprintln!("{original_levels}"); + for segment in segments { - log::trace!("releasing sealed memtable {}", segment.metadata.id); - sealed_memtables.remove(segment.metadata.id); + log::trace!("releasing sealed memtable {}", segment.id()); + sealed_memtables.remove(segment.id()); } Ok(()) @@ -354,13 +355,11 @@ impl AbstractTree for Tree { key: K, seqno: SeqNo, ) -> crate::Result> { - Ok(self - .get_internal_entry(key, true, Some(seqno))? - .map(|x| x.value)) + Ok(self.get_internal_entry(key, Some(seqno))?.map(|x| x.value)) } fn get>(&self, key: K) -> crate::Result> { - Ok(self.get_internal_entry(key, true, None)?.map(|x| x.value)) + Ok(self.get_internal_entry(key, None)?.map(|x| x.value)) } fn iter_with_seqno( @@ -403,31 +402,23 @@ impl AbstractTree for Tree { Box::new(self.create_prefix(prefix, None, None)) } - fn insert, V: AsRef<[u8]>>(&self, key: K, value: V, seqno: SeqNo) -> (u32, u32) { - let value = - InternalValue::from_components(key.as_ref(), value.as_ref(), seqno, ValueType::Value); - self.append_entry(value) - } - - fn raw_insert_with_lock, V: AsRef<[u8]>>( + fn insert, V: Into>( &self, - lock: &RwLockWriteGuard<'_, Memtable>, key: K, value: V, seqno: SeqNo, - r#type: ValueType, ) -> (u32, u32) { - let value = InternalValue::from_components(key.as_ref(), value.as_ref(), seqno, r#type); - lock.insert(value) + let value = InternalValue::from_components(key, value, seqno, ValueType::Value); + self.append_entry(value) } - fn remove>(&self, key: K, seqno: SeqNo) -> (u32, u32) { - let value = InternalValue::new_tombstone(key.as_ref(), seqno); + fn remove>(&self, key: K, seqno: SeqNo) -> (u32, u32) { + let value = InternalValue::new_tombstone(key, seqno); self.append_entry(value) } - fn remove_weak>(&self, key: K, seqno: SeqNo) -> (u32, u32) { - let value = InternalValue::new_weak_tombstone(key.as_ref(), seqno); + fn remove_weak>(&self, key: K, seqno: SeqNo) -> (u32, u32) { + let value = InternalValue::new_weak_tombstone(key, seqno); self.append_entry(value) } } @@ -500,16 +491,9 @@ impl Tree { log::debug!("Finalized segment write at {segment_folder:?}"); - let block_index = Arc::new(TwoLevelBlockIndex::from_file( - &segment_file_path, - trailer.offsets.tli_ptr, - (self.id, segment_id).into(), - self.config.descriptor_table.clone(), - self.config.block_cache.clone(), - )?); - - #[cfg(feature = "bloom")] - let bloom_ptr = trailer.offsets.bloom_ptr; + let block_index = + FullBlockIndex::from_file(&segment_file_path, &trailer.metadata, &trailer.offsets)?; + let block_index = Arc::new(BlockIndexImpl::Full(block_index)); let created_segment: Segment = SegmentInner { tree_id: self.id, @@ -522,14 +506,13 @@ impl Tree { block_cache: self.config.block_cache.clone(), #[cfg(feature = "bloom")] - bloom_filter: Segment::load_bloom(&segment_file_path, bloom_ptr)?, + bloom_filter: Segment::load_bloom(&segment_file_path, trailer.offsets.bloom_ptr)?, } .into(); - self.config.descriptor_table.insert( - segment_file_path, - (self.id, created_segment.metadata.id).into(), - ); + self.config + .descriptor_table + .insert(segment_file_path, created_segment.global_id()); log::debug!("Flushed segment to {segment_folder:?}"); @@ -581,25 +564,18 @@ impl Tree { &self, memtable_lock: &RwLockWriteGuard<'_, Memtable>, key: K, - evict_tombstone: bool, seqno: Option, ) -> crate::Result> { if let Some(entry) = memtable_lock.get(&key, seqno) { - if evict_tombstone { - return Ok(ignore_tombstone_value(entry)); - } - return Ok(Some(entry)); + return Ok(ignore_tombstone_value(entry)); }; // Now look in sealed memtables if let Some(entry) = self.get_internal_entry_from_sealed_memtables(&key, seqno) { - if evict_tombstone { - return Ok(ignore_tombstone_value(entry)); - } - return Ok(Some(entry)); + return Ok(ignore_tombstone_value(entry)); } - self.get_internal_entry_from_segments(key, evict_tombstone, seqno) + self.get_internal_entry_from_segments(key, seqno) } fn get_internal_entry_from_sealed_memtables>( @@ -621,7 +597,6 @@ impl Tree { fn get_internal_entry_from_segments>( &self, key: K, - evict_tombstone: bool, // TODO: remove?, just always true seqno: Option, ) -> crate::Result> { // NOTE: Create key hash for hash sharing @@ -651,10 +626,7 @@ impl Tree { let maybe_item = segment.get_with_hash(&key, seqno, key_hash)?; if let Some(item) = maybe_item { - if evict_tombstone { - return Ok(ignore_tombstone_value(item)); - } - return Ok(Some(item)); + return Ok(ignore_tombstone_value(item)); } } @@ -671,10 +643,7 @@ impl Tree { let maybe_item = segment.get_with_hash(&key, seqno, key_hash)?; if let Some(item) = maybe_item { - if evict_tombstone { - return Ok(ignore_tombstone_value(item)); - } - return Ok(Some(item)); + return Ok(ignore_tombstone_value(item)); } } } @@ -686,7 +655,6 @@ impl Tree { pub fn get_internal_entry>( &self, key: K, - evict_tombstone: bool, // TODO: remove?, just always true seqno: Option, ) -> crate::Result> { // TODO: consolidate memtable & sealed behind single RwLock @@ -694,23 +662,18 @@ impl Tree { let memtable_lock = self.active_memtable.read().expect("lock is poisoned"); if let Some(entry) = memtable_lock.get(&key, seqno) { - if evict_tombstone { - return Ok(ignore_tombstone_value(entry)); - } - return Ok(Some(entry)); + return Ok(ignore_tombstone_value(entry)); }; + drop(memtable_lock); // Now look in sealed memtables if let Some(entry) = self.get_internal_entry_from_sealed_memtables(&key, seqno) { - if evict_tombstone { - return Ok(ignore_tombstone_value(entry)); - } - return Ok(Some(entry)); + return Ok(ignore_tombstone_value(entry)); } // Now look in segments... this may involve disk I/O - self.get_internal_entry_from_segments(key, evict_tombstone, seqno) + self.get_internal_entry_from_segments(key, seqno) } #[doc(hidden)] @@ -835,13 +798,9 @@ impl Tree { &config.block_cache, &config.descriptor_table, )?; - levels.sort_levels(); + levels.update_metadata(); - let highest_segment_id = levels - .iter() - .map(|x| x.metadata.id) - .max() - .unwrap_or_default(); + let highest_segment_id = levels.iter().map(Segment::id).max().unwrap_or_default(); let inner = TreeInner { id: tree_id, @@ -907,12 +866,11 @@ impl Tree { let tree_path = tree_path.as_ref(); - log::info!("Recovering LSM-tree at {tree_path:?}"); - let level_manifest_path = tree_path.join(LEVELS_MANIFEST_FILE); + log::info!("Recovering manifest at {level_manifest_path:?}"); - let segment_ids_to_recover = LevelManifest::recover_ids(&level_manifest_path)?; - let cnt = segment_ids_to_recover.len(); + let segment_id_map = LevelManifest::recover_ids(&level_manifest_path)?; + let cnt = segment_id_map.len(); log::debug!("Recovering {cnt} disk segments from {tree_path:?}"); @@ -957,15 +915,16 @@ impl Tree { crate::Error::Unrecoverable })?; - if segment_ids_to_recover.contains(&segment_id) { + if let Some(&level_idx) = segment_id_map.get(&segment_id) { let segment = Segment::recover( &segment_file_path, tree_id, block_cache.clone(), descriptor_table.clone(), + level_idx == 0 || level_idx == 1, )?; - descriptor_table.insert(&segment_file_path, (tree_id, segment.metadata.id).into()); + descriptor_table.insert(&segment_file_path, segment.global_id()); segments.push(segment); log::debug!("Recovered segment from {segment_file_path:?}"); @@ -979,8 +938,11 @@ impl Tree { } } - if segments.len() < segment_ids_to_recover.len() { - log::error!("Recovered less segments than expected: {segment_ids_to_recover:?}"); + if segments.len() < cnt { + log::error!( + "Recovered less segments than expected: {:?}", + segment_id_map.keys(), + ); return Err(crate::Error::Unrecoverable); } diff --git a/tests/blob_drop_after_flush.rs b/tests/blob_drop_after_flush.rs index 31e820a4..8b0b74c8 100644 --- a/tests/blob_drop_after_flush.rs +++ b/tests/blob_drop_after_flush.rs @@ -1,7 +1,8 @@ use lsm_tree::{AbstractTree, Config}; use std::time::Duration; +use test_log::test; -#[test_log::test] +#[test] fn blob_drop_after_flush() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; diff --git a/tests/blob_gc.rs b/tests/blob_gc.rs index 8436f9c7..2366da18 100644 --- a/tests/blob_gc.rs +++ b/tests/blob_gc.rs @@ -1,6 +1,7 @@ use lsm_tree::{AbstractTree, Config, SequenceNumberCounter}; +use test_log::test; -#[test_log::test] +#[test] fn blob_gc_1() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; @@ -43,7 +44,7 @@ fn blob_gc_1() -> lsm_tree::Result<()> { Ok(()) } -#[test_log::test] +#[test] fn blob_gc_2() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; @@ -92,7 +93,7 @@ fn blob_gc_2() -> lsm_tree::Result<()> { Ok(()) } -#[test_log::test] +#[test] fn blob_gc_3() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; diff --git a/tests/blob_gc_watermark.rs b/tests/blob_gc_watermark.rs index 721d10e4..8089e8ea 100644 --- a/tests/blob_gc_watermark.rs +++ b/tests/blob_gc_watermark.rs @@ -1,6 +1,7 @@ use lsm_tree::{AbstractTree, Config, SequenceNumberCounter}; +use test_log::test; -#[test_log::test] +#[test] fn blob_gc_seqno_watermark() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; diff --git a/tests/blob_tree_flush.rs b/tests/blob_tree_flush.rs index 1041b106..ff43396c 100644 --- a/tests/blob_tree_flush.rs +++ b/tests/blob_tree_flush.rs @@ -1,6 +1,7 @@ use lsm_tree::{AbstractTree, Config, SequenceNumberCounter}; +use test_log::test; -#[test_log::test] +#[test] fn blob_gc_flush_tombstone() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; diff --git a/tests/major_compaction.rs b/tests/major_compaction.rs index 9976002f..f5f92bf5 100644 --- a/tests/major_compaction.rs +++ b/tests/major_compaction.rs @@ -20,17 +20,17 @@ fn tree_major_compaction() -> lsm_tree::Result<()> { tree.major_compact(u64::MAX, 1_000 /* NOTE: Simulate some time passing */)?; assert_eq!(1, tree.segment_count()); - let item = tree.get_internal_entry("a", true, None)?.unwrap(); + let item = tree.get_internal_entry("a", None)?.unwrap(); assert_eq!(&*item.key.user_key, "a".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 0); - let item = tree.get_internal_entry("b", true, None)?.unwrap(); + let item = tree.get_internal_entry("b", None)?.unwrap(); assert_eq!(&*item.key.user_key, "b".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 1); - let item = tree.get_internal_entry("c", true, None)?.unwrap(); + let item = tree.get_internal_entry("c", None)?.unwrap(); assert_eq!(&*item.key.user_key, "c".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 2); diff --git a/tests/multi_trees.rs b/tests/multi_trees.rs index 67f22e9a..ae4ddd8b 100644 --- a/tests/multi_trees.rs +++ b/tests/multi_trees.rs @@ -1,4 +1,5 @@ use lsm_tree::{AbstractTree, Config}; +use test_log::test; #[test] fn tree_multi_segment_ids() -> lsm_tree::Result<()> { diff --git a/tests/segment_point_reads.rs b/tests/segment_point_reads.rs index 5dd39b03..107e5a95 100644 --- a/tests/segment_point_reads.rs +++ b/tests/segment_point_reads.rs @@ -47,7 +47,7 @@ fn segment_point_reads_mvcc() -> lsm_tree::Result<()> { for x in 0..ITEM_COUNT as u64 { let key = x.to_be_bytes(); - let item = tree.get_internal_entry(key, true, None)?.unwrap(); + let item = tree.get_internal_entry(key, None)?.unwrap(); assert_eq!(item.key.seqno, 2); assert_eq!(&*item.value, b"2"); @@ -89,7 +89,7 @@ fn segment_point_reads_mvcc_slab() -> lsm_tree::Result<()> { tree.flush_active_memtable(0)?; for key in &keys { - let item = tree.get_internal_entry(key, true, None)?.unwrap(); + let item = tree.get_internal_entry(key, None)?.unwrap(); assert_eq!(item.key.seqno, ITEM_COUNT as u64 - 1); } diff --git a/tests/segment_range.rs b/tests/segment_range.rs index 6acbba00..3fa6b3ff 100644 --- a/tests/segment_range.rs +++ b/tests/segment_range.rs @@ -48,3 +48,83 @@ fn segment_ranges() -> lsm_tree::Result<()> { Ok(()) } + +#[test] +fn segment_range_last_back() -> lsm_tree::Result<()> { + let folder = tempfile::tempdir()?.into_path(); + + let tree = Config::new(folder) + .data_block_size(1_024) + .index_block_size(1_024) + .open()?; + + let value = (0..2_000).map(|_| 0).collect::>(); + + for x in 0..10_u64 { + let key = x.to_be_bytes(); + tree.insert(key, &value, 0); + } + tree.flush_active_memtable(0)?; + + let iter = tree.range(0u64.to_be_bytes()..10u64.to_be_bytes()); + assert_eq!(10, iter.count()); + + let iter = tree.range(0u64.to_be_bytes()..10u64.to_be_bytes()); + assert_eq!(10, iter.rev().count()); + + let mut iter = tree.range(0u64.to_be_bytes()..5u64.to_be_bytes()); + + assert_eq!(0u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(1u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(2u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(3u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(4u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert!(iter.next_back().is_none()); + + Ok(()) +} + +#[test] +fn segment_range_last_back_2() -> lsm_tree::Result<()> { + let folder = tempfile::tempdir()?.into_path(); + + let tree = Config::new(folder) + .data_block_size(1_024) + .index_block_size(1_024) + .open()?; + + let value = (0..2_000).map(|_| 0).collect::>(); + + for x in 0..10_u64 { + let key = x.to_be_bytes(); + tree.insert(key, &value, 0); + } + tree.insert(10u64.to_be_bytes(), [], 0); + tree.insert(11u64.to_be_bytes(), [], 0); + tree.flush_active_memtable(0)?; + + let iter = tree.range(0u64.to_be_bytes()..10u64.to_be_bytes()); + assert_eq!(10, iter.count()); + + let iter = tree.range(0u64.to_be_bytes()..10u64.to_be_bytes()); + assert_eq!(10, iter.rev().count()); + + let mut iter = tree.range(0u64.to_be_bytes()..12u64.to_be_bytes()); + + assert_eq!(0u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(1u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(2u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(3u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(4u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(5u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(6u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(7u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(8u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(9u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(10u64.to_be_bytes(), &*iter.next().unwrap().unwrap().0); + assert_eq!(11u64.to_be_bytes(), &*iter.next_back().unwrap().unwrap().0); + assert!(iter.next().is_none()); + assert!(iter.next_back().is_none()); + + Ok(()) +} diff --git a/tests/segment_remove_weak.rs b/tests/segment_remove_weak.rs new file mode 100644 index 00000000..614de031 --- /dev/null +++ b/tests/segment_remove_weak.rs @@ -0,0 +1,22 @@ +use lsm_tree::{AbstractTree, Config}; +use test_log::test; + +#[test] +fn segment_remove_weak_simple() -> lsm_tree::Result<()> { + let folder = tempfile::tempdir()?.into_path(); + + let tree = Config::new(folder) + .data_block_size(1_024) + .index_block_size(1_024) + .open()?; + + tree.insert("a", "a", 0); + tree.insert("a", "b", 1); + tree.remove_weak("a", 2); + + tree.flush_active_memtable(0)?; + + assert!(tree.get("a")?.is_none()); + + Ok(()) +} diff --git a/tests/snapshot_compact.rs b/tests/snapshot_compact.rs index 90f6e0d3..fae1878d 100644 --- a/tests/snapshot_compact.rs +++ b/tests/snapshot_compact.rs @@ -1,8 +1,9 @@ use lsm_tree::{AbstractTree, Config, SequenceNumberCounter}; +use test_log::test; const ITEM_COUNT: usize = 100; -#[test_log::test] +#[test] fn snapshot_after_compaction() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; diff --git a/tests/snapshot_len.rs b/tests/snapshot_len.rs index e01c35e7..4f576220 100644 --- a/tests/snapshot_len.rs +++ b/tests/snapshot_len.rs @@ -1,8 +1,9 @@ use lsm_tree::{AbstractTree, Config, SequenceNumberCounter}; +use test_log::test; const ITEM_COUNT: usize = 100; -#[test_log::test] +#[test] fn snapshot_basic() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; diff --git a/tests/snapshot_point_read.rs b/tests/snapshot_point_read.rs index f459c2a9..87f1cf32 100644 --- a/tests/snapshot_point_read.rs +++ b/tests/snapshot_point_read.rs @@ -60,7 +60,7 @@ fn snapshot_lots_of_versions() -> lsm_tree::Result<()> { for seqno in 1..version_count { let item = tree - .get_internal_entry(key, true, Some(seqno))? + .get_internal_entry(key, Some(seqno))? .expect("should exist"); assert_eq!(format!("abc{}", version_count).as_bytes(), &*item.value); diff --git a/tests/tree_different_block_size.rs b/tests/tree_different_block_size.rs index f0073133..387cee7e 100644 --- a/tests/tree_different_block_size.rs +++ b/tests/tree_different_block_size.rs @@ -1,8 +1,9 @@ use lsm_tree::{AbstractTree, Config, SequenceNumberCounter}; +use test_log::test; const ITEM_COUNT: usize = 1_000; -#[test_log::test] +#[test] fn tree_block_size_after_recovery() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?; diff --git a/tests/tree_disjoint_iter.rs b/tests/tree_disjoint_iter.rs index 4bc9d42e..407552c1 100644 --- a/tests/tree_disjoint_iter.rs +++ b/tests/tree_disjoint_iter.rs @@ -1,4 +1,5 @@ use lsm_tree::{AbstractTree, Config, Slice}; +use test_log::test; macro_rules! iter_closed { ($iter:expr) => { @@ -10,7 +11,7 @@ macro_rules! iter_closed { }; } -#[test_log::test] +#[test] fn tree_disjoint_iter() -> lsm_tree::Result<()> { let tempdir = tempfile::tempdir()?; let tree = crate::Config::new(&tempdir).open()?; diff --git a/tests/tree_flush_eviction.rs b/tests/tree_flush_eviction.rs index d756dcc0..2e78e5f0 100644 --- a/tests/tree_flush_eviction.rs +++ b/tests/tree_flush_eviction.rs @@ -1,6 +1,5 @@ -use std::sync::Arc; - use lsm_tree::AbstractTree; +use std::sync::Arc; use test_log::test; #[test] diff --git a/tests/tree_iter_lifetime.rs b/tests/tree_iter_lifetime.rs index 9fc86139..8c8212e7 100644 --- a/tests/tree_iter_lifetime.rs +++ b/tests/tree_iter_lifetime.rs @@ -1,5 +1,6 @@ use lsm_tree::{AbstractTree, KvPair}; use std::path::Path; +use test_log::test; fn iterrr( path: &Path, @@ -14,7 +15,7 @@ fn iterrr( Ok(tree.iter()) } -#[test_log::test] +#[test] fn tree_iter_lifetime() -> lsm_tree::Result<()> { let folder = tempfile::tempdir().unwrap(); assert_eq!(100, iterrr(folder.path())?.count()); diff --git a/tests/tree_mvcc_simple.rs b/tests/tree_mvcc_simple.rs index 674d7233..1d3e2456 100644 --- a/tests/tree_mvcc_simple.rs +++ b/tests/tree_mvcc_simple.rs @@ -1,6 +1,7 @@ use lsm_tree::{AbstractTree, Config}; +use test_log::test; -#[test_log::test] +#[test] fn tree_read_mvcc() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?.into_path(); diff --git a/tests/tree_recover_counter.rs b/tests/tree_recover_counter.rs index 16f69cc8..464fcaf3 100644 --- a/tests/tree_recover_counter.rs +++ b/tests/tree_recover_counter.rs @@ -1,4 +1,5 @@ use lsm_tree::{AbstractTree, Config}; +use test_log::test; #[test] fn tree_recover_segment_counter() -> lsm_tree::Result<()> { @@ -26,7 +27,7 @@ fn tree_recover_segment_counter() -> lsm_tree::Result<()> { { let first_level = &tree.levels.read().expect("lock is poisoned").levels[0]; - assert_eq!(0, first_level.segments[0].metadata.id); + assert_eq!(0, first_level.segments[0].id()); } tree.insert("b", "b", 0); @@ -41,7 +42,7 @@ fn tree_recover_segment_counter() -> lsm_tree::Result<()> { { let first_level = &tree.levels.read().expect("lock is poisoned").levels[0]; - assert_eq!(1, first_level.segments[1].metadata.id); + assert_eq!(1, first_level.segments[1].id()); } } diff --git a/tests/tree_seqno.rs b/tests/tree_seqno.rs index a45a9ef4..6cf46217 100644 --- a/tests/tree_seqno.rs +++ b/tests/tree_seqno.rs @@ -1,6 +1,7 @@ use lsm_tree::{AbstractTree, Config}; +use test_log::test; -#[test_log::test] +#[test] fn tree_highest_seqno() -> lsm_tree::Result<()> { let folder = tempfile::tempdir()?.into_path(); diff --git a/tests/tree_write_read.rs b/tests/tree_write_read.rs index 45ef1bce..8f6e856b 100644 --- a/tests/tree_write_read.rs +++ b/tests/tree_write_read.rs @@ -11,17 +11,17 @@ fn tree_write_and_read() -> lsm_tree::Result<()> { tree.insert("b".as_bytes(), nanoid::nanoid!().as_bytes(), 1); tree.insert("c".as_bytes(), nanoid::nanoid!().as_bytes(), 2); - let item = tree.get_internal_entry("a", true, None)?.unwrap(); + let item = tree.get_internal_entry("a", None)?.unwrap(); assert_eq!(&*item.key.user_key, "a".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 0); - let item = tree.get_internal_entry("b", true, None)?.unwrap(); + let item = tree.get_internal_entry("b", None)?.unwrap(); assert_eq!(&*item.key.user_key, "b".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 1); - let item = tree.get_internal_entry("c", true, None)?.unwrap(); + let item = tree.get_internal_entry("c", None)?.unwrap(); assert_eq!(&*item.key.user_key, "c".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 2); @@ -30,17 +30,17 @@ fn tree_write_and_read() -> lsm_tree::Result<()> { let tree = Config::new(folder).open()?; - let item = tree.get_internal_entry("a", true, None)?.unwrap(); + let item = tree.get_internal_entry("a", None)?.unwrap(); assert_eq!(&*item.key.user_key, "a".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 0); - let item = tree.get_internal_entry("b", true, None)?.unwrap(); + let item = tree.get_internal_entry("b", None)?.unwrap(); assert_eq!(&*item.key.user_key, "b".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 1); - let item = tree.get_internal_entry("c", true, None)?.unwrap(); + let item = tree.get_internal_entry("c", None)?.unwrap(); assert_eq!(&*item.key.user_key, "c".as_bytes()); assert!(!item.is_tombstone()); assert_eq!(item.key.seqno, 2);