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