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