lsm_tree/tree/
inner.rs

1// Copyright (c) 2024-present, fjall-rs
2// This source code is licensed under both the Apache 2.0 and MIT License
3// (found in the LICENSE-* files in the repository)
4
5use crate::{
6    compaction::state::CompactionState,
7    config::Config,
8    stop_signal::StopSignal,
9    version::{persist_version, SuperVersions, Version},
10    SequenceNumberCounter, TableId,
11};
12use std::sync::{atomic::AtomicU64, Arc, Mutex, RwLock};
13
14#[cfg(feature = "metrics")]
15use crate::metrics::Metrics;
16
17/// Unique tree ID
18///
19/// Tree IDs are monotonically increasing integers.
20pub type TreeId = u64;
21
22/// Unique memtable ID
23///
24/// Memtable IDs map one-to-one to some table.
25pub type MemtableId = u64;
26
27/// Hands out a unique (monotonically increasing) tree ID.
28pub fn get_next_tree_id() -> TreeId {
29    static TREE_ID_COUNTER: AtomicU64 = AtomicU64::new(0);
30    TREE_ID_COUNTER.fetch_add(1, std::sync::atomic::Ordering::Relaxed)
31}
32
33pub struct TreeInner {
34    /// Unique tree ID
35    pub id: TreeId,
36
37    /// Hands out a unique (monotonically increasing) memtable ID
38    #[doc(hidden)]
39    pub memtable_id_counter: SequenceNumberCounter,
40
41    /// Hands out a unique (monotonically increasing) table ID
42    #[doc(hidden)]
43    pub table_id_counter: SequenceNumberCounter,
44
45    // This is not really used in the normal tree, but we need it in the blob tree
46    /// Hands out a unique (monotonically increasing) blob file ID
47    pub(crate) blob_file_id_counter: SequenceNumberCounter,
48
49    pub(crate) version_history: Arc<RwLock<SuperVersions>>,
50
51    pub(crate) compaction_state: Arc<Mutex<CompactionState>>,
52
53    /// Tree configuration
54    pub config: Arc<Config>,
55
56    /// Compaction may take a while; setting the signal to `true`
57    /// will interrupt the compaction and kill the worker.
58    pub(crate) stop_signal: StopSignal,
59
60    /// Used by major compaction to be the exclusive compaction going on.
61    ///
62    /// Minor compactions use `major_compaction_lock.read()` instead, so they
63    /// can be concurrent next to each other.
64    pub(crate) major_compaction_lock: RwLock<()>,
65
66    /// Serializes flush operations.
67    pub(crate) flush_lock: Mutex<()>,
68    #[doc(hidden)]
69    #[cfg(feature = "metrics")]
70    pub metrics: Arc<Metrics>,
71}
72
73impl TreeInner {
74    pub(crate) fn create_new(config: Config) -> crate::Result<Self> {
75        let version = Version::new(
76            0,
77            if config.kv_separation_opts.is_some() {
78                crate::TreeType::Blob
79            } else {
80                crate::TreeType::Standard
81            },
82        );
83        persist_version(&config.path, &version)?;
84
85        Ok(Self {
86            id: get_next_tree_id(),
87            memtable_id_counter: SequenceNumberCounter::new(1),
88            table_id_counter: SequenceNumberCounter::default(),
89            blob_file_id_counter: SequenceNumberCounter::default(),
90            config: Arc::new(config),
91            version_history: Arc::new(RwLock::new(SuperVersions::new(version))),
92            stop_signal: StopSignal::default(),
93            major_compaction_lock: RwLock::default(),
94            flush_lock: Mutex::default(),
95            compaction_state: Arc::new(Mutex::new(CompactionState::default())),
96
97            #[cfg(feature = "metrics")]
98            metrics: Metrics::default().into(),
99        })
100    }
101
102    pub fn get_next_table_id(&self) -> TableId {
103        self.table_id_counter.next()
104    }
105}
106
107impl Drop for TreeInner {
108    fn drop(&mut self) {
109        log::debug!("Dropping TreeInner");
110
111        log::trace!("Sending stop signal to compactors");
112        self.stop_signal.send();
113    }
114}