risingwave_hummock_sdk/
sstable_info.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::mem::size_of;
16use std::ops::Deref;
17use std::sync::Arc;
18
19use risingwave_pb::hummock::{PbBloomFilterType, PbKeyRange, PbSstableInfo};
20
21use crate::key_range::KeyRange;
22use crate::version::{ObjectIdReader, SstableIdReader};
23use crate::{HummockSstableId, HummockSstableObjectId};
24
25#[derive(Debug, PartialEq, Clone)]
26#[cfg_attr(any(test, feature = "test"), derive(Default))]
27pub struct SstableInfoInner {
28    pub object_id: HummockSstableObjectId,
29    pub sst_id: HummockSstableId,
30    pub key_range: KeyRange,
31    pub file_size: u64,
32    pub table_ids: Vec<u32>,
33    pub meta_offset: u64,
34    pub stale_key_count: u64,
35    pub total_key_count: u64,
36    pub min_epoch: u64,
37    pub max_epoch: u64,
38    pub uncompressed_file_size: u64,
39    pub range_tombstone_count: u64,
40    pub bloom_filter_kind: PbBloomFilterType,
41    pub sst_size: u64,
42}
43
44impl SstableInfoInner {
45    pub fn estimated_encode_len(&self) -> usize {
46        let mut basic = size_of::<u64>() // object_id
47            + size_of::<u64>() // sstable_id
48            + size_of::<u64>() // file_size
49            + self.table_ids.len() * size_of::<u32>() // table_ids
50            + size_of::<u64>() // meta_offset
51            + size_of::<u64>() // stale_key_count
52            + size_of::<u64>() // total_key_count
53            + size_of::<u64>() // min_epoch
54            + size_of::<u64>() // max_epoch
55            + size_of::<u64>() // uncompressed_file_size
56            + size_of::<u64>() // range_tombstone_count
57            + size_of::<u32>() // bloom_filter_kind
58            + size_of::<u64>(); // sst_size
59        basic += self.key_range.left.len() + self.key_range.right.len() + size_of::<bool>();
60
61        basic
62    }
63
64    pub fn to_protobuf(&self) -> PbSstableInfo {
65        self.into()
66    }
67}
68
69impl From<PbSstableInfo> for SstableInfoInner {
70    fn from(pb_sstable_info: PbSstableInfo) -> Self {
71        assert!(pb_sstable_info.table_ids.is_sorted());
72        Self {
73            object_id: pb_sstable_info.object_id.into(),
74            sst_id: pb_sstable_info.sst_id.into(),
75            key_range: {
76                // Due to the stripped key range, the key range may be `None`
77                if let Some(pb_keyrange) = pb_sstable_info.key_range {
78                    KeyRange {
79                        left: pb_keyrange.left.into(),
80                        right: pb_keyrange.right.into(),
81                        right_exclusive: pb_keyrange.right_exclusive,
82                    }
83                } else {
84                    KeyRange::inf()
85                }
86            },
87            file_size: pb_sstable_info.file_size,
88            table_ids: pb_sstable_info.table_ids.clone(),
89            meta_offset: pb_sstable_info.meta_offset,
90            stale_key_count: pb_sstable_info.stale_key_count,
91            total_key_count: pb_sstable_info.total_key_count,
92            min_epoch: pb_sstable_info.min_epoch,
93            max_epoch: pb_sstable_info.max_epoch,
94            uncompressed_file_size: pb_sstable_info.uncompressed_file_size,
95            range_tombstone_count: pb_sstable_info.range_tombstone_count,
96            bloom_filter_kind: PbBloomFilterType::try_from(pb_sstable_info.bloom_filter_kind)
97                .unwrap(),
98            sst_size: if pb_sstable_info.sst_size == 0 {
99                pb_sstable_info.file_size
100            } else {
101                pb_sstable_info.sst_size
102            },
103        }
104    }
105}
106
107impl From<&PbSstableInfo> for SstableInfoInner {
108    fn from(pb_sstable_info: &PbSstableInfo) -> Self {
109        assert!(pb_sstable_info.table_ids.is_sorted());
110        Self {
111            object_id: pb_sstable_info.object_id.into(),
112            sst_id: pb_sstable_info.sst_id.into(),
113            key_range: {
114                if let Some(pb_keyrange) = &pb_sstable_info.key_range {
115                    KeyRange {
116                        left: pb_keyrange.left.clone().into(),
117                        right: pb_keyrange.right.clone().into(),
118                        right_exclusive: pb_keyrange.right_exclusive,
119                    }
120                } else {
121                    KeyRange::inf()
122                }
123            },
124            file_size: pb_sstable_info.file_size,
125            table_ids: pb_sstable_info.table_ids.clone(),
126            meta_offset: pb_sstable_info.meta_offset,
127            stale_key_count: pb_sstable_info.stale_key_count,
128            total_key_count: pb_sstable_info.total_key_count,
129            min_epoch: pb_sstable_info.min_epoch,
130            max_epoch: pb_sstable_info.max_epoch,
131            uncompressed_file_size: pb_sstable_info.uncompressed_file_size,
132            range_tombstone_count: pb_sstable_info.range_tombstone_count,
133            bloom_filter_kind: PbBloomFilterType::try_from(pb_sstable_info.bloom_filter_kind)
134                .unwrap(),
135            sst_size: if pb_sstable_info.sst_size == 0 {
136                pb_sstable_info.file_size
137            } else {
138                pb_sstable_info.sst_size
139            },
140        }
141    }
142}
143
144impl From<SstableInfoInner> for PbSstableInfo {
145    fn from(sstable_info: SstableInfoInner) -> Self {
146        assert!(sstable_info.table_ids.is_sorted());
147        PbSstableInfo {
148            object_id: sstable_info.object_id.inner(),
149            sst_id: sstable_info.sst_id.inner(),
150            key_range: {
151                let keyrange = sstable_info.key_range;
152                if keyrange.inf_key_range() {
153                    // For empty key range, we don't need to encode it
154                    // Timetravel will use the default key range to stripped the PbSstableInfo
155                    // Note: If new fields are added, using Default to implement stripped may not work, resulting in an increase in encode size.
156                    None
157                } else {
158                    let pb_key_range = PbKeyRange {
159                        left: keyrange.left.into(),
160                        right: keyrange.right.into(),
161                        right_exclusive: keyrange.right_exclusive,
162                    };
163                    Some(pb_key_range)
164                }
165            },
166
167            file_size: sstable_info.file_size,
168            table_ids: sstable_info.table_ids.clone(),
169            meta_offset: sstable_info.meta_offset,
170            stale_key_count: sstable_info.stale_key_count,
171            total_key_count: sstable_info.total_key_count,
172            min_epoch: sstable_info.min_epoch,
173            max_epoch: sstable_info.max_epoch,
174            uncompressed_file_size: sstable_info.uncompressed_file_size,
175            range_tombstone_count: sstable_info.range_tombstone_count,
176            bloom_filter_kind: sstable_info.bloom_filter_kind.into(),
177            sst_size: sstable_info.sst_size,
178        }
179    }
180}
181
182impl From<&SstableInfoInner> for PbSstableInfo {
183    fn from(sstable_info: &SstableInfoInner) -> Self {
184        assert!(sstable_info.table_ids.is_sorted());
185        PbSstableInfo {
186            object_id: sstable_info.object_id.inner(),
187            sst_id: sstable_info.sst_id.inner(),
188            key_range: {
189                let keyrange = &sstable_info.key_range;
190                if keyrange.inf_key_range() {
191                    None
192                } else {
193                    let pb_key_range = PbKeyRange {
194                        left: keyrange.left.to_vec(),
195                        right: keyrange.right.to_vec(),
196                        right_exclusive: keyrange.right_exclusive,
197                    };
198                    Some(pb_key_range)
199                }
200            },
201
202            file_size: sstable_info.file_size,
203            table_ids: sstable_info.table_ids.clone(),
204            meta_offset: sstable_info.meta_offset,
205            stale_key_count: sstable_info.stale_key_count,
206            total_key_count: sstable_info.total_key_count,
207            min_epoch: sstable_info.min_epoch,
208            max_epoch: sstable_info.max_epoch,
209            uncompressed_file_size: sstable_info.uncompressed_file_size,
210            range_tombstone_count: sstable_info.range_tombstone_count,
211            bloom_filter_kind: sstable_info.bloom_filter_kind.into(),
212            sst_size: sstable_info.sst_size,
213        }
214    }
215}
216
217impl SstableInfo {
218    pub fn remove_key_range(&mut self) {
219        let mut sst = self.get_inner();
220        sst.key_range = KeyRange::default();
221        *self = sst.into()
222    }
223}
224
225impl SstableIdReader for SstableInfoInner {
226    fn sst_id(&self) -> HummockSstableId {
227        self.sst_id
228    }
229}
230
231impl ObjectIdReader for SstableInfoInner {
232    fn object_id(&self) -> HummockSstableObjectId {
233        self.object_id
234    }
235}
236
237#[derive(Debug, PartialEq, Clone)]
238#[cfg_attr(any(test, feature = "test"), derive(Default))]
239pub struct SstableInfo(Arc<SstableInfoInner>);
240
241impl From<&PbSstableInfo> for SstableInfo {
242    fn from(s: &PbSstableInfo) -> Self {
243        SstableInfo(SstableInfoInner::from(s).into())
244    }
245}
246
247impl From<PbSstableInfo> for SstableInfo {
248    fn from(s: PbSstableInfo) -> Self {
249        SstableInfo(SstableInfoInner::from(s).into())
250    }
251}
252
253impl From<SstableInfo> for PbSstableInfo {
254    fn from(s: SstableInfo) -> Self {
255        (&s).into()
256    }
257}
258
259impl From<SstableInfoInner> for SstableInfo {
260    fn from(s: SstableInfoInner) -> Self {
261        Self(s.into())
262    }
263}
264
265impl From<&SstableInfo> for PbSstableInfo {
266    fn from(s: &SstableInfo) -> Self {
267        s.0.as_ref().into()
268    }
269}
270
271impl Deref for SstableInfo {
272    type Target = SstableInfoInner;
273
274    fn deref(&self) -> &Self::Target {
275        &self.0
276    }
277}
278
279impl SstableInfo {
280    pub fn get_inner(&self) -> SstableInfoInner {
281        (*self.0).clone()
282    }
283}
284
285impl SstableIdReader for SstableInfo {
286    fn sst_id(&self) -> HummockSstableId {
287        self.sst_id
288    }
289}
290
291impl ObjectIdReader for SstableInfo {
292    fn object_id(&self) -> HummockSstableObjectId {
293        self.object_id
294    }
295}