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
58mod object_id_manager;
59pub use error::*;
60pub use object_id_manager::*;
61pub use risingwave_common::cache::{CacheableEntry, LookupResult, LruCache};
62pub use validator::*;
63use value::*;
64
65use self::iterator::HummockIterator;
66pub use self::sstable_store::*;
67use crate::mem_table::ImmutableMemtable;
68use crate::monitor::StoreLocalStatistic;
69use crate::store::ReadOptions;
70
71pub async fn get_from_sstable_info(
72 sstable_store_ref: SstableStoreRef,
73 sstable_info: &SstableInfo,
74 full_key: FullKey<&[u8]>,
75 read_options: &ReadOptions,
76 dist_key_hash: Option<u64>,
77 local_stats: &mut StoreLocalStatistic,
78) -> HummockResult<Option<impl HummockIterator>> {
79 let sstable = sstable_store_ref.sstable(sstable_info, local_stats).await?;
80
81 if let Some(hash) = dist_key_hash
84 && !hit_sstable_bloom_filter(
85 &sstable,
86 &(
87 Bound::Included(full_key.user_key),
88 Bound::Included(full_key.user_key),
89 ),
90 hash,
91 local_stats,
92 )
93 {
94 return Ok(None);
95 }
96
97 let mut iter = SstableIterator::create(
98 sstable,
99 sstable_store_ref.clone(),
100 Arc::new(SstableIteratorReadOptions::from_read_options(read_options)),
101 sstable_info,
102 );
103 iter.seek(full_key).await?;
104 if !iter.is_valid() {
106 return Ok(None);
107 }
108
109 iter.collect_local_statistic(local_stats);
110
111 let value = if iter.key().user_key == full_key.user_key {
114 Some(iter)
115 } else {
116 None
117 };
118
119 Ok(value)
120}
121
122pub fn hit_sstable_bloom_filter(
123 sstable_ref: &Sstable,
124 user_key_range: &UserKeyRangeRef<'_>,
125 prefix_hash: u64,
126 local_stats: &mut StoreLocalStatistic,
127) -> bool {
128 local_stats.bloom_filter_check_counts += 1;
129 let may_exist = sstable_ref.may_match_hash(user_key_range, prefix_hash);
130 if !may_exist {
131 local_stats.bloom_filter_true_negative_counts += 1;
132 }
133 may_exist
134}
135
136pub fn get_from_batch<'a>(
138 imm: &'a ImmutableMemtable,
139 table_key: TableKey<&[u8]>,
140 read_epoch: HummockEpoch,
141 read_options: &ReadOptions,
142 local_stats: &mut StoreLocalStatistic,
143) -> Option<(HummockValue<&'a Bytes>, EpochWithGap)> {
144 imm.get(table_key, read_epoch, read_options).inspect(|_| {
145 local_stats.get_shared_buffer_hit_counts += 1;
146 })
147}