risingwave_storage/hummock/
mod.rs1use std::ops::Bound;
18use std::sync::Arc;
19
20use bytes::Bytes;
21use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKeyRangeRef};
22use risingwave_hummock_sdk::sstable_info::SstableInfo;
23use risingwave_hummock_sdk::{HummockEpoch, *};
24
25pub mod block_cache;
26pub use block_cache::*;
27
28pub mod sstable;
29pub use sstable::*;
30
31pub mod compactor;
32mod error;
33pub mod hummock_meta_client;
34pub mod iterator;
35pub mod shared_buffer;
36pub mod sstable_store;
37#[cfg(any(test, feature = "test"))]
38pub mod test_utils;
39pub mod utils;
40pub use utils::MemoryLimiter;
41pub mod backup_reader;
42pub mod event_handler;
43pub mod local_version;
44pub mod observer_manager;
45pub mod store;
46pub use store::*;
47mod validator;
48pub mod value;
49pub mod write_limiter;
50
51pub mod recent_filter;
52pub use recent_filter::*;
53
54pub mod block_stream;
55mod iceberg_compactor_runner;
56mod time_travel_version_cache;
57
58pub(crate) mod vector;
59
60mod object_id_manager;
61pub use error::*;
62pub use object_id_manager::*;
63pub use risingwave_common::cache::{CacheableEntry, LookupResult, LruCache};
64pub use validator::*;
65use value::*;
66
67use self::iterator::HummockIterator;
68pub use self::sstable_store::*;
69use crate::mem_table::ImmutableMemtable;
70use crate::monitor::StoreLocalStatistic;
71use crate::store::ReadOptions;
72
73pub async fn get_from_sstable_info(
74 sstable_store_ref: SstableStoreRef,
75 sstable_info: &SstableInfo,
76 full_key: FullKey<&[u8]>,
77 read_options: &ReadOptions,
78 dist_key_hash: Option<u64>,
79 local_stats: &mut StoreLocalStatistic,
80) -> HummockResult<Option<impl HummockIterator>> {
81 let sstable = sstable_store_ref.sstable(sstable_info, local_stats).await?;
82
83 if let Some(hash) = dist_key_hash
86 && !hit_sstable_bloom_filter(
87 &sstable,
88 &(
89 Bound::Included(full_key.user_key),
90 Bound::Included(full_key.user_key),
91 ),
92 hash,
93 local_stats,
94 )
95 {
96 return Ok(None);
97 }
98
99 let mut iter = SstableIterator::create(
100 sstable,
101 sstable_store_ref.clone(),
102 Arc::new(SstableIteratorReadOptions::from_read_options(read_options)),
103 sstable_info,
104 );
105 iter.seek(full_key).await?;
106 if !iter.is_valid() {
108 return Ok(None);
109 }
110
111 iter.collect_local_statistic(local_stats);
112
113 let value = if iter.key().user_key == full_key.user_key {
116 Some(iter)
117 } else {
118 None
119 };
120
121 Ok(value)
122}
123
124pub fn hit_sstable_bloom_filter(
125 sstable_ref: &Sstable,
126 user_key_range: &UserKeyRangeRef<'_>,
127 prefix_hash: u64,
128 local_stats: &mut StoreLocalStatistic,
129) -> bool {
130 local_stats.bloom_filter_check_counts += 1;
131 let may_exist = sstable_ref.may_match_hash(user_key_range, prefix_hash);
132 if !may_exist {
133 local_stats.bloom_filter_true_negative_counts += 1;
134 }
135 may_exist
136}
137
138pub fn get_from_batch<'a>(
140 imm: &'a ImmutableMemtable,
141 table_key: TableKey<&[u8]>,
142 read_epoch: HummockEpoch,
143 read_options: &ReadOptions,
144 local_stats: &mut StoreLocalStatistic,
145) -> Option<(HummockValue<&'a Bytes>, EpochWithGap)> {
146 imm.get(table_key, read_epoch, read_options).inspect(|_| {
147 local_stats.get_shared_buffer_hit_counts += 1;
148 })
149}