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