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