From b250852fc6d42ce7b0e4c37a62c1bcca9e7c0bf7 Mon Sep 17 00:00:00 2001 From: Patrick Huang Date: Tue, 29 Aug 2023 17:21:26 +0800 Subject: [PATCH 1/6] refactor(storage): Use TableKey struct in the state store interfaces fix traced_store/synpoint/failpoint --- src/jni_core/src/hummock_iterator.rs | 2 +- src/storage/benches/bench_merge_iter.rs | 9 +- src/storage/clippy.toml | 26 + src/storage/hummock_sdk/src/key.rs | 7 + .../benches/bench_hummock_iter.rs | 7 +- .../src/bin/replay/replay_impl.rs | 20 +- .../hummock_test/src/compactor_tests.rs | 31 +- .../hummock_test/src/failpoint_tests.rs | 27 +- .../hummock_test/src/hummock_storage_tests.rs | 286 ++++----- .../hummock_test/src/snapshot_tests.rs | 27 +- .../hummock_test/src/state_store_tests.rs | 206 ++++--- .../hummock_test/src/sync_point_tests.rs | 92 ++- src/storage/hummock_test/src/test_utils.rs | 16 +- .../compactor/shared_buffer_compact.rs | 2 +- src/storage/src/hummock/error.rs | 6 - src/storage/src/hummock/event_handler/mod.rs | 2 +- .../src/hummock/event_handler/uploader.rs | 8 +- .../src/hummock/iterator/merge_inner.rs | 2 +- src/storage/src/hummock/iterator/mod.rs | 2 - .../src/hummock/iterator/test_utils.rs | 6 +- src/storage/src/hummock/mod.rs | 311 +--------- .../shared_buffer/shared_buffer_batch.rs | 14 +- .../src/hummock/store/hummock_storage.rs | 573 ++++++++++++++++++ ...tate_store.rs => local_hummock_storage.rs} | 54 +- src/storage/src/hummock/store/mod.rs | 7 +- src/storage/src/hummock/store/version.rs | 7 +- src/storage/src/hummock/test_utils.rs | 10 +- src/storage/src/hummock/utils.rs | 17 +- src/storage/src/lib.rs | 1 - src/storage/src/mem_table.rs | 122 ++-- src/storage/src/memory.rs | 70 ++- src/storage/src/monitor/monitored_store.rs | 20 +- src/storage/src/monitor/traced_store.rs | 36 +- src/storage/src/panic_store.rs | 23 +- src/storage/src/row_serde/row_serde_util.rs | 5 +- src/storage/src/store.rs | 38 +- src/storage/src/store_impl.rs | 72 ++- .../src/table/batch_table/storage_table.rs | 11 +- .../common/log_store/kv_log_store/reader.rs | 3 +- .../common/log_store/kv_log_store/serde.rs | 30 +- src/stream/src/common/table/state_table.rs | 28 +- .../src/compaction_test_runner.rs | 6 +- .../src/delete_range_runner.rs | 19 +- src/tests/compaction_test/src/lib.rs | 1 + 44 files changed, 1386 insertions(+), 876 deletions(-) create mode 100644 src/storage/clippy.toml create mode 100644 src/storage/src/hummock/store/hummock_storage.rs rename src/storage/src/hummock/store/{state_store.rs => local_hummock_storage.rs} (92%) diff --git a/src/jni_core/src/hummock_iterator.rs b/src/jni_core/src/hummock_iterator.rs index 92bb09885f960..7395a0f82273f 100644 --- a/src/jni_core/src/hummock_iterator.rs +++ b/src/jni_core/src/hummock_iterator.rs @@ -29,8 +29,8 @@ use risingwave_pb::java_binding::key_range::Bound; use risingwave_pb::java_binding::{KeyRange, ReadPlan}; use risingwave_storage::error::{StorageError, StorageResult}; use risingwave_storage::hummock::local_version::pinned_version::PinnedVersion; -use risingwave_storage::hummock::store::state_store::HummockStorageIterator; use risingwave_storage::hummock::store::version::HummockVersionReader; +use risingwave_storage::hummock::store::HummockStorageIterator; use risingwave_storage::hummock::{CachePolicy, FileCache, SstableStore}; use risingwave_storage::monitor::HummockStateStoreMetrics; use risingwave_storage::row_serde::value_serde::ValueRowSerdeNew; diff --git a/src/storage/benches/bench_merge_iter.rs b/src/storage/benches/bench_merge_iter.rs index 295ab2784a7a9..67c2642a86e64 100644 --- a/src/storage/benches/bench_merge_iter.rs +++ b/src/storage/benches/bench_merge_iter.rs @@ -17,6 +17,7 @@ use std::cell::RefCell; use bytes::Bytes; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use futures::executor::block_on; +use risingwave_hummock_sdk::key::TableKey; use risingwave_storage::hummock::iterator::{ Forward, HummockIterator, HummockIteratorUnion, OrderedMergeIteratorInner, UnorderedMergeIteratorInner, @@ -35,7 +36,9 @@ fn gen_interleave_shared_buffer_batch_iter( let mut batch_data = vec![]; for j in 0..batch_size { batch_data.push(( - Bytes::copy_from_slice(format!("test_key_{:08}", j * batch_count + i).as_bytes()), + TableKey(Bytes::copy_from_slice( + format!("test_key_{:08}", j * batch_count + i).as_bytes(), + )), HummockValue::put(Bytes::copy_from_slice("value".as_bytes())), )); } @@ -63,7 +66,9 @@ fn gen_interleave_shared_buffer_batch_enum_iter( let mut batch_data = vec![]; for j in 0..batch_size { batch_data.push(( - Bytes::copy_from_slice(format!("test_key_{:08}", j * batch_count + i).as_bytes()), + TableKey(Bytes::copy_from_slice( + format!("test_key_{:08}", j * batch_count + i).as_bytes(), + )), HummockValue::put(Bytes::copy_from_slice("value".as_bytes())), )); } diff --git a/src/storage/clippy.toml b/src/storage/clippy.toml new file mode 100644 index 0000000000000..e87606a3672d9 --- /dev/null +++ b/src/storage/clippy.toml @@ -0,0 +1,26 @@ +disallowed-methods = [ + { path = "std::iter::Iterator::zip", reason = "Please use `zip_eq_fast` if it's available. Otherwise use `zip_eq_debug`" }, + { path = "itertools::Itertools::zip_eq", reason = "Please use `zip_eq_fast` if it's available. Otherwise use `zip_eq_debug`" }, + { path = "futures::stream::select_all", reason = "Please use `risingwave_common::util::select_all` instead." }, + { path = "risingwave_common::array::JsonbVal::from_serde", reason = "Please add dedicated methods as part of `JsonbRef`/`JsonbVal`, rather than take inner `serde_json::Value` out, process, and put back." }, + { path = "std::panic::catch_unwind", reason = "Please use `risingwave_common::util::panic::rw_catch_unwind` instead." }, + { path = "futures::FutureExt::catch_unwind", reason = "Please use `risingwave_common::util::panic::FutureCatchUnwindExt::rw_catch_unwind` instead." }, +] +disallowed-macros = [ + { path = "lazy_static::lazy_static", reason = "Please use `std::sync::LazyLock` instead." }, +] +doc-valid-idents = [ + "RisingWave", + "MinIO", + "ProtoBuf", + "BloomFilter", + "gRPC", + "PostgreSQL", + "MySQL", + "TopN", + "VNode" +] +avoid-breaking-exported-api = false +upper-case-acronyms-aggressive = true +too-many-arguments-threshold = 10 +ignore-interior-mutability = ["risingwave_hummock_sdk::key::TableKey"] \ No newline at end of file diff --git a/src/storage/hummock_sdk/src/key.rs b/src/storage/hummock_sdk/src/key.rs index f58f32dd9aac9..34a28f51aa95f 100644 --- a/src/storage/hummock_sdk/src/key.rs +++ b/src/storage/hummock_sdk/src/key.rs @@ -20,6 +20,7 @@ use std::ptr; use bytes::{Buf, BufMut, Bytes, BytesMut}; use risingwave_common::catalog::TableId; +use risingwave_common::estimate_size::EstimateSize; use risingwave_common::hash::VirtualNode; use crate::HummockEpoch; @@ -411,6 +412,12 @@ impl> TableKey { } } +impl EstimateSize for TableKey { + fn estimated_heap_size(&self) -> usize { + self.0.estimated_size() + } +} + #[inline] pub fn map_table_key_range(range: (Bound, Bound)) -> TableKeyRange { (range.0.map(TableKey), range.1.map(TableKey)) diff --git a/src/storage/hummock_test/benches/bench_hummock_iter.rs b/src/storage/hummock_test/benches/bench_hummock_iter.rs index 3bd6738f9f9a2..1c6294fc672be 100644 --- a/src/storage/hummock_test/benches/bench_hummock_iter.rs +++ b/src/storage/hummock_test/benches/bench_hummock_iter.rs @@ -19,6 +19,7 @@ use bytes::Bytes; use criterion::{criterion_group, criterion_main, Criterion}; use futures::{pin_mut, TryStreamExt}; use risingwave_common::cache::CachePriority; +use risingwave_hummock_sdk::key::TableKey; use risingwave_hummock_test::get_notification_client_for_test; use risingwave_hummock_test::local_state_store_test_utils::LocalStateStoreTestExt; use risingwave_hummock_test::test_utils::TestIngestBatch; @@ -34,13 +35,15 @@ use risingwave_storage::StateStore; fn gen_interleave_shared_buffer_batch_iter( batch_size: usize, batch_count: usize, -) -> Vec> { +) -> Vec, StorageValue)>> { let mut ret = Vec::new(); for i in 0..batch_count { let mut batch_data = vec![]; for j in 0..batch_size { batch_data.push(( - Bytes::copy_from_slice(format!("test_key_{:08}", j * batch_count + i).as_bytes()), + TableKey(Bytes::copy_from_slice( + format!("test_key_{:08}", j * batch_count + i).as_bytes(), + )), StorageValue::new_put(Bytes::copy_from_slice("value".as_bytes())), )); } diff --git a/src/storage/hummock_test/src/bin/replay/replay_impl.rs b/src/storage/hummock_test/src/bin/replay/replay_impl.rs index e96855d75f925..84dc71d88599f 100644 --- a/src/storage/hummock_test/src/bin/replay/replay_impl.rs +++ b/src/storage/hummock_test/src/bin/replay/replay_impl.rs @@ -20,6 +20,7 @@ use futures_async_stream::{for_await, try_stream}; use risingwave_common::error::Result as RwResult; use risingwave_common::util::addr::HostAddr; use risingwave_common_service::observer_manager::{Channel, NotificationClient}; +use risingwave_hummock_sdk::key::TableKey; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_hummock_trace::{ GlobalReplay, LocalReplay, LocalReplayRead, ReplayItem, ReplayRead, ReplayStateStore, @@ -30,7 +31,7 @@ use risingwave_meta::manager::{MessageStatus, MetaSrvEnv, NotificationManagerRef use risingwave_pb::common::WorkerNode; use risingwave_pb::meta::subscribe_response::{Info, Operation as RespOperation}; use risingwave_pb::meta::{SubscribeResponse, SubscribeType}; -use risingwave_storage::hummock::store::state_store::LocalHummockStorage; +use risingwave_storage::hummock::store::LocalHummockStorage; use risingwave_storage::hummock::HummockStorage; use risingwave_storage::store::{ LocalStateStore, StateStoreIterItemStream, StateStoreRead, SyncResult, @@ -107,8 +108,8 @@ impl ReplayRead for GlobalReplayImpl { read_options: TracedReadOptions, ) -> Result>> { let key_range = ( - key_range.0.map(TracedBytes::into), - key_range.1.map(TracedBytes::into), + key_range.0.map(TracedBytes::into).map(TableKey), + key_range.1.map(TracedBytes::into).map(TableKey), ); let iter = self @@ -129,7 +130,7 @@ impl ReplayRead for GlobalReplayImpl { ) -> Result> { Ok(self .store - .get(key.into(), epoch, read_options.into()) + .get(TableKey(key.into()), epoch, read_options.into()) .await .unwrap() .map(TracedBytes::from)) @@ -240,7 +241,10 @@ impl LocalReplayRead for LocalReplayImpl { key_range: (Bound, Bound), read_options: TracedReadOptions, ) -> Result>> { - let key_range = (key_range.0.map(|b| b.into()), key_range.1.map(|b| b.into())); + let key_range = ( + key_range.0.map(|b| TableKey(b.into())), + key_range.1.map(|b| TableKey(b.into())), + ); let iter = LocalStateStore::iter(&self.0, key_range, read_options.into()) .await @@ -257,7 +261,7 @@ impl LocalReplayRead for LocalReplayImpl { read_options: TracedReadOptions, ) -> Result> { Ok( - LocalStateStore::get(&self.0, key.into(), read_options.into()) + LocalStateStore::get(&self.0, TableKey(key.into()), read_options.into()) .await .unwrap() .map(TracedBytes::from), @@ -275,7 +279,7 @@ impl ReplayWrite for LocalReplayImpl { ) -> Result<()> { LocalStateStore::insert( &mut self.0, - key.into(), + TableKey(key.into()), new_val.into(), old_val.map(|b| b.into()), ) @@ -284,7 +288,7 @@ impl ReplayWrite for LocalReplayImpl { } fn delete(&mut self, key: TracedBytes, old_val: TracedBytes) -> Result<()> { - LocalStateStore::delete(&mut self.0, key.into(), old_val.into()).unwrap(); + LocalStateStore::delete(&mut self.0, TableKey(key.into()), old_val.into()).unwrap(); Ok(()) } } diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 5864fa9c0a484..dc432b468a0f3 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -30,7 +30,7 @@ pub(crate) mod tests { use risingwave_common_service::observer_manager::NotificationClient; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; - use risingwave_hummock_sdk::key::{next_key, TABLE_PREFIX_LEN}; + use risingwave_hummock_sdk::key::{next_key, TableKey, TABLE_PREFIX_LEN}; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; use risingwave_meta::hummock::compaction::compaction_config::CompactionConfigBuilder; use risingwave_meta::hummock::compaction::{default_level_selector, ManualCompactionOption}; @@ -141,7 +141,10 @@ pub(crate) mod tests { new_val.extend_from_slice(&epoch.to_be_bytes()); local .ingest_batch( - vec![(key.clone(), StorageValue::new_put(Bytes::from(new_val)))], + vec![( + TableKey(key.clone()), + StorageValue::new_put(Bytes::from(new_val)), + )], vec![], WriteOptions { epoch, @@ -253,7 +256,7 @@ pub(crate) mod tests { .pin_snapshot(worker_node2.id) .await .unwrap(); - let key = key.freeze(); + let key = TableKey(key.freeze()); const SST_COUNT: u64 = 32; const TEST_WATERMARK: u64 = 8; prepare_test_put_data( @@ -359,7 +362,7 @@ pub(crate) mod tests { key.clone(), ((TEST_WATERMARK - 1) * 1000) << 16, ReadOptions { - prefix_hint: Some(key.clone()), + prefix_hint: Some(key.clone().0), cache_policy: CachePolicy::Fill(CachePriority::High), ..Default::default() }, @@ -407,7 +410,7 @@ pub(crate) mod tests { let mut key = BytesMut::default(); key.put_u16(0); key.put_slice(b"same_key"); - let key = key.freeze(); + let key = TableKey(key.freeze()); const SST_COUNT: u64 = 16; let mut val = b"0"[..].repeat(1 << 20); @@ -528,7 +531,9 @@ pub(crate) mod tests { let mut key = idx.to_be_bytes().to_vec(); let ramdom_key = rand::thread_rng().gen::<[u8; 32]>(); key.extend_from_slice(&ramdom_key); - local.insert(Bytes::from(key), val.clone(), None).unwrap(); + local + .insert(TableKey(Bytes::from(key)), val.clone(), None) + .unwrap(); } local.flush(Vec::new()).await.unwrap(); local.seal_current_epoch(epoch + 1); @@ -709,7 +714,9 @@ pub(crate) mod tests { prefix.put_u16(1); prefix.put_slice(random_key.as_slice()); - storage.insert(prefix.freeze(), val.clone(), None).unwrap(); + storage + .insert(TableKey(prefix.freeze()), val.clone(), None) + .unwrap(); storage.flush(Vec::new()).await.unwrap(); storage.seal_current_epoch(next_epoch); other.seal_current_epoch(next_epoch); @@ -893,7 +900,9 @@ pub(crate) mod tests { prefix.put_u16(1); prefix.put_slice(random_key.as_slice()); - local.insert(prefix.freeze(), val.clone(), None).unwrap(); + local + .insert(TableKey(prefix.freeze()), val.clone(), None) + .unwrap(); local.flush(Vec::new()).await.unwrap(); local.seal_current_epoch(next_epoch); @@ -1084,7 +1093,7 @@ pub(crate) mod tests { let ramdom_key = [key_prefix.as_ref(), &rand::thread_rng().gen::<[u8; 32]>()].concat(); local - .insert(Bytes::from(ramdom_key), val.clone(), None) + .insert(TableKey(Bytes::from(ramdom_key)), val.clone(), None) .unwrap(); local.flush(Vec::new()).await.unwrap(); local.seal_current_epoch(next_epoch); @@ -1183,8 +1192,8 @@ pub(crate) mod tests { key_prefix.to_vec(), ] .concat(); - let start_bound_key = key_prefix; - let end_bound_key = Bytes::from(next_key(start_bound_key.as_ref())); + let start_bound_key = TableKey(key_prefix); + let end_bound_key = TableKey(Bytes::from(next_key(start_bound_key.as_ref()))); let scan_result = storage .scan( ( diff --git a/src/storage/hummock_test/src/failpoint_tests.rs b/src/storage/hummock_test/src/failpoint_tests.rs index 83af92d469afe..f71d479c3c55c 100644 --- a/src/storage/hummock_test/src/failpoint_tests.rs +++ b/src/storage/hummock_test/src/failpoint_tests.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use bytes::{BufMut, Bytes}; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; +use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::key::TABLE_PREFIX_LEN; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_meta::hummock::test_utils::setup_compute_env; @@ -34,7 +35,7 @@ use risingwave_storage::StateStore; use crate::get_notification_client_for_test; use crate::local_state_store_test_utils::LocalStateStoreTestExt; -use crate::test_utils::TestIngestBatch; +use crate::test_utils::{gen_key_from_str, TestIngestBatch}; #[tokio::test] #[ignore] @@ -62,15 +63,21 @@ async fn test_failpoints_state_store_read_upload() { let mut local = hummock_storage.new_local(NewLocalOptions::default()).await; - let anchor = Bytes::from("aa"); + let anchor = gen_key_from_str(VirtualNode::ZERO, "aa"); let mut batch1 = vec![ (anchor.clone(), StorageValue::new_put("111")), - (Bytes::from("cc"), StorageValue::new_put("222")), + ( + gen_key_from_str(VirtualNode::ZERO, "aa"), + StorageValue::new_put("222"), + ), ]; batch1.sort_by(|(k1, _), (k2, _)| k1.cmp(k2)); let mut batch2 = vec![ - (Bytes::from("cc"), StorageValue::new_put("333")), + ( + gen_key_from_str(VirtualNode::ZERO, "aa"), + StorageValue::new_put("333"), + ), (anchor.clone(), StorageValue::new_delete()), ]; // Make sure the batch is sorted. @@ -162,7 +169,10 @@ async fn test_failpoints_state_store_read_upload() { assert!(result.is_err()); let result = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), + ( + Bound::Unbounded, + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "ee")), + ), 2, ReadOptions { table_id: Default::default(), @@ -181,7 +191,7 @@ async fn test_failpoints_state_store_read_upload() { }; let value = hummock_storage .get( - Bytes::from("ee"), + gen_key_from_str(VirtualNode::ZERO, "ee"), 2, ReadOptions { prefix_hint: Some(Bytes::from(bee_prefix_hint)), @@ -233,7 +243,10 @@ async fn test_failpoints_state_store_read_upload() { assert_eq!(value, Bytes::from("111")); let iters = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), + ( + Bound::Unbounded, + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "ee")), + ), 5, ReadOptions { prefetch_options: PrefetchOptions::new_for_exhaust_iter(), diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 4e3d6e1aed919..8fb1b93e2d45e 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -20,7 +20,7 @@ use parking_lot::RwLock; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; -use risingwave_hummock_sdk::key::{map_table_key_range, FullKey, UserKey, TABLE_PREFIX_LEN}; +use risingwave_hummock_sdk::key::{FullKey, TableKey, TABLE_PREFIX_LEN}; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::hummock::store::version::{read_filter_for_batch, read_filter_for_local}; use risingwave_storage::hummock::CachePolicy; @@ -29,7 +29,7 @@ use risingwave_storage::store::*; use risingwave_storage::StateStore; use crate::local_state_store_test_utils::LocalStateStoreTestExt; -use crate::test_utils::{prepare_hummock_test_env, TestIngestBatch}; +use crate::test_utils::{gen_key_from_str, prepare_hummock_test_env, TestIngestBatch}; #[tokio::test] async fn test_storage_basic() { @@ -40,15 +40,14 @@ async fn test_storage_basic() { .storage .new_local(NewLocalOptions::for_test(TEST_TABLE_ID)) .await; - // First batch inserts the anchor and others. let mut batch1 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), StorageValue::new_put("111"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb"].concat()), + gen_key_from_str(VirtualNode::ZERO, "bb"), StorageValue::new_put("222"), ), ]; @@ -59,11 +58,11 @@ async fn test_storage_basic() { // Second batch modifies the anchor. let mut batch2 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc"].concat()), + gen_key_from_str(VirtualNode::ZERO, "cc"), StorageValue::new_put("333"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), StorageValue::new_put("111111"), ), ]; @@ -74,15 +73,15 @@ async fn test_storage_basic() { // Third batch deletes the anchor let mut batch3 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"dd"].concat()), + gen_key_from_str(VirtualNode::ZERO, "dd"), StorageValue::new_put("444"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"ee"].concat()), + gen_key_from_str(VirtualNode::ZERO, "ee"), StorageValue::new_put("555"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), StorageValue::new_delete(), ), ]; @@ -111,7 +110,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch1, ReadOptions { table_id: TEST_TABLE_ID, @@ -127,7 +126,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb"].concat()), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1, ReadOptions { table_id: TEST_TABLE_ID, @@ -144,7 +143,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"ab"].concat()), + gen_key_from_str(VirtualNode::ZERO, "ab"), epoch1, ReadOptions { table_id: TEST_TABLE_ID, @@ -174,7 +173,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch2, ReadOptions { table_id: TEST_TABLE_ID, @@ -207,7 +206,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch3, ReadOptions { table_id: TEST_TABLE_ID, @@ -223,7 +222,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"ff"].concat()), + gen_key_from_str(VirtualNode::ZERO, "ff"), epoch3, ReadOptions { table_id: TEST_TABLE_ID, @@ -242,9 +241,7 @@ async fn test_storage_basic() { .iter( ( Unbounded, - Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"ee"].concat(), - )), + Included(gen_key_from_str(VirtualNode::ZERO, "ee")), ), epoch1, ReadOptions { @@ -289,7 +286,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch1, ReadOptions { table_id: TEST_TABLE_ID, @@ -307,7 +304,7 @@ async fn test_storage_basic() { let value = test_env .storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch2, ReadOptions { table_id: TEST_TABLE_ID, @@ -326,9 +323,7 @@ async fn test_storage_basic() { .iter( ( Unbounded, - Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"ee"].concat(), - )), + Included(gen_key_from_str(VirtualNode::ZERO, "ee")), ), epoch2, ReadOptions { @@ -387,9 +382,7 @@ async fn test_storage_basic() { .iter( ( Unbounded, - Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"ee"].concat(), - )), + Included(gen_key_from_str(VirtualNode::ZERO, "ee")), ), epoch3, ReadOptions { @@ -476,11 +469,11 @@ async fn test_state_store_sync() { // ingest 16B batch let mut batch1 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aaaa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aaaa"), StorageValue::new_put("1111"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bbbb"].concat()), + gen_key_from_str(VirtualNode::ZERO, "bbbb"), StorageValue::new_put("2222"), ), ]; @@ -501,15 +494,15 @@ async fn test_state_store_sync() { // ingest 24B batch let mut batch2 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cccc"].concat()), + gen_key_from_str(VirtualNode::ZERO, "cccc"), StorageValue::new_put("3333"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"dddd"].concat()), + gen_key_from_str(VirtualNode::ZERO, "dddd"), StorageValue::new_put("4444"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"eeee"].concat()), + gen_key_from_str(VirtualNode::ZERO, "eeee"), StorageValue::new_put("5555"), ), ]; @@ -531,7 +524,7 @@ async fn test_state_store_sync() { // ingest more 8B then will trigger a sync behind the scene let mut batch3 = vec![( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"eeee"].concat()), + gen_key_from_str(VirtualNode::ZERO, "eeee"), StorageValue::new_put("6666"), )]; batch3.sort_by(|(k1, _), (k2, _)| k1.cmp(k2)); @@ -563,26 +556,11 @@ async fn test_state_store_sync() { { let kv_map = [ - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aaaa"].concat()), - "1111", - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bbbb"].concat()), - "2222", - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cccc"].concat()), - "3333", - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"dddd"].concat()), - "4444", - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"eeee"].concat()), - "5555", - ), + (gen_key_from_str(VirtualNode::ZERO, "aaaa"), "1111"), + (gen_key_from_str(VirtualNode::ZERO, "bbbb"), "2222"), + (gen_key_from_str(VirtualNode::ZERO, "cccc"), "3333"), + (gen_key_from_str(VirtualNode::ZERO, "dddd"), "4444"), + (gen_key_from_str(VirtualNode::ZERO, "eeee"), "5555"), ]; for (k, v) in kv_map { @@ -620,26 +598,11 @@ async fn test_state_store_sync() { { let kv_map = [ - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aaaa"].concat()), - "1111", - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bbbb"].concat()), - "2222", - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cccc"].concat()), - "3333", - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"dddd"].concat()), - "4444", - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"eeee"].concat()), - "6666", - ), + (gen_key_from_str(VirtualNode::ZERO, "aaaa"), "1111"), + (gen_key_from_str(VirtualNode::ZERO, "bbbb"), "2222"), + (gen_key_from_str(VirtualNode::ZERO, "cccc"), "3333"), + (gen_key_from_str(VirtualNode::ZERO, "dddd"), "4444"), + (gen_key_from_str(VirtualNode::ZERO, "eeee"), "6666"), ]; for (k, v) in kv_map { @@ -668,9 +631,7 @@ async fn test_state_store_sync() { .iter( ( Unbounded, - Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"eeee"].concat(), - )), + Included(gen_key_from_str(VirtualNode::ZERO, "eeee")), ), epoch1, ReadOptions { @@ -684,38 +645,18 @@ async fn test_state_store_sync() { futures::pin_mut!(iter); let kv_map = [ - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aaaa"].concat()), - "1111", - epoch1, - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bbbb"].concat()), - "2222", - epoch1, - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cccc"].concat()), - "3333", - epoch1, - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"dddd"].concat()), - "4444", - epoch1, - ), - ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"eeee"].concat()), - "5555", - epoch1, - ), + (gen_key_from_str(VirtualNode::ZERO, "aaaa"), "1111", epoch1), + (gen_key_from_str(VirtualNode::ZERO, "bbbb"), "2222", epoch1), + (gen_key_from_str(VirtualNode::ZERO, "cccc"), "3333", epoch1), + (gen_key_from_str(VirtualNode::ZERO, "dddd"), "4444", epoch1), + (gen_key_from_str(VirtualNode::ZERO, "eeee"), "5555", epoch1), ]; for (k, v, e) in kv_map { let result = iter.try_next().await.unwrap(); assert_eq!( result, - Some((FullKey::for_test(TEST_TABLE_ID, k, e), Bytes::from(v))) + Some((FullKey::new(TEST_TABLE_ID, k, e), Bytes::from(v))) ); } @@ -728,9 +669,7 @@ async fn test_state_store_sync() { .iter( ( Unbounded, - Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"eeee"].concat(), - )), + Included(gen_key_from_str(VirtualNode::ZERO, "eeee")), ), epoch2, ReadOptions { @@ -791,8 +730,14 @@ async fn test_delete_get() { hummock_storage.init_for_test(epoch1).await.unwrap(); let batch1 = vec![ - (Bytes::from("aa"), StorageValue::new_put("111")), - (Bytes::from("bb"), StorageValue::new_put("222")), + ( + gen_key_from_str(VirtualNode::ZERO, "aa"), + StorageValue::new_put("111"), + ), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("222"), + ), ]; hummock_storage .ingest_batch( @@ -814,7 +759,10 @@ async fn test_delete_get() { .unwrap(); let epoch2 = initial_epoch + 2; hummock_storage.seal_current_epoch(epoch2); - let batch2 = vec![(Bytes::from("bb"), StorageValue::new_delete())]; + let batch2 = vec![( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_delete(), + )]; hummock_storage .ingest_batch( batch2, @@ -836,7 +784,7 @@ async fn test_delete_get() { assert!(test_env .storage .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch2, ReadOptions { prefix_hint: None, @@ -868,8 +816,14 @@ async fn test_multiple_epoch_sync() { let epoch1 = initial_epoch + 1; hummock_storage.init_for_test(epoch1).await.unwrap(); let batch1 = vec![ - (Bytes::from("aa"), StorageValue::new_put("111")), - (Bytes::from("bb"), StorageValue::new_put("222")), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("111"), + ), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("222"), + ), ]; hummock_storage .ingest_batch( @@ -885,7 +839,10 @@ async fn test_multiple_epoch_sync() { let epoch2 = initial_epoch + 2; hummock_storage.seal_current_epoch(epoch2); - let batch2 = vec![(Bytes::from("bb"), StorageValue::new_delete())]; + let batch2 = vec![( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_delete(), + )]; hummock_storage .ingest_batch( batch2, @@ -901,8 +858,14 @@ async fn test_multiple_epoch_sync() { let epoch3 = initial_epoch + 3; hummock_storage.seal_current_epoch(epoch3); let batch3 = vec![ - (Bytes::from("aa"), StorageValue::new_put("444")), - (Bytes::from("bb"), StorageValue::new_put("555")), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("444"), + ), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("555"), + ), ]; hummock_storage .ingest_batch( @@ -921,7 +884,7 @@ async fn test_multiple_epoch_sync() { assert_eq!( hummock_storage_clone .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1, ReadOptions { table_id: TEST_TABLE_ID, @@ -936,7 +899,7 @@ async fn test_multiple_epoch_sync() { ); assert!(hummock_storage_clone .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch2, ReadOptions { table_id: TEST_TABLE_ID, @@ -951,7 +914,7 @@ async fn test_multiple_epoch_sync() { assert_eq!( hummock_storage_clone .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch3, ReadOptions { table_id: TEST_TABLE_ID, @@ -1000,18 +963,15 @@ async fn test_iter_with_min_epoch() { let epoch1 = (31 * 1000) << 16; - let gen_key = |index: usize| -> String { format!("\0\0key_{}", index) }; + let gen_key = |index: usize| -> TableKey { + gen_key_from_str(VirtualNode::ZERO, format!("\0\0key_{}", index).as_str()) + }; let gen_val = |index: usize| -> String { format!("val_{}", index) }; // epoch 1 write - let batch_epoch1: Vec<(Bytes, StorageValue)> = (0..10) - .map(|index| { - ( - Bytes::from(gen_key(index)), - StorageValue::new_put(gen_val(index)), - ) - }) + let batch_epoch1: Vec<(TableKey, StorageValue)> = (0..10) + .map(|index| (gen_key(index), StorageValue::new_put(gen_val(index)))) .collect(); hummock_storage.init_for_test(epoch1).await.unwrap(); @@ -1031,13 +991,8 @@ async fn test_iter_with_min_epoch() { let epoch2 = (32 * 1000) << 16; hummock_storage.seal_current_epoch(epoch2); // epoch 2 write - let batch_epoch2: Vec<(Bytes, StorageValue)> = (20..30) - .map(|index| { - ( - Bytes::from(gen_key(index)), - StorageValue::new_put(gen_val(index)), - ) - }) + let batch_epoch2: Vec<(TableKey, StorageValue)> = (20..30) + .map(|index| (gen_key(index), StorageValue::new_put(gen_val(index)))) .collect(); hummock_storage @@ -1219,40 +1174,27 @@ async fn test_hummock_version_reader() { let epoch1 = (31 * 1000) << 16; - let gen_key = |index: usize| -> String { format!("\0\0key_{}", index) }; + let gen_key = |index: usize| -> TableKey { + gen_key_from_str(VirtualNode::ZERO, format!("\0\0key_{}", index).as_str()) + }; let gen_val = |index: usize| -> String { format!("val_{}", index) }; // epoch 1 write - let batch_epoch1: Vec<(Bytes, StorageValue)> = (0..10) - .map(|index| { - ( - Bytes::from(gen_key(index)), - StorageValue::new_put(gen_val(index)), - ) - }) + let batch_epoch1: Vec<(TableKey, StorageValue)> = (0..10) + .map(|index| (gen_key(index), StorageValue::new_put(gen_val(index)))) .collect(); let epoch2 = (32 * 1000) << 16; // epoch 2 write - let batch_epoch2: Vec<(Bytes, StorageValue)> = (20..30) - .map(|index| { - ( - Bytes::from(gen_key(index)), - StorageValue::new_put(gen_val(index)), - ) - }) + let batch_epoch2: Vec<(TableKey, StorageValue)> = (20..30) + .map(|index| (gen_key(index), StorageValue::new_put(gen_val(index)))) .collect(); let epoch3 = (33 * 1000) << 16; // epoch 3 write - let batch_epoch3: Vec<(Bytes, StorageValue)> = (40..50) - .map(|index| { - ( - Bytes::from(gen_key(index)), - StorageValue::new_put(gen_val(index)), - ) - }) + let batch_epoch3: Vec<(TableKey, StorageValue)> = (40..50) + .map(|index| (gen_key(index), StorageValue::new_put(gen_val(index)))) .collect(); { hummock_storage.init_for_test(epoch1).await.unwrap(); @@ -1565,10 +1507,10 @@ async fn test_hummock_version_reader() { } { - let start_key = Bytes::from(gen_key(25)); - let end_key = Bytes::from(gen_key(50)); + let start_key = gen_key(25); + let end_key = gen_key(50); - let key_range = map_table_key_range((Included(start_key), Excluded(end_key))); + let key_range = (Included(start_key), Excluded(end_key)); { let read_snapshot = { @@ -1659,20 +1601,15 @@ async fn test_get_with_min_epoch() { let epoch1 = (31 * 1000) << 16; hummock_storage.init_for_test(epoch1).await.unwrap(); - let gen_key = |index: usize| -> Vec { - UserKey::for_test(TEST_TABLE_ID, format!("key_{}", index)).encode() + let gen_key = |index: usize| -> TableKey { + gen_key_from_str(VirtualNode::ZERO, format!("key_{}", index).as_str()) }; let gen_val = |index: usize| -> String { format!("val_{}", index) }; // epoch 1 write - let batch_epoch1: Vec<(Bytes, StorageValue)> = (0..10) - .map(|index| { - ( - Bytes::from(gen_key(index)), - StorageValue::new_put(gen_val(index)), - ) - }) + let batch_epoch1: Vec<(TableKey, StorageValue)> = (0..10) + .map(|index| (gen_key(index), StorageValue::new_put(gen_val(index)))) .collect(); hummock_storage @@ -1690,13 +1627,8 @@ async fn test_get_with_min_epoch() { let epoch2 = (32 * 1000) << 16; hummock_storage.seal_current_epoch(epoch2); // epoch 2 write - let batch_epoch2: Vec<(Bytes, StorageValue)> = (20..30) - .map(|index| { - ( - Bytes::from(gen_key(index)), - StorageValue::new_put(gen_val(index)), - ) - }) + let batch_epoch2: Vec<(TableKey, StorageValue)> = (20..30) + .map(|index| (gen_key(index), StorageValue::new_put(gen_val(index)))) .collect(); hummock_storage @@ -1713,7 +1645,7 @@ async fn test_get_with_min_epoch() { { // test before sync - let k = Bytes::from(gen_key(0)); + let k = gen_key(0); let prefix_hint = { let mut ret = Vec::with_capacity(TABLE_PREFIX_LEN + k.len()); ret.put_u32(TEST_TABLE_ID.table_id()); @@ -1811,7 +1743,7 @@ async fn test_get_with_min_epoch() { .unwrap(); test_env.storage.try_wait_epoch_for_test(epoch2).await; - let k = Bytes::from(gen_key(0)); + let k = gen_key(0); let prefix_hint = { let mut ret = Vec::with_capacity(TABLE_PREFIX_LEN + k.len()); ret.put_u32(TEST_TABLE_ID.table_id()); @@ -1857,7 +1789,7 @@ async fn test_get_with_min_epoch() { } { - let k = Bytes::from(gen_key(0)); + let k = gen_key(0); let v = test_env .storage .get( @@ -1876,7 +1808,7 @@ async fn test_get_with_min_epoch() { } { - let k = Bytes::from(gen_key(0)); + let k = gen_key(0); let v = test_env .storage .get( diff --git a/src/storage/hummock_test/src/snapshot_tests.rs b/src/storage/hummock_test/src/snapshot_tests.rs index 3870dae070903..ebc8b52358b8e 100644 --- a/src/storage/hummock_test/src/snapshot_tests.rs +++ b/src/storage/hummock_test/src/snapshot_tests.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use bytes::Bytes; use futures::TryStreamExt; use risingwave_common::cache::CachePriority; +use risingwave_hummock_sdk::key::{map_table_key_range, TableKey}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_meta::hummock::MockHummockMetaClient; use risingwave_rpc_client::HummockMetaClient; @@ -39,7 +40,7 @@ macro_rules! assert_count_range_scan { ); let it = $storage .iter( - bounds, + map_table_key_range(bounds), $epoch, ReadOptions { prefetch_options: PrefetchOptions::new_for_exhaust_iter(), @@ -110,8 +111,8 @@ async fn test_snapshot_inner( local .ingest_batch( vec![ - (Bytes::from("1"), StorageValue::new_put("test")), - (Bytes::from("2"), StorageValue::new_put("test")), + (TableKey(Bytes::from("1")), StorageValue::new_put("test")), + (TableKey(Bytes::from("2")), StorageValue::new_put("test")), ], vec![], WriteOptions { @@ -145,9 +146,9 @@ async fn test_snapshot_inner( local .ingest_batch( vec![ - (Bytes::from("1"), StorageValue::new_delete()), - (Bytes::from("3"), StorageValue::new_put("test")), - (Bytes::from("4"), StorageValue::new_put("test")), + (TableKey(Bytes::from("1")), StorageValue::new_delete()), + (TableKey(Bytes::from("3")), StorageValue::new_put("test")), + (TableKey(Bytes::from("4")), StorageValue::new_put("test")), ], vec![], WriteOptions { @@ -182,9 +183,9 @@ async fn test_snapshot_inner( local .ingest_batch( vec![ - (Bytes::from("2"), StorageValue::new_delete()), - (Bytes::from("3"), StorageValue::new_delete()), - (Bytes::from("4"), StorageValue::new_delete()), + (TableKey(Bytes::from("2")), StorageValue::new_delete()), + (TableKey(Bytes::from("3")), StorageValue::new_delete()), + (TableKey(Bytes::from("4")), StorageValue::new_delete()), ], vec![], WriteOptions { @@ -232,10 +233,10 @@ async fn test_snapshot_range_scan_inner( local .ingest_batch( vec![ - (Bytes::from("1"), StorageValue::new_put("test")), - (Bytes::from("2"), StorageValue::new_put("test")), - (Bytes::from("3"), StorageValue::new_put("test")), - (Bytes::from("4"), StorageValue::new_put("test")), + (TableKey(Bytes::from("1")), StorageValue::new_put("test")), + (TableKey(Bytes::from("2")), StorageValue::new_put("test")), + (TableKey(Bytes::from("3")), StorageValue::new_put("test")), + (TableKey(Bytes::from("4")), StorageValue::new_put("test")), ], vec![], WriteOptions { diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index bc68da1f9d298..21d132f689d2b 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -38,14 +38,16 @@ use risingwave_storage::store::*; use crate::get_notification_client_for_test; use crate::local_state_store_test_utils::LocalStateStoreTestExt; -use crate::test_utils::{with_hummock_storage_v2, HummockStateStoreTestTrait, TestIngestBatch}; +use crate::test_utils::{ + gen_key_from_str, with_hummock_storage_v2, HummockStateStoreTestTrait, TestIngestBatch, +}; #[tokio::test] async fn test_empty_read_v2() { let (hummock_storage, _meta_client) = with_hummock_storage_v2(Default::default()).await; assert!(hummock_storage .get( - Bytes::from("test_key"), + gen_key_from_str(VirtualNode::ZERO, "test_key"), u64::MAX, ReadOptions { table_id: TableId { table_id: 2333 }, @@ -82,12 +84,15 @@ async fn test_basic_inner( hummock_storage: impl HummockStateStoreTestTrait, meta_client: Arc, ) { - let anchor = Bytes::from("aa"); + let anchor = gen_key_from_str(VirtualNode::ZERO, "aa"); // First batch inserts the anchor and others. let mut batch1 = vec![ (anchor.clone(), StorageValue::new_put("111")), - (Bytes::from("bb"), StorageValue::new_put("222")), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("222"), + ), ]; // Make sure the batch is sorted. @@ -95,7 +100,10 @@ async fn test_basic_inner( // Second batch modifies the anchor. let mut batch2 = vec![ - (Bytes::from("cc"), StorageValue::new_put("333")), + ( + gen_key_from_str(VirtualNode::ZERO, "cc"), + StorageValue::new_put("333"), + ), (anchor.clone(), StorageValue::new_put("111111")), ]; @@ -104,8 +112,14 @@ async fn test_basic_inner( // Third batch deletes the anchor let mut batch3 = vec![ - (Bytes::from("dd"), StorageValue::new_put("444")), - (Bytes::from("ee"), StorageValue::new_put("555")), + ( + gen_key_from_str(VirtualNode::ZERO, "dd"), + StorageValue::new_put("444"), + ), + ( + gen_key_from_str(VirtualNode::ZERO, "ee"), + StorageValue::new_put("555"), + ), (anchor.clone(), StorageValue::new_delete()), ]; @@ -165,7 +179,7 @@ async fn test_basic_inner( assert_eq!(value, Bytes::from("111")); let value = hummock_storage .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -180,7 +194,7 @@ async fn test_basic_inner( // Test looking for a nonexistent key. `next()` would return the next key. let value = hummock_storage .get( - Bytes::from("ab"), + gen_key_from_str(VirtualNode::ZERO, "ab"), epoch1, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -255,7 +269,7 @@ async fn test_basic_inner( // Get non-existent maximum key. let value = hummock_storage .get( - Bytes::from("ff"), + gen_key_from_str(VirtualNode::ZERO, "ff"), epoch3, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -269,7 +283,10 @@ async fn test_basic_inner( // Write aa bb let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), + ( + Bound::Unbounded, + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "ee")), + ), epoch1, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -313,7 +330,10 @@ async fn test_basic_inner( // Update aa, write cc let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), + ( + Bound::Unbounded, + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "ee")), + ), epoch2, ReadOptions { prefetch_options: PrefetchOptions::new_for_exhaust_iter(), @@ -329,7 +349,10 @@ async fn test_basic_inner( // Delete aa, write dd,ee let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), + ( + Bound::Unbounded, + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "ee")), + ), epoch3, ReadOptions { prefetch_options: PrefetchOptions::new_for_exhaust_iter(), @@ -353,7 +376,7 @@ async fn test_basic_inner( .unwrap(); let value = hummock_storage .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch2, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -366,7 +389,7 @@ async fn test_basic_inner( assert_eq!(value, Bytes::from("222")); let value = hummock_storage .get( - Bytes::from("dd"), + gen_key_from_str(VirtualNode::ZERO, "dd"), epoch2, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -392,8 +415,14 @@ async fn test_state_store_sync_inner( // ingest 16B batch let mut batch1 = vec![ - (Bytes::from("\0\0aaaa"), StorageValue::new_put("1111")), - (Bytes::from("\0\0bbbb"), StorageValue::new_put("2222")), + ( + gen_key_from_str(VirtualNode::ZERO, "\0\0aaaa"), + StorageValue::new_put("1111"), + ), + ( + gen_key_from_str(VirtualNode::ZERO, "\0\0bbbb"), + StorageValue::new_put("2222"), + ), ]; // Make sure the batch is sorted. @@ -418,15 +447,15 @@ async fn test_state_store_sync_inner( // ingest 24B batch let mut batch2 = vec![ ( - Bytes::copy_from_slice(b"\0\0cccc"), + gen_key_from_str(VirtualNode::ZERO, "\0\0cccc"), StorageValue::new_put("3333"), ), ( - Bytes::copy_from_slice(b"\0\0dddd"), + gen_key_from_str(VirtualNode::ZERO, "\0\0dddd"), StorageValue::new_put("4444"), ), ( - Bytes::copy_from_slice(b"\0\0eeee"), + gen_key_from_str(VirtualNode::ZERO, "\0\0eeee"), StorageValue::new_put("5555"), ), ]; @@ -457,7 +486,7 @@ async fn test_state_store_sync_inner( // ingest more 8B then will trigger a sync behind the scene let mut batch3 = vec![( - Bytes::copy_from_slice(b"\0\0eeee"), + gen_key_from_str(VirtualNode::ZERO, "\0\0eeee"), StorageValue::new_put("5555"), )]; batch3.sort_by(|(k1, _), (k2, _)| k1.cmp(k2)); @@ -504,12 +533,15 @@ async fn test_reload_storage() { let (env, hummock_manager_ref, _cluster_manager_ref, worker_node) = setup_compute_env(8080).await; let (hummock_storage, meta_client) = with_hummock_storage_v2(Default::default()).await; - let anchor = Bytes::from("aa"); + let anchor = gen_key_from_str(VirtualNode::ZERO, "aa"); // First batch inserts the anchor and others. let mut batch1 = vec![ (anchor.clone(), StorageValue::new_put("111")), - (Bytes::from("bb"), StorageValue::new_put("222")), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("222"), + ), ]; // Make sure the batch is sorted. @@ -517,7 +549,10 @@ async fn test_reload_storage() { // Second batch modifies the anchor. let mut batch2 = vec![ - (Bytes::from("cc"), StorageValue::new_put("333")), + ( + gen_key_from_str(VirtualNode::ZERO, "cc"), + StorageValue::new_put("333"), + ), (anchor.clone(), StorageValue::new_put("111111")), ]; @@ -570,7 +605,7 @@ async fn test_reload_storage() { // Test looking for a nonexistent key. `next()` would return the next key. let value = hummock_storage .get( - Bytes::from("ab"), + gen_key_from_str(VirtualNode::ZERO, "ab"), epoch1, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -614,7 +649,10 @@ async fn test_reload_storage() { // Write aa bb let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), + ( + Bound::Unbounded, + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "ee")), + ), epoch1, ReadOptions { prefetch_options: PrefetchOptions::new_for_exhaust_iter(), @@ -659,7 +697,10 @@ async fn test_reload_storage() { // Update aa, write cc let iter = hummock_storage .iter( - (Bound::Unbounded, Bound::Included(Bytes::from("ee"))), + ( + Bound::Unbounded, + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "ee")), + ), epoch2, ReadOptions { prefetch_options: PrefetchOptions::new_for_exhaust_iter(), @@ -695,7 +736,7 @@ async fn test_write_anytime_inner( "111".as_bytes(), hummock_storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -710,7 +751,7 @@ async fn test_write_anytime_inner( "222".as_bytes(), hummock_storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb"].concat()), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -725,7 +766,7 @@ async fn test_write_anytime_inner( "333".as_bytes(), hummock_storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc"].concat()), + gen_key_from_str(VirtualNode::ZERO, "cc"), epoch, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -740,12 +781,8 @@ async fn test_write_anytime_inner( let iter = hummock_storage .iter( ( - Bound::Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat(), - )), - Bound::Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc"].concat(), - )), + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "aa")), + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "cc")), ), epoch, ReadOptions { @@ -801,15 +838,15 @@ async fn test_write_anytime_inner( let batch1 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), StorageValue::new_put("111"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb"].concat()), + gen_key_from_str(VirtualNode::ZERO, "bb"), StorageValue::new_put("222"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc"].concat()), + gen_key_from_str(VirtualNode::ZERO, "cc"), StorageValue::new_put("333"), ), ]; @@ -838,7 +875,7 @@ async fn test_write_anytime_inner( "111_new".as_bytes(), hummock_storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -852,7 +889,7 @@ async fn test_write_anytime_inner( assert!(hummock_storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb"].concat()), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -866,7 +903,7 @@ async fn test_write_anytime_inner( "333".as_bytes(), hummock_storage .get( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc"].concat()), + gen_key_from_str(VirtualNode::ZERO, "cc"), epoch, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -880,12 +917,8 @@ async fn test_write_anytime_inner( let iter = hummock_storage .iter( ( - Bound::Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat(), - )), - Bound::Included(Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc"].concat(), - )), + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "aa")), + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "bb")), ), epoch, ReadOptions { @@ -929,11 +962,11 @@ async fn test_write_anytime_inner( // Update aa, delete bb, cc unchanged let batch2 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aa"), StorageValue::new_put("111_new"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb"].concat()), + gen_key_from_str(VirtualNode::ZERO, "bb"), StorageValue::new_delete(), ), ]; @@ -1006,8 +1039,14 @@ async fn test_delete_get_inner( let initial_epoch = hummock_storage.get_pinned_version().max_committed_epoch(); let epoch1 = initial_epoch + 1; let batch1 = vec![ - (Bytes::from("aa"), StorageValue::new_put("111")), - (Bytes::from("bb"), StorageValue::new_put("222")), + ( + gen_key_from_str(VirtualNode::ZERO, "aa"), + StorageValue::new_put("111"), + ), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("222"), + ), ]; let mut local = hummock_storage.new_local(NewLocalOptions::default()).await; local.init_for_test(epoch1).await.unwrap(); @@ -1031,7 +1070,10 @@ async fn test_delete_get_inner( let epoch2 = initial_epoch + 2; local.seal_current_epoch(epoch2); - let batch2 = vec![(Bytes::from("bb"), StorageValue::new_delete())]; + let batch2 = vec![( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_delete(), + )]; local .ingest_batch( batch2, @@ -1056,7 +1098,7 @@ async fn test_delete_get_inner( .unwrap(); assert!(hummock_storage .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch2, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -1081,8 +1123,14 @@ async fn test_multiple_epoch_sync_inner( let initial_epoch = hummock_storage.get_pinned_version().max_committed_epoch(); let epoch1 = initial_epoch + 1; let batch1 = vec![ - (Bytes::from("aa"), StorageValue::new_put("111")), - (Bytes::from("bb"), StorageValue::new_put("222")), + ( + gen_key_from_str(VirtualNode::ZERO, "aa"), + StorageValue::new_put("111"), + ), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("222"), + ), ]; let mut local = hummock_storage.new_local(NewLocalOptions::default()).await; @@ -1101,7 +1149,10 @@ async fn test_multiple_epoch_sync_inner( let epoch2 = initial_epoch + 2; local.seal_current_epoch(epoch2); - let batch2 = vec![(Bytes::from("bb"), StorageValue::new_delete())]; + let batch2 = vec![( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_delete(), + )]; local .ingest_batch( batch2, @@ -1116,8 +1167,14 @@ async fn test_multiple_epoch_sync_inner( let epoch3 = initial_epoch + 3; let batch3 = vec![ - (Bytes::from("aa"), StorageValue::new_put("444")), - (Bytes::from("bb"), StorageValue::new_put("555")), + ( + gen_key_from_str(VirtualNode::ZERO, "aa"), + StorageValue::new_put("444"), + ), + ( + gen_key_from_str(VirtualNode::ZERO, "bb"), + StorageValue::new_put("555"), + ), ]; local.seal_current_epoch(epoch3); local @@ -1138,7 +1195,7 @@ async fn test_multiple_epoch_sync_inner( assert_eq!( hummock_storage_clone .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -1152,7 +1209,7 @@ async fn test_multiple_epoch_sync_inner( ); assert!(hummock_storage_clone .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch2, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -1165,7 +1222,7 @@ async fn test_multiple_epoch_sync_inner( assert_eq!( hummock_storage_clone .get( - Bytes::from("bb"), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch3, ReadOptions { cache_policy: CachePolicy::Fill(CachePriority::High), @@ -1220,10 +1277,18 @@ async fn test_gc_watermark_and_clear_shared_buffer() { let epoch1 = initial_epoch + 1; local_hummock_storage.init_for_test(epoch1).await.unwrap(); local_hummock_storage - .insert(Bytes::from("aa"), Bytes::from("111"), None) + .insert( + gen_key_from_str(VirtualNode::ZERO, "aa"), + Bytes::from("111"), + None, + ) .unwrap(); local_hummock_storage - .insert(Bytes::from("bb"), Bytes::from("222"), None) + .insert( + gen_key_from_str(VirtualNode::ZERO, "bb"), + Bytes::from("222"), + None, + ) .unwrap(); local_hummock_storage.flush(Vec::new()).await.unwrap(); @@ -1237,7 +1302,10 @@ async fn test_gc_watermark_and_clear_shared_buffer() { let epoch2 = initial_epoch + 2; local_hummock_storage.seal_current_epoch(epoch2); local_hummock_storage - .delete(Bytes::from("bb"), Bytes::from("222")) + .delete( + gen_key_from_str(VirtualNode::ZERO, "bb"), + Bytes::from("222"), + ) .unwrap(); local_hummock_storage.flush(Vec::new()).await.unwrap(); @@ -1343,11 +1411,11 @@ async fn test_replicated_local_hummock_storage() { // ingest 16B batch let mut batch1 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"aaaa"].concat()), + gen_key_from_str(VirtualNode::ZERO, "aaaa"), StorageValue::new_put("1111"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"bbbb"].concat()), + gen_key_from_str(VirtualNode::ZERO, "bbbb"), StorageValue::new_put("2222"), ), ]; @@ -1414,11 +1482,11 @@ async fn test_replicated_local_hummock_storage() { // ingest 16B batch let mut batch2 = vec![ ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"cccc"].concat()), + gen_key_from_str(VirtualNode::ZERO, "cccc"), StorageValue::new_put("3333"), ), ( - Bytes::from([VirtualNode::ZERO.to_be_bytes().as_slice(), b"dddd"].concat()), + gen_key_from_str(VirtualNode::ZERO, "dddd"), StorageValue::new_put("4444"), ), ]; diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 7194dd2d963ea..f8388e66343a5 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -21,6 +21,7 @@ use bytes::Bytes; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::hummock::CompactionFilterFlag; use risingwave_common::catalog::TableId; +use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{next_key, user_key}; @@ -39,7 +40,7 @@ use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::hummock::compactor::compactor_runner::compact; use risingwave_storage::hummock::compactor::CompactorContext; use risingwave_storage::hummock::{CachePolicy, GetObjectId, SstableObjectIdManager}; -use risingwave_storage::store::{LocalStateStore, NewLocalOptions, ReadOptions}; +use risingwave_storage::store::{LocalStateStore, NewLocalOptions, ReadOptions, StateStoreRead}; use risingwave_storage::StateStore; use serial_test::serial; @@ -48,6 +49,7 @@ use super::compactor_tests::tests::{ }; use crate::get_notification_client_for_test; use crate::local_state_store_test_utils::LocalStateStoreTestExt; +use crate::test_utils::gen_key_from_bytes; #[tokio::test] #[cfg(feature = "sync_point")] @@ -312,21 +314,33 @@ async fn test_syncpoints_get_in_delete_range_boundary() { for _ in 0..10 { local .insert( - Bytes::copy_from_slice(start_key.as_slice()), + gen_key_from_bytes(VirtualNode::ZERO, start_key.as_slice()), val0.clone(), None, ) .unwrap(); - start_key = next_key(&start_key); + start_key = next_key(start_key.as_slice()); } local - .insert(Bytes::from(b"\0\0ggg".as_slice()), val0.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0ggg"), + val0.clone(), + None, + ) .unwrap(); local - .insert(Bytes::from(b"\0\0hhh".as_slice()), val0.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0hhh"), + val0.clone(), + None, + ) .unwrap(); local - .insert(Bytes::from(b"\0\0kkk".as_slice()), val0.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0kkk"), + val0.clone(), + None, + ) .unwrap(); local.flush(Vec::new()).await.unwrap(); local.seal_current_epoch(101); @@ -339,10 +353,18 @@ async fn test_syncpoints_get_in_delete_range_boundary() { .await; local - .insert(Bytes::from(b"\0\0aaa".as_slice()), val1.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0aaa"), + val1.clone(), + None, + ) .unwrap(); local - .insert(Bytes::from(b"\0\0bbb".as_slice()), val1.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0bbb"), + val1.clone(), + None, + ) .unwrap(); local .flush(vec![( @@ -361,10 +383,18 @@ async fn test_syncpoints_get_in_delete_range_boundary() { .await; local - .insert(Bytes::from(b"\0\0hhh".as_slice()), val1.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0hhh"), + val1.clone(), + None, + ) .unwrap(); local - .insert(Bytes::from(b"\0\0iii".as_slice()), val1.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0iii"), + val1.clone(), + None, + ) .unwrap(); local .flush(vec![( @@ -384,10 +414,18 @@ async fn test_syncpoints_get_in_delete_range_boundary() { .await; local - .insert(Bytes::from(b"\0\0lll".as_slice()), val1.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0lll"), + val1.clone(), + None, + ) .unwrap(); local - .insert(Bytes::from(b"\0\0mmm".as_slice()), val1.clone(), None) + .insert( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0mmm"), + val1.clone(), + None, + ) .unwrap(); local.flush(Vec::new()).await.unwrap(); local.seal_current_epoch(u64::MAX); @@ -424,22 +462,38 @@ async fn test_syncpoints_get_in_delete_range_boundary() { ..Default::default() }; let get_result = storage - .get(Bytes::from("\0\0hhh"), 120, read_options.clone()) + .get( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0hhh"), + 120, + read_options.clone(), + ) .await .unwrap(); assert_eq!(get_result.unwrap(), val1); let get_result = storage - .get(Bytes::from("\0\0ggg"), 120, read_options.clone()) + .get( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0ggg"), + 120, + read_options.clone(), + ) .await .unwrap(); assert!(get_result.is_none()); let get_result = storage - .get(Bytes::from("\0\0aaa"), 120, read_options.clone()) + .get( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0aaa"), + 120, + read_options.clone(), + ) .await .unwrap(); assert_eq!(get_result.unwrap(), val1); let get_result = storage - .get(Bytes::from("\0\0aab"), 120, read_options.clone()) + .get( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0aab"), + 120, + read_options.clone(), + ) .await .unwrap(); assert_eq!(get_result.unwrap(), val0); @@ -452,7 +506,11 @@ async fn test_syncpoints_get_in_delete_range_boundary() { } }); let get_result = storage - .get(Bytes::from("\0\0kkk"), 120, read_options.clone()) + .get( + gen_key_from_bytes(VirtualNode::ZERO, b"\0\0kkk"), + 120, + read_options.clone(), + ) .await .unwrap(); assert_eq!(get_result.unwrap(), val0); diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index d69c835930aa2..cd713e3977777 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -18,8 +18,10 @@ use std::sync::Arc; use bytes::Bytes; use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_common::hash::VirtualNode; use risingwave_common_service::observer_manager::ObserverManager; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_hummock_sdk::key::TableKey; use risingwave_meta::hummock::test_utils::{ register_table_ids_to_compaction_group, setup_compute_env, }; @@ -48,6 +50,16 @@ use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; use crate::mock_notification_client::get_notification_client_for_test; +pub fn gen_key_from_bytes(vnode: VirtualNode, payload: &[u8]) -> TableKey { + TableKey(Bytes::from( + [vnode.to_be_bytes().as_slice(), payload].concat(), + )) +} + +pub fn gen_key_from_str(vnode: VirtualNode, payload: &str) -> TableKey { + gen_key_from_bytes(vnode, payload.as_bytes()) +} + pub async fn prepare_first_valid_version( env: MetaSrvEnv, hummock_manager_ref: HummockManagerRef, @@ -91,7 +103,7 @@ pub async fn prepare_first_valid_version( pub trait TestIngestBatch: LocalStateStore { async fn ingest_batch( &mut self, - kv_pairs: Vec<(Bytes, StorageValue)>, + kv_pairs: Vec<(TableKey, StorageValue)>, delete_ranges: Vec<(Bound, Bound)>, write_options: WriteOptions, ) -> StorageResult; @@ -101,7 +113,7 @@ pub trait TestIngestBatch: LocalStateStore { impl TestIngestBatch for S { async fn ingest_batch( &mut self, - kv_pairs: Vec<(Bytes, StorageValue)>, + kv_pairs: Vec<(TableKey, StorageValue)>, delete_ranges: Vec<(Bound, Bound)>, write_options: WriteOptions, ) -> StorageResult { diff --git a/src/storage/src/hummock/compactor/shared_buffer_compact.rs b/src/storage/src/hummock/compactor/shared_buffer_compact.rs index bcbaa19e2c3b1..0eca74f1dcaba 100644 --- a/src/storage/src/hummock/compactor/shared_buffer_compact.rs +++ b/src/storage/src/hummock/compactor/shared_buffer_compact.rs @@ -41,7 +41,6 @@ use crate::hummock::shared_buffer::shared_buffer_batch::{ SharedBufferBatch, SharedBufferBatchInner, SharedBufferVersionedEntry, }; use crate::hummock::sstable::CompactionDeleteRangesBuilder; -use crate::hummock::store::memtable::ImmutableMemtable; use crate::hummock::utils::MemoryTracker; use crate::hummock::value::HummockValue; use crate::hummock::{ @@ -49,6 +48,7 @@ use crate::hummock::{ CompactionDeleteRanges, GetObjectId, HummockError, HummockResult, SstableBuilderOptions, SstableObjectIdManagerRef, }; +use crate::mem_table::ImmutableMemtable; const GC_DELETE_KEYS_FOR_FLUSH: bool = false; const GC_WATERMARK_FOR_FLUSH: u64 = 0; diff --git a/src/storage/src/hummock/error.rs b/src/storage/src/hummock/error.rs index efd25c8076383..3243beb0cdcda 100644 --- a/src/storage/src/hummock/error.rs +++ b/src/storage/src/hummock/error.rs @@ -38,8 +38,6 @@ enum HummockErrorInner { ObjectIoError(Box), #[error("Meta error {0}.")] MetaError(String), - #[error("Invalid WriteBatch.")] - InvalidWriteBatch, #[error("SharedBuffer error {0}.")] SharedBufferError(String), #[error("Wait epoch error {0}.")] @@ -105,10 +103,6 @@ impl HummockError { HummockErrorInner::MetaError(error.to_string()).into() } - pub fn invalid_write_batch() -> HummockError { - HummockErrorInner::InvalidWriteBatch.into() - } - pub fn shared_buffer_error(error: impl ToString) -> HummockError { HummockErrorInner::SharedBufferError(error.to_string()).into() } diff --git a/src/storage/src/hummock/event_handler/mod.rs b/src/storage/src/hummock/event_handler/mod.rs index 7b9bfd09835cd..4a80ff1a51033 100644 --- a/src/storage/src/hummock/event_handler/mod.rs +++ b/src/storage/src/hummock/event_handler/mod.rs @@ -22,8 +22,8 @@ use risingwave_pb::hummock::version_update_payload; use tokio::sync::{mpsc, oneshot}; use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferBatch; -use crate::hummock::store::memtable::ImmutableMemtable; use crate::hummock::HummockResult; +use crate::mem_table::ImmutableMemtable; use crate::store::SyncResult; pub mod hummock_event_handler; diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index f57ac33bfe6a2..995a9d181e2f5 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -36,10 +36,10 @@ use crate::hummock::compactor::{merge_imms_in_memory, CompactionExecutor}; use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::LocalInstanceId; use crate::hummock::local_version::pinned_version::PinnedVersion; -use crate::hummock::store::memtable::{ImmId, ImmutableMemtable}; use crate::hummock::store::version::StagingSstableInfo; use crate::hummock::utils::MemoryTracker; -use crate::hummock::{HummockError, HummockResult}; +use crate::hummock::{HummockError, HummockResult, ImmutableMemtable}; +use crate::mem_table::ImmId; use crate::monitor::HummockStateStoreMetrics; use crate::opts::StorageOpts; @@ -1044,9 +1044,9 @@ mod tests { }; use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferBatch; - use crate::hummock::store::memtable::{ImmId, ImmutableMemtable}; use crate::hummock::value::HummockValue; use crate::hummock::{HummockError, HummockResult, MemoryLimiter}; + use crate::mem_table::{ImmId, ImmutableMemtable}; use crate::monitor::HummockStateStoreMetrics; use crate::opts::StorageOpts; use crate::storage_value::StorageValue; @@ -1081,7 +1081,7 @@ mod tests { limiter: Option<&MemoryLimiter>, ) -> ImmutableMemtable { let sorted_items = SharedBufferBatch::build_shared_buffer_item_batches(vec![( - Bytes::from(dummy_table_key()), + TableKey(Bytes::from(dummy_table_key())), StorageValue::new_delete(), )]); let size = SharedBufferBatch::measure_batch_size(&sorted_items); diff --git a/src/storage/src/hummock/iterator/merge_inner.rs b/src/storage/src/hummock/iterator/merge_inner.rs index cf92df72abcdf..8a4e29ae58633 100644 --- a/src/storage/src/hummock/iterator/merge_inner.rs +++ b/src/storage/src/hummock/iterator/merge_inner.rs @@ -137,7 +137,7 @@ impl OrderedMergeIteratorInner { impl OrderedMergeIteratorInner> { /// Used in `merge_imms_in_memory` to merge immutable memtables. - pub fn current_item(&self) -> (Bytes, (HummockEpoch, HummockValue)) { + pub fn current_item(&self) -> (TableKey, (HummockEpoch, HummockValue)) { let item = self .heap .peek() diff --git a/src/storage/src/hummock/iterator/mod.rs b/src/storage/src/hummock/iterator/mod.rs index d1e61a8621b57..2a20039eecba8 100644 --- a/src/storage/src/hummock/iterator/mod.rs +++ b/src/storage/src/hummock/iterator/mod.rs @@ -87,7 +87,6 @@ pub trait HummockIterator: Send + 'static { /// /// # Panics /// This function will panic if the iterator is invalid. - // TODO: Add lifetime fn key(&self) -> FullKey<&[u8]>; /// Retrieves the current value, decoded as [`HummockValue`]. @@ -99,7 +98,6 @@ pub trait HummockIterator: Send + 'static { /// # Panics /// This function will panic if the iterator is invalid, or the value cannot be decoded into /// [`HummockValue`]. - // TODO: Add lifetime fn value(&self) -> HummockValue<&[u8]>; /// Indicates whether the iterator can be used. diff --git a/src/storage/src/hummock/iterator/test_utils.rs b/src/storage/src/hummock/iterator/test_utils.rs index 175079da96ce4..a11e46e879dda 100644 --- a/src/storage/src/hummock/iterator/test_utils.rs +++ b/src/storage/src/hummock/iterator/test_utils.rs @@ -19,7 +19,7 @@ use bytes::Bytes; use itertools::Itertools; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; -use risingwave_hummock_sdk::key::{FullKey, UserKey}; +use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKey}; use risingwave_hummock_sdk::{HummockEpoch, HummockSstableObjectId}; use risingwave_object_store::object::{ InMemObjectStore, ObjectStore, ObjectStoreImpl, ObjectStoreRef, @@ -118,10 +118,10 @@ pub fn iterator_test_value_of(idx: usize) -> Vec { pub fn transform_shared_buffer( batches: Vec<(Vec, HummockValue)>, -) -> Vec<(Bytes, HummockValue)> { +) -> Vec<(TableKey, HummockValue)> { batches .into_iter() - .map(|(k, v)| (k.into(), v)) + .map(|(k, v)| (TableKey(k.into()), v)) .collect_vec() } diff --git a/src/storage/src/hummock/mod.rs b/src/storage/src/hummock/mod.rs index 0e93eeff782bc..f0f9f981c393f 100644 --- a/src/storage/src/hummock/mod.rs +++ b/src/storage/src/hummock/mod.rs @@ -14,21 +14,14 @@ //! Hummock is the state store of the streaming system. -use std::ops::{Bound, Deref}; -use std::sync::atomic::AtomicU64; +use std::ops::Bound; use std::sync::Arc; use std::time::Duration; -use arc_swap::ArcSwap; use bytes::Bytes; use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKeyRangeRef}; use risingwave_hummock_sdk::{HummockEpoch, *}; -#[cfg(any(test, feature = "test"))] -use risingwave_pb::hummock::HummockVersion; -use risingwave_pb::hummock::{version_update_payload, SstableInfo}; -use risingwave_rpc_client::HummockMetaClient; -use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; -use tracing::log::error; +use risingwave_pb::hummock::SstableInfo; mod block_cache; pub use block_cache::*; @@ -50,7 +43,6 @@ pub mod hummock_meta_client; pub mod iterator; pub mod shared_buffer; pub mod sstable_store; -mod state_store; #[cfg(any(test, feature = "test"))] pub mod test_utils; pub mod utils; @@ -60,6 +52,7 @@ pub mod event_handler; pub mod local_version; pub mod observer_manager; pub mod store; +pub use store::*; pub mod vacuum; mod validator; pub mod value; @@ -67,306 +60,14 @@ pub mod write_limiter; pub use error::*; pub use risingwave_common::cache::{CacheableEntry, LookupResult, LruCache}; -use risingwave_common_service::observer_manager::{NotificationClient, ObserverManager}; pub use validator::*; use value::*; -use self::event_handler::refiller::CacheRefillConfig; -use self::event_handler::ReadVersionMappingType; use self::iterator::HummockIterator; pub use self::sstable_store::*; -use super::monitor::HummockStateStoreMetrics; -use crate::filter_key_extractor::FilterKeyExtractorManager; -use crate::hummock::backup_reader::{BackupReader, BackupReaderRef}; -use crate::hummock::compactor::CompactorContext; -use crate::hummock::event_handler::hummock_event_handler::BufferTracker; -use crate::hummock::event_handler::{HummockEvent, HummockEventHandler}; -use crate::hummock::local_version::pinned_version::{start_pinned_version_worker, PinnedVersion}; -use crate::hummock::observer_manager::HummockObserverNode; -use crate::hummock::store::memtable::ImmutableMemtable; -use crate::hummock::store::version::HummockVersionReader; -use crate::hummock::write_limiter::{WriteLimiter, WriteLimiterRef}; -use crate::monitor::{CompactorMetrics, StoreLocalStatistic}; -use crate::store::{NewLocalOptions, ReadOptions}; - -struct HummockStorageShutdownGuard { - shutdown_sender: UnboundedSender, -} - -impl Drop for HummockStorageShutdownGuard { - fn drop(&mut self) { - let _ = self - .shutdown_sender - .send(HummockEvent::Shutdown) - .inspect_err(|e| error!("unable to send shutdown: {:?}", e)); - } -} - -/// Hummock is the state store backend. -#[derive(Clone)] -pub struct HummockStorage { - hummock_event_sender: UnboundedSender, - - context: CompactorContext, - - sstable_object_id_manager: SstableObjectIdManagerRef, - - buffer_tracker: BufferTracker, - - version_update_notifier_tx: Arc>, - - seal_epoch: Arc, - - pinned_version: Arc>, - - hummock_version_reader: HummockVersionReader, - - _shutdown_guard: Arc, - - read_version_mapping: Arc, - - backup_reader: BackupReaderRef, - - /// current_epoch < min_current_epoch cannot be read. - min_current_epoch: Arc, - - write_limiter: WriteLimiterRef, -} - -impl HummockStorage { - /// Creates a [`HummockStorage`]. - #[allow(clippy::too_many_arguments)] - pub async fn new( - options: Arc, - sstable_store: SstableStoreRef, - hummock_meta_client: Arc, - notification_client: impl NotificationClient, - filter_key_extractor_manager: Arc, - state_store_metrics: Arc, - compactor_metrics: Arc, - ) -> HummockResult { - let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( - hummock_meta_client.clone(), - options.sstable_id_remote_fetch_number, - )); - let backup_reader = BackupReader::new( - &options.backup_storage_url, - &options.backup_storage_directory, - ) - .await - .map_err(HummockError::read_backup_error)?; - let write_limiter = Arc::new(WriteLimiter::default()); - let (event_tx, mut event_rx) = unbounded_channel(); - - let observer_manager = ObserverManager::new( - notification_client, - HummockObserverNode::new( - filter_key_extractor_manager.clone(), - backup_reader.clone(), - event_tx.clone(), - write_limiter.clone(), - ), - ) - .await; - observer_manager.start().await; - - let hummock_version = match event_rx.recv().await { - Some(HummockEvent::VersionUpdate(version_update_payload::Payload::PinnedVersion(version))) => version, - _ => unreachable!("the hummock observer manager is the first one to take the event tx. Should be full hummock version") - }; - - let (pin_version_tx, pin_version_rx) = unbounded_channel(); - let pinned_version = PinnedVersion::new(hummock_version, pin_version_tx); - tokio::spawn(start_pinned_version_worker( - pin_version_rx, - hummock_meta_client.clone(), - )); - - let compactor_context = CompactorContext::new_local_compact_context( - options.clone(), - sstable_store.clone(), - compactor_metrics.clone(), - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - filter_key_extractor_manager.clone(), - ), - ); - - let seal_epoch = Arc::new(AtomicU64::new(pinned_version.max_committed_epoch())); - let min_current_epoch = Arc::new(AtomicU64::new(pinned_version.max_committed_epoch())); - let hummock_event_handler = HummockEventHandler::new( - event_tx.clone(), - event_rx, - pinned_version, - compactor_context.clone(), - sstable_object_id_manager.clone(), - state_store_metrics.clone(), - CacheRefillConfig { - timeout: Duration::from_millis(options.cache_refill_timeout_ms), - data_refill_levels: options - .cache_refill_data_refill_levels - .iter() - .copied() - .collect(), - concurrency: options.cache_refill_concurrency, - }, - ); - - let instance = Self { - context: compactor_context, - sstable_object_id_manager, - buffer_tracker: hummock_event_handler.buffer_tracker().clone(), - version_update_notifier_tx: hummock_event_handler.version_update_notifier_tx(), - seal_epoch, - hummock_event_sender: event_tx.clone(), - pinned_version: hummock_event_handler.pinned_version(), - hummock_version_reader: HummockVersionReader::new( - sstable_store, - state_store_metrics.clone(), - ), - _shutdown_guard: Arc::new(HummockStorageShutdownGuard { - shutdown_sender: event_tx, - }), - read_version_mapping: hummock_event_handler.read_version_mapping(), - backup_reader, - min_current_epoch, - write_limiter, - }; - - tokio::spawn(hummock_event_handler.start_hummock_event_handler_worker()); - - Ok(instance) - } - - async fn new_local_inner(&self, option: NewLocalOptions) -> LocalHummockStorage { - let (tx, rx) = tokio::sync::oneshot::channel(); - self.hummock_event_sender - .send(HummockEvent::RegisterReadVersion { - table_id: option.table_id, - new_read_version_sender: tx, - is_replicated: option.is_replicated, - }) - .unwrap(); - - let (basic_read_version, instance_guard) = rx.await.unwrap(); - let version_update_notifier_tx = self.version_update_notifier_tx.clone(); - LocalHummockStorage::new( - instance_guard, - basic_read_version, - self.hummock_version_reader.clone(), - self.hummock_event_sender.clone(), - self.buffer_tracker.get_memory_limiter().clone(), - self.write_limiter.clone(), - option, - version_update_notifier_tx, - ) - } - - pub fn sstable_store(&self) -> SstableStoreRef { - self.context.sstable_store.clone() - } - - pub fn sstable_object_id_manager(&self) -> &SstableObjectIdManagerRef { - &self.sstable_object_id_manager - } - - pub fn filter_key_extractor_manager(&self) -> &FilterKeyExtractorManager { - &self.context.filter_key_extractor_manager - } - - pub fn get_memory_limiter(&self) -> Arc { - self.buffer_tracker.get_memory_limiter().clone() - } - - pub fn get_pinned_version(&self) -> PinnedVersion { - self.pinned_version.load().deref().deref().clone() - } - - pub fn backup_reader(&self) -> BackupReaderRef { - self.backup_reader.clone() - } -} - -#[cfg(any(test, feature = "test"))] -impl HummockStorage { - /// Used in the compaction test tool - pub async fn update_version_and_wait(&self, version: HummockVersion) { - use tokio::task::yield_now; - let version_id = version.id; - self.hummock_event_sender - .send(HummockEvent::VersionUpdate( - version_update_payload::Payload::PinnedVersion(version), - )) - .unwrap(); - loop { - if self.pinned_version.load().id() >= version_id { - break; - } - - yield_now().await - } - } - - pub async fn wait_version(&self, version: HummockVersion) { - use tokio::task::yield_now; - loop { - if self.pinned_version.load().id() >= version.id { - break; - } - - yield_now().await - } - } - - pub fn get_shared_buffer_size(&self) -> usize { - self.buffer_tracker.get_buffer_size() - } - - pub async fn try_wait_epoch_for_test(&self, wait_epoch: u64) { - let mut rx = self.version_update_notifier_tx.subscribe(); - while *(rx.borrow_and_update()) < wait_epoch { - rx.changed().await.unwrap(); - } - } - - /// Creates a [`HummockStorage`] with default stats. Should only be used by tests. - pub async fn for_test( - options: Arc, - sstable_store: SstableStoreRef, - hummock_meta_client: Arc, - notification_client: impl NotificationClient, - ) -> HummockResult { - Self::new( - options, - sstable_store, - hummock_meta_client, - notification_client, - Arc::new(RpcFilterKeyExtractorManager::default()), - Arc::new(HummockStateStoreMetrics::unused()), - Arc::new(CompactorMetrics::unused()), - ) - .await - } - - pub fn storage_opts(&self) -> &Arc { - &self.context.storage_opts - } - - pub fn version_reader(&self) -> &HummockVersionReader { - &self.hummock_version_reader - } - - #[cfg(any(test, feature = "test"))] - pub async fn wait_version_update(&self, old_id: u64) -> u64 { - use tokio::task::yield_now; - loop { - let cur_id = self.pinned_version.load().id(); - if cur_id > old_id { - return cur_id; - } - yield_now().await; - } - } -} +use crate::mem_table::ImmutableMemtable; +use crate::monitor::StoreLocalStatistic; +use crate::store::ReadOptions; pub async fn get_from_sstable_info( sstable_store_ref: SstableStoreRef, diff --git a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs index 7e67b87434e4a..8a3ab574ef3d5 100644 --- a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs +++ b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs @@ -32,10 +32,10 @@ use crate::hummock::iterator::{ Backward, DeleteRangeIterator, DirectionEnum, Forward, HummockIterator, HummockIteratorDirection, }; -use crate::hummock::store::memtable::ImmId; use crate::hummock::utils::{range_overlap, MemoryTracker}; use crate::hummock::value::HummockValue; use crate::hummock::{HummockEpoch, HummockResult, MonotonicDeleteEvent}; +use crate::mem_table::ImmId; use crate::storage_value::StorageValue; use crate::store::ReadOptions; @@ -51,13 +51,13 @@ fn whether_update_largest_key, Q: AsRef<[u8]>>( } /// The key is `table_key`, which does not contain table id or epoch. -pub(crate) type SharedBufferItem = (Bytes, HummockValue); +pub(crate) type SharedBufferItem = (TableKey, HummockValue); pub type SharedBufferBatchId = u64; /// A shared buffer may contain data from multiple epochs, /// there are multiple versions for a given key (`table_key`), we put those versions into a vector /// and sort them in descending order, aka newest to oldest. -pub type SharedBufferVersionedEntry = (Bytes, Vec<(HummockEpoch, HummockValue)>); +pub type SharedBufferVersionedEntry = (TableKey, Vec<(HummockEpoch, HummockValue)>); type PointRangePair = (PointRange>, PointRange>); struct SharedBufferDeleteRangeMeta { @@ -142,11 +142,11 @@ impl SharedBufferBatchInner { if let Some(item) = payload.last() { if whether_update_largest_key(&largest_table_key, &item.0) { - largest_table_key = Bound::Included(item.0.clone()); + largest_table_key = Bound::Included(item.0.clone().0); } } if let Some(item) = payload.first() { - if smallest_empty || item.0.lt(&smallest_table_key.as_ref()) { + if smallest_empty || item.0.as_ref().lt(smallest_table_key.as_ref()) { smallest_table_key.clear(); smallest_table_key.extend_from_slice(item.0.as_ref()); } @@ -562,7 +562,7 @@ impl SharedBufferBatch { } pub fn build_shared_buffer_item_batches( - kv_pairs: Vec<(Bytes, StorageValue)>, + kv_pairs: Vec<(TableKey, StorageValue)>, ) -> Vec { kv_pairs .into_iter() @@ -675,7 +675,7 @@ impl SharedBufferBatchIterator { } } - pub(crate) fn current_item(&self) -> (&Bytes, &(HummockEpoch, HummockValue)) { + pub(crate) fn current_item(&self) -> (&TableKey, &(HummockEpoch, HummockValue)) { assert!(self.is_valid(), "iterator is not valid"); let (idx, version_idx) = match D::direction() { DirectionEnum::Forward => (self.current_idx, self.current_version_idx), diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs new file mode 100644 index 0000000000000..e5b3c95867d3c --- /dev/null +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -0,0 +1,573 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::future::Future; +use std::ops::{Bound, Deref}; +use std::sync::atomic::{AtomicU64, Ordering as MemOrdering}; +use std::sync::Arc; +use std::time::Duration; + +use arc_swap::ArcSwap; +use bytes::Bytes; +use itertools::Itertools; +use more_asserts::assert_gt; +use risingwave_common::catalog::TableId; +use risingwave_common_service::observer_manager::{NotificationClient, ObserverManager}; +use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; +use risingwave_hummock_sdk::HummockReadEpoch; +#[cfg(any(test, feature = "test"))] +use risingwave_pb::hummock::HummockVersion; +use risingwave_pb::hummock::{version_update_payload, SstableInfo}; +use risingwave_rpc_client::HummockMetaClient; +use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; +use tokio::sync::oneshot; +use tracing::log::error; + +use super::local_hummock_storage::{HummockStorageIterator, LocalHummockStorage}; +use super::version::{CommittedVersion, HummockVersionReader}; +use crate::error::StorageResult; +use crate::filter_key_extractor::{FilterKeyExtractorManager, FilterKeyExtractorManagerRef}; +use crate::hummock::backup_reader::{BackupReader, BackupReaderRef}; +use crate::hummock::compactor::CompactorContext; +use crate::hummock::event_handler::hummock_event_handler::BufferTracker; +use crate::hummock::event_handler::{HummockEvent, HummockEventHandler, ReadVersionMappingType}; +use crate::hummock::local_version::pinned_version::{start_pinned_version_worker, PinnedVersion}; +use crate::hummock::observer_manager::HummockObserverNode; +use crate::hummock::store::version::read_filter_for_batch; +use crate::hummock::utils::{validate_safe_epoch, wait_for_epoch}; +use crate::hummock::write_limiter::{WriteLimiter, WriteLimiterRef}; +use crate::hummock::{ + HummockEpoch, HummockError, HummockResult, MemoryLimiter, SstableObjectIdManager, + SstableObjectIdManagerRef, SstableStoreRef, +}; +use crate::mem_table::ImmutableMemtable; +use crate::monitor::{CompactorMetrics, HummockStateStoreMetrics, StoreLocalStatistic}; +use crate::opts::StorageOpts; +use crate::store::*; +use crate::StateStore; + +struct HummockStorageShutdownGuard { + shutdown_sender: UnboundedSender, +} + +impl Drop for HummockStorageShutdownGuard { + fn drop(&mut self) { + let _ = self + .shutdown_sender + .send(HummockEvent::Shutdown) + .inspect_err(|e| error!("unable to send shutdown: {:?}", e)); + } +} + +/// `HummockStorage` is the entry point of the Hummock state store backend. +/// It implements the `StateStore` and `StateStoreRead` traits but not the `StateStoreWrite` trait +/// since all writes should be done via `LocalHummockStorage` to ensure the single writer property +/// of hummock. `LocalHummockStorage` instance can be created via `new_local` call. +/// Hummock is the state store backend. +#[derive(Clone)] +pub struct HummockStorage { + hummock_event_sender: UnboundedSender, + + context: CompactorContext, + + sstable_object_id_manager: SstableObjectIdManagerRef, + + buffer_tracker: BufferTracker, + + version_update_notifier_tx: Arc>, + + seal_epoch: Arc, + + pinned_version: Arc>, + + hummock_version_reader: HummockVersionReader, + + _shutdown_guard: Arc, + + read_version_mapping: Arc, + + backup_reader: BackupReaderRef, + + /// current_epoch < min_current_epoch cannot be read. + min_current_epoch: Arc, + + write_limiter: WriteLimiterRef, +} + +impl HummockStorage { + /// Creates a [`HummockStorage`]. + #[allow(clippy::too_many_arguments)] + pub async fn new( + options: Arc, + sstable_store: SstableStoreRef, + hummock_meta_client: Arc, + notification_client: impl NotificationClient, + filter_key_extractor_manager: Arc, + state_store_metrics: Arc, + compactor_metrics: Arc, + ) -> HummockResult { + let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( + hummock_meta_client.clone(), + options.sstable_id_remote_fetch_number, + )); + let backup_reader = BackupReader::new( + &options.backup_storage_url, + &options.backup_storage_directory, + ) + .await + .map_err(HummockError::read_backup_error)?; + let write_limiter = Arc::new(WriteLimiter::default()); + let (event_tx, mut event_rx) = unbounded_channel(); + + let observer_manager = ObserverManager::new( + notification_client, + HummockObserverNode::new( + filter_key_extractor_manager.clone(), + backup_reader.clone(), + event_tx.clone(), + write_limiter.clone(), + ), + ) + .await; + observer_manager.start().await; + + let hummock_version = match event_rx.recv().await { + Some(HummockEvent::VersionUpdate(version_update_payload::Payload::PinnedVersion(version))) => version, + _ => unreachable!("the hummock observer manager is the first one to take the event tx. Should be full hummock version") + }; + + let (pin_version_tx, pin_version_rx) = unbounded_channel(); + let pinned_version = PinnedVersion::new(hummock_version, pin_version_tx); + tokio::spawn(start_pinned_version_worker( + pin_version_rx, + hummock_meta_client.clone(), + )); + + let compactor_context = CompactorContext::new_local_compact_context( + options.clone(), + sstable_store.clone(), + compactor_metrics.clone(), + FilterKeyExtractorManager::RpcFilterKeyExtractorManager( + filter_key_extractor_manager.clone(), + ), + ); + + let seal_epoch = Arc::new(AtomicU64::new(pinned_version.max_committed_epoch())); + let min_current_epoch = Arc::new(AtomicU64::new(pinned_version.max_committed_epoch())); + let hummock_event_handler = HummockEventHandler::new( + event_tx.clone(), + event_rx, + pinned_version, + compactor_context.clone(), + sstable_object_id_manager.clone(), + state_store_metrics.clone(), + CacheRefillConfig { + timeout: Duration::from_millis(options.cache_refill_timeout_ms), + data_refill_levels: options + .cache_refill_data_refill_levels + .iter() + .copied() + .collect(), + concurrency: options.cache_refill_concurrency, + }, + ); + + let instance = Self { + context: compactor_context, + sstable_object_id_manager, + buffer_tracker: hummock_event_handler.buffer_tracker().clone(), + version_update_notifier_tx: hummock_event_handler.version_update_notifier_tx(), + seal_epoch, + hummock_event_sender: event_tx.clone(), + pinned_version: hummock_event_handler.pinned_version(), + hummock_version_reader: HummockVersionReader::new( + sstable_store, + state_store_metrics.clone(), + ), + _shutdown_guard: Arc::new(HummockStorageShutdownGuard { + shutdown_sender: event_tx, + }), + read_version_mapping: hummock_event_handler.read_version_mapping(), + backup_reader, + min_current_epoch, + write_limiter, + }; + + tokio::spawn(hummock_event_handler.start_hummock_event_handler_worker()); + + Ok(instance) + } + + /// Gets the value of a specified `key` in the table specified in `read_options`. + /// The result is based on a snapshot corresponding to the given `epoch`. + /// if `key` has consistent hash virtual node value, then such value is stored in `value_meta` + /// + /// If `Ok(Some())` is returned, the key is found. If `Ok(None)` is returned, + /// the key is not found. If `Err()` is returned, the searching for the key + /// failed due to other non-EOF errors. + async fn get_inner( + &self, + key: TableKey, + epoch: HummockEpoch, + read_options: ReadOptions, + ) -> StorageResult> { + let key_range = (Bound::Included(key.clone()), Bound::Included(key.clone())); + + let read_version_tuple = if read_options.read_version_from_backup { + self.build_read_version_tuple_from_backup(epoch).await? + } else { + self.build_read_version_tuple(epoch, read_options.table_id, &key_range)? + }; + + self.hummock_version_reader + .get(key, epoch, read_options, read_version_tuple) + .await + } + + async fn iter_inner( + &self, + key_range: TableKeyRange, + epoch: u64, + read_options: ReadOptions, + ) -> StorageResult> { + let read_version_tuple = if read_options.read_version_from_backup { + self.build_read_version_tuple_from_backup(epoch).await? + } else { + self.build_read_version_tuple(epoch, read_options.table_id, &key_range)? + }; + + self.hummock_version_reader + .iter(key_range, epoch, read_options, read_version_tuple) + .await + } + + async fn build_read_version_tuple_from_backup( + &self, + epoch: u64, + ) -> StorageResult<(Vec, Vec, CommittedVersion)> { + match self.backup_reader.try_get_hummock_version(epoch).await { + Ok(Some(backup_version)) => { + validate_safe_epoch(backup_version.safe_epoch(), epoch)?; + Ok((Vec::default(), Vec::default(), backup_version)) + } + Ok(None) => Err(HummockError::read_backup_error(format!( + "backup include epoch {} not found", + epoch + )) + .into()), + Err(e) => Err(e), + } + } + + fn build_read_version_tuple( + &self, + epoch: u64, + table_id: TableId, + key_range: &TableKeyRange, + ) -> StorageResult<(Vec, Vec, CommittedVersion)> { + let pinned_version = self.pinned_version.load(); + validate_safe_epoch(pinned_version.safe_epoch(), epoch)?; + + // check epoch if lower mce + let read_version_tuple: (Vec, Vec, CommittedVersion) = + if epoch <= pinned_version.max_committed_epoch() { + // read committed_version directly without build snapshot + (Vec::default(), Vec::default(), (**pinned_version).clone()) + } else { + let read_version_vec = { + let read_guard = self.read_version_mapping.read(); + read_guard + .get(&table_id) + .map(|v| { + v.values() + .filter(|v| !v.read_arc().is_replicated()) + .cloned() + .collect_vec() + }) + .unwrap_or(Vec::new()) + }; + + // When the system has just started and no state has been created, the memory state + // may be empty + if read_version_vec.is_empty() { + (Vec::default(), Vec::default(), (**pinned_version).clone()) + } else { + let (imm_vec, sst_vec) = + read_filter_for_batch(epoch, table_id, key_range, read_version_vec)?; + let committed_version = (**pinned_version).clone(); + + (imm_vec, sst_vec, committed_version) + } + }; + + Ok(read_version_tuple) + } + + async fn new_local_inner(&self, option: NewLocalOptions) -> LocalHummockStorage { + let (tx, rx) = tokio::sync::oneshot::channel(); + self.hummock_event_sender + .send(HummockEvent::RegisterReadVersion { + table_id: option.table_id, + new_read_version_sender: tx, + is_replicated: option.is_replicated, + }) + .unwrap(); + + let (basic_read_version, instance_guard) = rx.await.unwrap(); + let version_update_notifier_tx = self.version_update_notifier_tx.clone(); + LocalHummockStorage::new( + instance_guard, + basic_read_version, + self.hummock_version_reader.clone(), + self.hummock_event_sender.clone(), + self.buffer_tracker.get_memory_limiter().clone(), + self.write_limiter.clone(), + option, + version_update_notifier_tx, + ) + } + + pub fn sstable_store(&self) -> SstableStoreRef { + self.context.sstable_store.clone() + } + + pub fn sstable_object_id_manager(&self) -> &SstableObjectIdManagerRef { + &self.sstable_object_id_manager + } + + pub fn filter_key_extractor_manager(&self) -> &FilterKeyExtractorManager { + &self.context.filter_key_extractor_manager + } + + pub fn get_memory_limiter(&self) -> Arc { + self.buffer_tracker.get_memory_limiter().clone() + } + + pub fn get_pinned_version(&self) -> PinnedVersion { + self.pinned_version.load().deref().deref().clone() + } + + pub fn backup_reader(&self) -> BackupReaderRef { + self.backup_reader.clone() + } +} + +impl StateStoreRead for HummockStorage { + type IterStream = StreamTypeOfIter; + + fn get( + &self, + key: TableKey, + epoch: u64, + read_options: ReadOptions, + ) -> impl Future>> + '_ { + self.get_inner(key, epoch, read_options) + } + + fn iter( + &self, + key_range: TableKeyRange, + epoch: u64, + read_options: ReadOptions, + ) -> impl Future> + '_ { + self.iter_inner(key_range, epoch, read_options) + } +} + +impl StateStore for HummockStorage { + type Local = LocalHummockStorage; + + /// Waits until the local hummock version contains the epoch. If `wait_epoch` is `Current`, + /// we will only check whether it is le `sealed_epoch` and won't wait. + async fn try_wait_epoch(&self, wait_epoch: HummockReadEpoch) -> StorageResult<()> { + self.validate_read_epoch(wait_epoch)?; + let wait_epoch = match wait_epoch { + HummockReadEpoch::Committed(epoch) => { + assert_ne!(epoch, HummockEpoch::MAX, "epoch should not be u64::MAX"); + epoch + } + _ => return Ok(()), + }; + wait_for_epoch(&self.version_update_notifier_tx, wait_epoch).await + } + + async fn sync(&self, epoch: u64) -> StorageResult { + let (tx, rx) = oneshot::channel(); + self.hummock_event_sender + .send(HummockEvent::AwaitSyncEpoch { + new_sync_epoch: epoch, + sync_result_sender: tx, + }) + .expect("should send success"); + Ok(rx.await.expect("should wait success")?) + } + + fn seal_epoch(&self, epoch: u64, is_checkpoint: bool) { + // Update `seal_epoch` synchronously, + // as `HummockEvent::SealEpoch` is handled asynchronously. + let prev_epoch = self.seal_epoch.swap(epoch, MemOrdering::SeqCst); + assert_gt!(epoch, prev_epoch); + + if is_checkpoint { + let _ = self.min_current_epoch.compare_exchange( + HummockEpoch::MAX, + epoch, + MemOrdering::SeqCst, + MemOrdering::SeqCst, + ); + } + self.hummock_event_sender + .send(HummockEvent::SealEpoch { + epoch, + is_checkpoint, + }) + .expect("should send success"); + StoreLocalStatistic::flush_all(); + } + + async fn clear_shared_buffer(&self) -> StorageResult<()> { + let (tx, rx) = oneshot::channel(); + self.hummock_event_sender + .send(HummockEvent::Clear(tx)) + .expect("should send success"); + rx.await.expect("should wait success"); + + let epoch = self.pinned_version.load().max_committed_epoch(); + self.min_current_epoch + .store(HummockEpoch::MAX, MemOrdering::SeqCst); + self.seal_epoch.store(epoch, MemOrdering::SeqCst); + + Ok(()) + } + + fn new_local(&self, option: NewLocalOptions) -> impl Future + Send + '_ { + self.new_local_inner(option) + } + + fn validate_read_epoch(&self, epoch: HummockReadEpoch) -> StorageResult<()> { + if let HummockReadEpoch::Current(read_current_epoch) = epoch { + assert_ne!( + read_current_epoch, + HummockEpoch::MAX, + "epoch should not be u64::MAX" + ); + let sealed_epoch = self.seal_epoch.load(MemOrdering::SeqCst); + if read_current_epoch > sealed_epoch { + tracing::warn!( + "invalid barrier read {} > max seal epoch {}", + read_current_epoch, + sealed_epoch + ); + return Err(HummockError::read_current_epoch().into()); + } + + let min_current_epoch = self.min_current_epoch.load(MemOrdering::SeqCst); + if read_current_epoch < min_current_epoch { + tracing::warn!( + "invalid barrier read {} < min current epoch {}", + read_current_epoch, + min_current_epoch + ); + return Err(HummockError::read_current_epoch().into()); + } + } + Ok(()) + } +} + +#[cfg(any(test, feature = "test"))] +impl HummockStorage { + pub async fn seal_and_sync_epoch(&self, epoch: u64) -> StorageResult { + self.seal_epoch(epoch, true); + self.sync(epoch).await + } + + /// Used in the compaction test tool + pub async fn update_version_and_wait(&self, version: HummockVersion) { + use tokio::task::yield_now; + let version_id = version.id; + self.hummock_event_sender + .send(HummockEvent::VersionUpdate( + version_update_payload::Payload::PinnedVersion(version), + )) + .unwrap(); + loop { + if self.pinned_version.load().id() >= version_id { + break; + } + + yield_now().await + } + } + + pub async fn wait_version(&self, version: HummockVersion) { + use tokio::task::yield_now; + loop { + if self.pinned_version.load().id() >= version.id { + break; + } + + yield_now().await + } + } + + pub fn get_shared_buffer_size(&self) -> usize { + self.buffer_tracker.get_buffer_size() + } + + pub async fn try_wait_epoch_for_test(&self, wait_epoch: u64) { + let mut rx = self.version_update_notifier_tx.subscribe(); + while *(rx.borrow_and_update()) < wait_epoch { + rx.changed().await.unwrap(); + } + } + + /// Creates a [`HummockStorage`] with default stats. Should only be used by tests. + pub async fn for_test( + options: Arc, + sstable_store: SstableStoreRef, + hummock_meta_client: Arc, + notification_client: impl NotificationClient, + ) -> HummockResult { + Self::new( + options, + sstable_store, + hummock_meta_client, + notification_client, + Arc::new(FilterKeyExtractorManager::default()), + Arc::new(HummockStateStoreMetrics::unused()), + Arc::new(CompactorMetrics::unused()), + ) + .await + } + + pub fn storage_opts(&self) -> &Arc { + &self.context.storage_opts + } + + pub fn version_reader(&self) -> &HummockVersionReader { + &self.hummock_version_reader + } + + pub async fn wait_version_update(&self, old_id: u64) -> u64 { + use tokio::task::yield_now; + loop { + let cur_id = self.pinned_version.load().id(); + if cur_id > old_id { + return cur_id; + } + yield_now().await; + } + } +} diff --git a/src/storage/src/hummock/store/state_store.rs b/src/storage/src/hummock/store/local_hummock_storage.rs similarity index 92% rename from src/storage/src/hummock/store/state_store.rs rename to src/storage/src/hummock/store/local_hummock_storage.rs index f5a6f65b8d6fc..18ae72300cd67 100644 --- a/src/storage/src/hummock/store/state_store.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -20,7 +20,7 @@ use await_tree::InstrumentAwait; use bytes::Bytes; use parking_lot::RwLock; use risingwave_common::catalog::{TableId, TableOption}; -use risingwave_hummock_sdk::key::{map_table_key_range, TableKey, TableKeyRange}; +use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; use risingwave_hummock_sdk::HummockEpoch; use tokio::sync::mpsc; use tracing::{warn, Instrument}; @@ -48,6 +48,8 @@ use crate::storage_value::StorageValue; use crate::store::*; use crate::StateStoreIter; +/// `LocalHummockStorage` is a handle for a state table shard to access data from and write data to +/// the hummock state backend. It is created via `HummockStorage::new_local`. pub struct LocalHummockStorage { mem_table: MemTable, @@ -141,24 +143,22 @@ impl LocalHummockStorage { pub async fn may_exist_inner( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> StorageResult { if self.mem_table.iter(key_range.clone()).next().is_some() { return Ok(true); } - let table_key_range = map_table_key_range(key_range); - let read_snapshot = read_filter_for_local( HummockEpoch::MAX, // Use MAX epoch to make sure we read from latest read_options.table_id, - &table_key_range, + &key_range, self.read_version.clone(), )?; self.hummock_version_reader - .may_exist(table_key_range, read_options, read_snapshot) + .may_exist(key_range, read_options, read_snapshot) .await } } @@ -168,22 +168,22 @@ impl StateStoreRead for LocalHummockStorage { fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> impl Future>> + '_ { assert!(epoch <= self.epoch()); - self.get_inner(TableKey(key), epoch, read_options) + self.get_inner(key, epoch, read_options) } fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> impl Future> + '_ { assert!(epoch <= self.epoch()); - self.iter_inner(map_table_key_range(key_range), epoch, read_options) + self.iter_inner(key_range, epoch, read_options) .instrument(tracing::trace_span!("hummock_iter")) } } @@ -193,18 +193,19 @@ impl LocalStateStore for LocalHummockStorage { fn may_exist( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future> + Send + '_ { self.may_exist_inner(key_range, read_options) } - async fn get(&self, key: Bytes, read_options: ReadOptions) -> StorageResult> { + async fn get( + &self, + key: TableKey, + read_options: ReadOptions, + ) -> StorageResult> { match self.mem_table.buffer.get(&key) { - None => { - self.get_inner(TableKey(key), self.epoch(), read_options) - .await - } + None => self.get_inner(key, self.epoch(), read_options).await, Some(op) => match op { KeyOp::Insert(value) | KeyOp::Update((_, value)) => Ok(Some(value.clone())), KeyOp::Delete(_) => Ok(None), @@ -215,19 +216,13 @@ impl LocalStateStore for LocalHummockStorage { #[allow(clippy::manual_async_fn)] fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { async move { let stream = self - .iter_inner( - map_table_key_range(key_range.clone()), - self.epoch(), - read_options, - ) + .iter_inner(key_range.clone(), self.epoch(), read_options) .await?; - let (l, r) = key_range; - let key_range = (l.map(Bytes::from), r.map(Bytes::from)); Ok(merge_stream( self.mem_table.iter(key_range), stream, @@ -237,7 +232,12 @@ impl LocalStateStore for LocalHummockStorage { } } - fn insert(&mut self, key: Bytes, new_val: Bytes, old_val: Option) -> StorageResult<()> { + fn insert( + &mut self, + key: TableKey, + new_val: Bytes, + old_val: Option, + ) -> StorageResult<()> { match old_val { None => self.mem_table.insert(key, new_val)?, Some(old_val) => self.mem_table.update(key, old_val, new_val)?, @@ -245,7 +245,7 @@ impl LocalStateStore for LocalHummockStorage { Ok(()) } - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()> { + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()> { self.mem_table.delete(key, old_val)?; Ok(()) } @@ -360,7 +360,7 @@ impl LocalStateStore for LocalHummockStorage { impl LocalHummockStorage { async fn flush_inner( &mut self, - kv_pairs: Vec<(Bytes, StorageValue)>, + kv_pairs: Vec<(TableKey, StorageValue)>, delete_ranges: Vec<(Bound, Bound)>, write_options: WriteOptions, ) -> StorageResult { diff --git a/src/storage/src/hummock/store/mod.rs b/src/storage/src/hummock/store/mod.rs index 1f74b4f004e5c..0831ff35d2dea 100644 --- a/src/storage/src/hummock/store/mod.rs +++ b/src/storage/src/hummock/store/mod.rs @@ -12,6 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod memtable; -pub mod state_store; +pub mod hummock_storage; +pub mod local_hummock_storage; pub mod version; + +pub use hummock_storage::*; +pub use local_hummock_storage::*; diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 38f7cb4e54e3f..41273e51657c8 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -31,8 +31,7 @@ use risingwave_pb::hummock::{HummockVersionDelta, LevelType, SstableInfo}; use sync_point::sync_point; use tracing::Instrument; -use super::memtable::{ImmId, ImmutableMemtable}; -use super::state_store::StagingDataIterator; +use super::StagingDataIterator; use crate::error::StorageResult; use crate::hummock::iterator::{ ConcatIterator, ForwardMergeRangeIterator, HummockIteratorUnion, OrderedMergeIteratorInner, @@ -41,7 +40,7 @@ use crate::hummock::iterator::{ use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::sstable::SstableIteratorReadOptions; use crate::hummock::sstable_store::SstableStoreRef; -use crate::hummock::store::state_store::HummockStorageIterator; +use crate::hummock::store::HummockStorageIterator; use crate::hummock::utils::{ check_subset_preserve_order, filter_single_sst, prune_nonoverlapping_ssts, prune_overlapping_ssts, range_overlap, search_sst_idx, @@ -50,6 +49,7 @@ use crate::hummock::{ get_from_batch, get_from_sstable_info, hit_sstable_bloom_filter, Sstable, SstableDeleteRangeIterator, SstableIterator, }; +use crate::mem_table::{ImmId, ImmutableMemtable}; use crate::monitor::{ GetLocalMetricsGuard, HummockStateStoreMetrics, MayExistLocalMetricsGuard, StoreLocalStatistic, }; @@ -112,7 +112,6 @@ impl StagingSstableInfo { #[derive(Clone)] pub enum StagingData { - // ImmMem(Arc), ImmMem(ImmutableMemtable), MergedImmMem(ImmutableMemtable), Sst(StagingSstableInfo), diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index cded494d128a3..3381b629ddfe6 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -21,7 +21,7 @@ use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_common::must_match; -use risingwave_hummock_sdk::key::{FullKey, PointRange, UserKey}; +use risingwave_hummock_sdk::key::{FullKey, PointRange, TableKey, UserKey}; use risingwave_hummock_sdk::{HummockEpoch, HummockSstableObjectId}; use risingwave_pb::hummock::{KeyRange, SstableInfo}; @@ -63,19 +63,19 @@ pub fn default_opts_for_test() -> StorageOpts { } } -pub fn gen_dummy_batch(n: u64) -> Vec<(Bytes, StorageValue)> { +pub fn gen_dummy_batch(n: u64) -> Vec<(TableKey, StorageValue)> { vec![( - Bytes::from(iterator_test_table_key_of(n as usize)), + TableKey(Bytes::from(iterator_test_table_key_of(n as usize))), StorageValue::new_put(b"value1".to_vec()), )] } -pub fn gen_dummy_batch_several_keys(n: usize) -> Vec<(Bytes, StorageValue)> { +pub fn gen_dummy_batch_several_keys(n: usize) -> Vec<(TableKey, StorageValue)> { let mut kvs = vec![]; let v = Bytes::from(b"value1".to_vec().repeat(100)); for idx in 0..n { kvs.push(( - Bytes::from(iterator_test_table_key_of(idx)), + TableKey(Bytes::from(iterator_test_table_key_of(idx))), StorageValue::new_put(v.clone()), )); } diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index d24ff4ab09ec9..7ccb3fbf04790 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -372,7 +372,7 @@ pub(crate) const ENABLE_SANITY_CHECK: bool = cfg!(debug_assertions); /// Make sure the key to insert should not exist in storage. pub(crate) async fn do_insert_sanity_check( - key: Bytes, + key: TableKey, value: Bytes, inner: &impl StateStoreRead, epoch: u64, @@ -400,7 +400,7 @@ pub(crate) async fn do_insert_sanity_check( /// Make sure that the key to delete should exist in storage and the value should be matched. pub(crate) async fn do_delete_sanity_check( - key: Bytes, + key: TableKey, old_value: Bytes, inner: &impl StateStoreRead, epoch: u64, @@ -437,7 +437,7 @@ pub(crate) async fn do_delete_sanity_check( /// Make sure that the key to update should exist in storage and the value should be matched pub(crate) async fn do_update_sanity_check( - key: Bytes, + key: TableKey, old_value: Bytes, new_value: Bytes, inner: &impl StateStoreRead, @@ -497,9 +497,9 @@ fn validate_delete_range(left: &Bound, right: &Bound) -> bool { } pub(crate) fn filter_with_delete_range<'a>( - kv_iter: impl Iterator + 'a, + kv_iter: impl Iterator, KeyOp)> + 'a, mut delete_ranges_iter: impl Iterator, Bound)> + 'a, -) -> impl Iterator + 'a { +) -> impl Iterator, KeyOp)> + 'a { let mut range = delete_ranges_iter.next(); if let Some((range_start, range_end)) = range { assert!( @@ -511,10 +511,11 @@ pub(crate) fn filter_with_delete_range<'a>( } kv_iter.filter(move |(ref key, _)| { if let Some(range_bound) = range { - if cmp_delete_range_left_bounds(Included(key), range_bound.0.as_ref()) == Ordering::Less + if cmp_delete_range_left_bounds(Included(&key.0), range_bound.0.as_ref()) + == Ordering::Less { true - } else if range_bound.contains(key) { + } else if range_bound.contains(key.as_ref()) { false } else { // Key has exceeded the current key range. Advance to the next range. @@ -532,7 +533,7 @@ pub(crate) fn filter_with_delete_range<'a>( { // Not fall in the next delete range break true; - } else if range_bound.contains(key) { + } else if range_bound.contains(key.as_ref()) { // Fall in the next delete range break false; } else { diff --git a/src/storage/src/lib.rs b/src/storage/src/lib.rs index 5ebcb4fe78b00..72b925170a6ef 100644 --- a/src/storage/src/lib.rs +++ b/src/storage/src/lib.rs @@ -58,7 +58,6 @@ pub mod error; pub mod opts; pub mod store_impl; pub mod table; -pub mod write_batch; pub mod filter_key_extractor; pub mod mem_table; diff --git a/src/storage/src/mem_table.rs b/src/storage/src/mem_table.rs index 1ff090690aa4a..daa810afd96e8 100644 --- a/src/storage/src/mem_table.rs +++ b/src/storage/src/mem_table.rs @@ -23,10 +23,11 @@ use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::estimate_size::{EstimateSize, KvSize}; -use risingwave_hummock_sdk::key::{FullKey, TableKey}; +use risingwave_hummock_sdk::key::{FullKey, TableKey, TableKeyRange}; use thiserror::Error; use crate::error::{StorageError, StorageResult}; +use crate::hummock::shared_buffer::shared_buffer_batch::{SharedBufferBatch, SharedBufferBatchId}; use crate::hummock::utils::{ cmp_delete_range_left_bounds, do_delete_sanity_check, do_insert_sanity_check, do_update_sanity_check, filter_with_delete_range, ENABLE_SANITY_CHECK, @@ -35,6 +36,10 @@ use crate::row_serde::value_serde::ValueRowSerde; use crate::storage_value::StorageValue; use crate::store::*; +pub type ImmutableMemtable = SharedBufferBatch; + +pub type ImmId = SharedBufferBatchId; + #[derive(Clone, Debug, EstimateSize)] pub enum KeyOp { Insert(Bytes), @@ -46,7 +51,7 @@ pub enum KeyOp { /// `MemTable` is a buffer for modify operations without encoding #[derive(Clone)] pub struct MemTable { - pub(crate) buffer: BTreeMap, + pub(crate) buffer: BTreeMap, KeyOp>, pub(crate) is_consistent_op: bool, pub(crate) kv_size: KvSize, } @@ -54,7 +59,11 @@ pub struct MemTable { #[derive(Error, Debug)] pub enum MemTableError { #[error("Inconsistent operation")] - InconsistentOperation { key: Bytes, prev: KeyOp, new: KeyOp }, + InconsistentOperation { + key: TableKey, + prev: KeyOp, + new: KeyOp, + }, } type Result = std::result::Result>; @@ -77,13 +86,8 @@ impl MemTable { !self.buffer.is_empty() } - /// read methods - pub fn get_key_op(&self, pk: &[u8]) -> Option<&KeyOp> { - self.buffer.get(pk) - } - /// write methods - pub fn insert(&mut self, pk: Bytes, value: Bytes) -> Result<()> { + pub fn insert(&mut self, pk: TableKey, value: Bytes) -> Result<()> { if !self.is_consistent_op { let key_len = std::mem::size_of::() + pk.len(); let insert_value = KeyOp::Insert(value); @@ -125,7 +129,7 @@ impl MemTable { } } - pub fn delete(&mut self, pk: Bytes, old_value: Bytes) -> Result<()> { + pub fn delete(&mut self, pk: TableKey, old_value: Bytes) -> Result<()> { let key_len = std::mem::size_of::() + pk.len(); if !self.is_consistent_op { let delete_value = KeyOp::Delete(old_value); @@ -185,7 +189,12 @@ impl MemTable { } } - pub fn update(&mut self, pk: Bytes, old_value: Bytes, new_value: Bytes) -> Result<()> { + pub fn update( + &mut self, + pk: TableKey, + old_value: Bytes, + new_value: Bytes, + ) -> Result<()> { if !self.is_consistent_op { let key_len = std::mem::size_of::() + pk.len(); @@ -245,13 +254,16 @@ impl MemTable { } } - pub fn into_parts(self) -> BTreeMap { + pub fn into_parts(self) -> BTreeMap, KeyOp> { self.buffer } - pub fn iter<'a, R>(&'a self, key_range: R) -> impl Iterator + pub fn iter<'a, R>( + &'a self, + key_range: R, + ) -> impl Iterator, &'a KeyOp)> where - R: RangeBounds + 'a, + R: RangeBounds> + 'a, { self.buffer.range(key_range) } @@ -291,7 +303,7 @@ impl KeyOp { #[try_stream(ok = StateStoreIterItem, error = StorageError)] pub(crate) async fn merge_stream<'a>( - mem_table_iter: impl Iterator + 'a, + mem_table_iter: impl Iterator, &'a KeyOp)> + 'a, inner_stream: impl StateStoreReadIterStream, table_id: TableId, epoch: u64, @@ -314,17 +326,14 @@ pub(crate) async fn merge_stream<'a>( let (key, key_op) = mem_table_iter.next().unwrap(); match key_op { KeyOp::Insert(value) | KeyOp::Update((_, value)) => { - yield ( - FullKey::new(table_id, TableKey(key.clone()), epoch), - value.clone(), - ) + yield (FullKey::new(table_id, key.clone(), epoch), value.clone()) } _ => {} } } (Some(Ok((inner_key, _))), Some((mem_table_key, _))) => { debug_assert_eq!(inner_key.user_key.table_id, table_id); - match inner_key.user_key.table_key.0.cmp(mem_table_key) { + match inner_key.user_key.table_key.cmp(mem_table_key) { Ordering::Less => { // yield data from storage let (key, value) = inner_stream.next().await.unwrap()?; @@ -354,10 +363,7 @@ pub(crate) async fn merge_stream<'a>( match key_op { KeyOp::Insert(value) => { - yield ( - FullKey::new(table_id, TableKey(key.clone()), epoch), - value.clone(), - ); + yield (FullKey::new(table_id, key.clone(), epoch), value.clone()); } KeyOp::Delete(_) => {} KeyOp::Update(_) => unreachable!( @@ -409,13 +415,17 @@ impl LocalStateStore for MemtableLocalState #[allow(clippy::unused_async)] async fn may_exist( &self, - _key_range: IterKeyRange, + _key_range: TableKeyRange, _read_options: ReadOptions, ) -> StorageResult { Ok(true) } - async fn get(&self, key: Bytes, read_options: ReadOptions) -> StorageResult> { + async fn get( + &self, + key: TableKey, + read_options: ReadOptions, + ) -> StorageResult> { match self.mem_table.buffer.get(&key) { None => self.inner.get(key, self.epoch(), read_options).await, Some(op) => match op { @@ -428,7 +438,7 @@ impl LocalStateStore for MemtableLocalState #[allow(clippy::manual_async_fn)] fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { async move { @@ -436,8 +446,6 @@ impl LocalStateStore for MemtableLocalState .inner .iter(key_range.clone(), self.epoch(), read_options) .await?; - let (l, r) = key_range; - let key_range = (l.map(Bytes::from), r.map(Bytes::from)); Ok(merge_stream( self.mem_table.iter(key_range), stream, @@ -447,7 +455,12 @@ impl LocalStateStore for MemtableLocalState } } - fn insert(&mut self, key: Bytes, new_val: Bytes, old_val: Option) -> StorageResult<()> { + fn insert( + &mut self, + key: TableKey, + new_val: Bytes, + old_val: Option, + ) -> StorageResult<()> { match old_val { None => self.mem_table.insert(key, new_val)?, Some(old_val) => self.mem_table.update(key, old_val, new_val)?, @@ -455,7 +468,7 @@ impl LocalStateStore for MemtableLocalState Ok(()) } - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()> { + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()> { Ok(self.mem_table.delete(key, old_val)?) } @@ -567,6 +580,7 @@ impl LocalStateStore for MemtableLocalState #[cfg(test)] mod tests { use bytes::Bytes; + use risingwave_hummock_sdk::key::TableKey; use crate::mem_table::{KeyOp, MemTable}; @@ -575,7 +589,9 @@ mod tests { let mut mem_table = MemTable::new(true); assert_eq!(mem_table.kv_size.size(), 0); - mem_table.insert("key1".into(), "value1".into()).unwrap(); + mem_table + .insert(TableKey("key1".into()), "value1".into()) + .unwrap(); assert_eq!( mem_table.kv_size.size(), std::mem::size_of::() @@ -587,7 +603,9 @@ mod tests { // delete mem_table.drain(); assert_eq!(mem_table.kv_size.size(), 0); - mem_table.delete("key2".into(), "value2".into()).unwrap(); + mem_table + .delete(TableKey("key2".into()), "value2".into()) + .unwrap(); assert_eq!( mem_table.kv_size.size(), std::mem::size_of::() @@ -595,7 +613,9 @@ mod tests { + std::mem::size_of::() + Bytes::from("value2").len() ); - mem_table.insert("key2".into(), "value22".into()).unwrap(); + mem_table + .insert(TableKey("key2".into()), "value22".into()) + .unwrap(); assert_eq!( mem_table.kv_size.size(), std::mem::size_of::() @@ -605,7 +625,9 @@ mod tests { + Bytes::from("value2").len() ); - mem_table.delete("key2".into(), "value22".into()).unwrap(); + mem_table + .delete(TableKey("key2".into()), "value22".into()) + .unwrap(); assert_eq!( mem_table.kv_size.size(), @@ -618,7 +640,9 @@ mod tests { // update mem_table.drain(); assert_eq!(mem_table.kv_size.size(), 0); - mem_table.insert("key3".into(), "value3".into()).unwrap(); + mem_table + .insert(TableKey("key3".into()), "value3".into()) + .unwrap(); assert_eq!( mem_table.kv_size.size(), std::mem::size_of::() @@ -629,7 +653,7 @@ mod tests { // update-> insert mem_table - .update("key3".into(), "value3".into(), "value333".into()) + .update(TableKey("key3".into()), "value3".into(), "value333".into()) .unwrap(); assert_eq!( mem_table.kv_size.size(), @@ -641,7 +665,7 @@ mod tests { mem_table.drain(); mem_table - .update("key4".into(), "value4".into(), "value44".into()) + .update(TableKey("key4".into()), "value4".into(), "value44".into()) .unwrap(); assert_eq!( @@ -653,7 +677,11 @@ mod tests { + Bytes::from("value44").len() ); mem_table - .update("key4".into(), "value44".into(), "value4444".into()) + .update( + TableKey("key4".into()), + "value44".into(), + "value4444".into(), + ) .unwrap(); assert_eq!( @@ -671,7 +699,9 @@ mod tests { let mut mem_table = MemTable::new(false); assert_eq!(mem_table.kv_size.size(), 0); - mem_table.insert("key1".into(), "value1".into()).unwrap(); + mem_table + .insert(TableKey("key1".into()), "value1".into()) + .unwrap(); assert_eq!( mem_table.kv_size.size(), std::mem::size_of::() @@ -680,7 +710,9 @@ mod tests { + Bytes::from("value1").len() ); - mem_table.insert("key1".into(), "value111".into()).unwrap(); + mem_table + .insert(TableKey("key1".into()), "value111".into()) + .unwrap(); assert_eq!( mem_table.kv_size.size(), std::mem::size_of::() @@ -691,7 +723,7 @@ mod tests { mem_table.drain(); mem_table - .update("key4".into(), "value4".into(), "value44".into()) + .update(TableKey("key4".into()), "value4".into(), "value44".into()) .unwrap(); assert_eq!( @@ -703,7 +735,11 @@ mod tests { + Bytes::from("value44").len() ); mem_table - .update("key4".into(), "value44".into(), "value4444".into()) + .update( + TableKey("key4".into()), + "value44".into(), + "value4444".into(), + ) .unwrap(); assert_eq!( diff --git a/src/storage/src/memory.rs b/src/storage/src/memory.rs index 26b37c83cbf4a..ac52b65e5488f 100644 --- a/src/storage/src/memory.rs +++ b/src/storage/src/memory.rs @@ -20,7 +20,7 @@ use std::sync::{Arc, LazyLock}; use bytes::Bytes; use parking_lot::RwLock; use risingwave_common::catalog::TableId; -use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKey}; +use risingwave_hummock_sdk::key::{FullKey, TableKey, TableKeyRange, UserKey}; use risingwave_hummock_sdk::{HummockEpoch, HummockReadEpoch}; use crate::error::StorageResult; @@ -501,7 +501,7 @@ impl MemoryStateStore { impl RangeKvStateStore { fn scan( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, table_id: TableId, limit: Option, @@ -538,7 +538,7 @@ impl StateStoreRead for RangeKvStateStore { #[allow(clippy::unused_async)] async fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> StorageResult> { @@ -556,7 +556,7 @@ impl StateStoreRead for RangeKvStateStore { #[allow(clippy::unused_async)] async fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> StorageResult { @@ -575,7 +575,7 @@ impl StateStoreWrite for RangeKvStateStore { #[allow(clippy::unused_async)] async fn ingest_batch( &self, - mut kv_pairs: Vec<(Bytes, StorageValue)>, + mut kv_pairs: Vec<(TableKey, StorageValue)>, delete_ranges: Vec<(Bound, Bound)>, write_options: WriteOptions, ) -> StorageResult { @@ -594,7 +594,7 @@ impl StateStoreWrite for RangeKvStateStore { ), None, )? { - delete_keys.insert(key.user_key.table_key.0); + delete_keys.insert(key.user_key.table_key); } } for key in delete_keys { @@ -606,7 +606,7 @@ impl StateStoreWrite for RangeKvStateStore { .ingest_batch(kv_pairs.into_iter().map(|(key, value)| { size += key.len() + value.size(); ( - FullKey::new(write_options.table_id, TableKey(key), epoch), + FullKey::new(write_options.table_id, key, epoch), value.user_value, ) }))?; @@ -729,8 +729,14 @@ mod tests { state_store .ingest_batch( vec![ - (b"a".to_vec().into(), StorageValue::new_put(b"v1".to_vec())), - (b"b".to_vec().into(), StorageValue::new_put(b"v1".to_vec())), + ( + TableKey(Bytes::from(b"a".to_vec())), + StorageValue::new_put(b"v1".to_vec()), + ), + ( + TableKey(Bytes::from(b"b".to_vec())), + StorageValue::new_put(b"v1".to_vec()), + ), ], vec![], WriteOptions { @@ -743,8 +749,14 @@ mod tests { state_store .ingest_batch( vec![ - (b"a".to_vec().into(), StorageValue::new_put(b"v2".to_vec())), - (b"b".to_vec().into(), StorageValue::new_delete()), + ( + TableKey(Bytes::from(b"a".to_vec())), + StorageValue::new_put(b"v2".to_vec()), + ), + ( + TableKey(Bytes::from(b"b".to_vec())), + StorageValue::new_delete(), + ), ], vec![], WriteOptions { @@ -758,8 +770,8 @@ mod tests { state_store .scan( ( - Bound::Included(Bytes::from("a")), - Bound::Included(Bytes::from("b")), + Bound::Included(TableKey(Bytes::from("a"))), + Bound::Included(TableKey(Bytes::from("b"))), ), 0, TableId::default(), @@ -785,8 +797,8 @@ mod tests { state_store .scan( ( - Bound::Included(Bytes::from("a")), - Bound::Included(Bytes::from("b")), + Bound::Included(TableKey(Bytes::from("a"))), + Bound::Included(TableKey(Bytes::from("b"))), ), 0, TableId::default(), @@ -804,8 +816,8 @@ mod tests { state_store .scan( ( - Bound::Included(Bytes::from("a")), - Bound::Included(Bytes::from("b")), + Bound::Included(TableKey(Bytes::from("a"))), + Bound::Included(TableKey(Bytes::from("b"))), ), 1, TableId::default(), @@ -821,42 +833,54 @@ mod tests { ); assert_eq!( state_store - .get(Bytes::from("a"), 0, ReadOptions::default(),) + .get(TableKey(Bytes::from("a")), 0, ReadOptions::default(),) .await .unwrap(), Some(Bytes::from("v1")) ); assert_eq!( state_store - .get(Bytes::copy_from_slice(b"b"), 0, ReadOptions::default(),) + .get( + TableKey(Bytes::copy_from_slice(b"b")), + 0, + ReadOptions::default(), + ) .await .unwrap(), Some(b"v1".to_vec().into()) ); assert_eq!( state_store - .get(Bytes::copy_from_slice(b"c"), 0, ReadOptions::default(),) + .get( + TableKey(Bytes::copy_from_slice(b"c")), + 0, + ReadOptions::default(), + ) .await .unwrap(), None ); assert_eq!( state_store - .get(Bytes::copy_from_slice(b"a"), 1, ReadOptions::default(),) + .get( + TableKey(Bytes::copy_from_slice(b"a")), + 1, + ReadOptions::default(), + ) .await .unwrap(), Some(b"v2".to_vec().into()) ); assert_eq!( state_store - .get(Bytes::from("b"), 1, ReadOptions::default(),) + .get(TableKey(Bytes::from("b")), 1, ReadOptions::default(),) .await .unwrap(), None ); assert_eq!( state_store - .get(Bytes::from("c"), 1, ReadOptions::default()) + .get(TableKey(Bytes::from("c")), 1, ReadOptions::default()) .await .unwrap(), None diff --git a/src/storage/src/monitor/monitored_store.rs b/src/storage/src/monitor/monitored_store.rs index 77924a999709d..fd3e235201eb4 100644 --- a/src/storage/src/monitor/monitored_store.rs +++ b/src/storage/src/monitor/monitored_store.rs @@ -20,6 +20,7 @@ use bytes::Bytes; use futures::{Future, TryFutureExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; use risingwave_hummock_sdk::HummockReadEpoch; use tokio::time::Instant; use tracing::error; @@ -175,7 +176,7 @@ impl StateStoreRead for MonitoredStateStore { fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> impl Future>> + '_ { @@ -186,7 +187,7 @@ impl StateStoreRead for MonitoredStateStore { fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> impl Future> + '_ { @@ -203,7 +204,7 @@ impl LocalStateStore for MonitoredStateStore { async fn may_exist( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> StorageResult { let table_id_label = read_options.table_id.to_string(); @@ -223,7 +224,7 @@ impl LocalStateStore for MonitoredStateStore { fn get( &self, - key: Bytes, + key: TableKey, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { let table_id = read_options.table_id; @@ -234,7 +235,7 @@ impl LocalStateStore for MonitoredStateStore { fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { let table_id = read_options.table_id; @@ -243,12 +244,17 @@ impl LocalStateStore for MonitoredStateStore { .map_ok(identity) } - fn insert(&mut self, key: Bytes, new_val: Bytes, old_val: Option) -> StorageResult<()> { + fn insert( + &mut self, + key: TableKey, + new_val: Bytes, + old_val: Option, + ) -> StorageResult<()> { // TODO: collect metrics self.inner.insert(key, new_val, old_val) } - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()> { + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()> { // TODO: collect metrics self.inner.delete(key, old_val) } diff --git a/src/storage/src/monitor/traced_store.rs b/src/storage/src/monitor/traced_store.rs index 505c0460552a7..e92b5974844e2 100644 --- a/src/storage/src/monitor/traced_store.rs +++ b/src/storage/src/monitor/traced_store.rs @@ -17,6 +17,7 @@ use std::ops::Bound; use bytes::Bytes; use futures::{Future, TryFutureExt, TryStreamExt}; use futures_async_stream::try_stream; +use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_hummock_trace::{ init_collector, should_use_trace, ConcurrentId, MayTraceSpan, OperationResult, StorageType, @@ -82,13 +83,13 @@ impl TracedStateStore { async fn traced_get( &self, - key: Bytes, + key: TableKey, epoch: Option, read_options: ReadOptions, get_future: impl Future>>, ) -> StorageResult> { let span = TraceSpan::new_get_span( - key.clone(), + key.0.clone(), epoch, read_options.clone().into(), self.storage_type, @@ -111,7 +112,7 @@ impl LocalStateStore for TracedStateStore { fn may_exist( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future> + Send + '_ { self.inner.may_exist(key_range, read_options) @@ -119,7 +120,7 @@ impl LocalStateStore for TracedStateStore { fn get( &self, - key: Bytes, + key: TableKey, read_options: ReadOptions, ) -> impl Future>> + '_ { self.traced_get( @@ -132,11 +133,13 @@ impl LocalStateStore for TracedStateStore { fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { + let (l, r) = key_range.clone(); + let bytes_key_range = (l.map(|l| l.0), r.map(|r| r.0)); let span = TraceSpan::new_iter_span( - key_range.clone(), + bytes_key_range, None, read_options.clone().into(), self.storage_type, @@ -145,9 +148,14 @@ impl LocalStateStore for TracedStateStore { .map_ok(identity) } - fn insert(&mut self, key: Bytes, new_val: Bytes, old_val: Option) -> StorageResult<()> { + fn insert( + &mut self, + key: TableKey, + new_val: Bytes, + old_val: Option, + ) -> StorageResult<()> { let span = TraceSpan::new_insert_span( - key.clone(), + key.0.clone(), new_val.clone(), old_val.clone(), self.storage_type, @@ -158,8 +166,8 @@ impl LocalStateStore for TracedStateStore { res } - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()> { - let span = TraceSpan::new_delete_span(key.clone(), old_val.clone(), self.storage_type); + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()> { + let span = TraceSpan::new_delete_span(key.0.clone(), old_val.clone(), self.storage_type); let res = self.inner.delete(key, old_val); @@ -263,7 +271,7 @@ impl StateStoreRead for TracedStateStore { fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { @@ -277,12 +285,14 @@ impl StateStoreRead for TracedStateStore { fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> impl Future> + '_ { + let (l, r) = key_range.clone(); + let bytes_key_range = (l.map(|l| l.0), r.map(|r| r.0)); let span = TraceSpan::new_iter_span( - key_range.clone(), + bytes_key_range, Some(epoch), read_options.clone().into(), self.storage_type, diff --git a/src/storage/src/panic_store.rs b/src/storage/src/panic_store.rs index 53162e29637fd..de4e9f38032d7 100644 --- a/src/storage/src/panic_store.rs +++ b/src/storage/src/panic_store.rs @@ -18,6 +18,7 @@ use std::task::{Context, Poll}; use bytes::Bytes; use futures::Stream; +use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; use risingwave_hummock_sdk::HummockReadEpoch; use crate::error::StorageResult; @@ -35,7 +36,7 @@ impl StateStoreRead for PanicStateStore { #[allow(clippy::unused_async)] async fn get( &self, - _key: Bytes, + _key: TableKey, _epoch: u64, _read_options: ReadOptions, ) -> StorageResult> { @@ -45,7 +46,7 @@ impl StateStoreRead for PanicStateStore { #[allow(clippy::unused_async)] async fn iter( &self, - _key_range: IterKeyRange, + _key_range: TableKeyRange, _epoch: u64, _read_options: ReadOptions, ) -> StorageResult { @@ -57,11 +58,11 @@ impl StateStoreWrite for PanicStateStore { #[allow(clippy::unused_async)] async fn ingest_batch( &self, - _kv_pairs: Vec<(Bytes, StorageValue)>, + _kv_pairs: Vec<(TableKey, StorageValue)>, _delete_ranges: Vec<(Bound, Bound)>, _write_options: WriteOptions, ) -> StorageResult { - panic!("should not read from the state store!"); + panic!("should not write to the state store!"); } } @@ -71,21 +72,25 @@ impl LocalStateStore for PanicStateStore { #[allow(clippy::unused_async)] async fn may_exist( &self, - _key_range: IterKeyRange, + _key_range: TableKeyRange, _read_options: ReadOptions, ) -> StorageResult { panic!("should not call may_exist from the state store!"); } #[allow(clippy::unused_async)] - async fn get(&self, _key: Bytes, _read_options: ReadOptions) -> StorageResult> { + async fn get( + &self, + _key: TableKey, + _read_options: ReadOptions, + ) -> StorageResult> { panic!("should not operate on the panic state store!"); } #[allow(clippy::unused_async)] async fn iter( &self, - _key_range: IterKeyRange, + _key_range: TableKeyRange, _read_options: ReadOptions, ) -> StorageResult> { panic!("should not operate on the panic state store!"); @@ -93,14 +98,14 @@ impl LocalStateStore for PanicStateStore { fn insert( &mut self, - _key: Bytes, + _key: TableKey, _new_val: Bytes, _old_val: Option, ) -> StorageResult<()> { panic!("should not operate on the panic state store!"); } - fn delete(&mut self, _key: Bytes, _old_val: Bytes) -> StorageResult<()> { + fn delete(&mut self, _key: TableKey, _old_val: Bytes) -> StorageResult<()> { panic!("should not operate on the panic state store!"); } diff --git a/src/storage/src/row_serde/row_serde_util.rs b/src/storage/src/row_serde/row_serde_util.rs index d426789829797..b8ef0dba29fec 100644 --- a/src/storage/src/row_serde/row_serde_util.rs +++ b/src/storage/src/row_serde/row_serde_util.rs @@ -16,6 +16,7 @@ use bytes::{BufMut, Bytes, BytesMut}; use risingwave_common::hash::VirtualNode; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::util::row_serde::OrderedRowSerde; +use risingwave_hummock_sdk::key::TableKey; pub fn serialize_pk(pk: impl Row, serializer: &OrderedRowSerde) -> Bytes { let mut buf = BytesMut::with_capacity(pk.len()); @@ -27,11 +28,11 @@ pub fn serialize_pk_with_vnode( pk: impl Row, serializer: &OrderedRowSerde, vnode: VirtualNode, -) -> Bytes { +) -> TableKey { let mut buffer = BytesMut::new(); buffer.put_slice(&vnode.to_be_bytes()[..]); pk.memcmp_serialize_into(serializer, &mut buffer); - buffer.freeze() + TableKey(buffer.freeze()) } pub fn deserialize_pk_with_vnode( diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index 4150e31d616e8..2cef99ab4e651 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -22,7 +22,7 @@ use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::util::epoch::{Epoch, EpochPair}; -use risingwave_hummock_sdk::key::{FullKey, KeyPayloadType}; +use risingwave_hummock_sdk::key::{FullKey, TableKey, TableKeyRange}; use risingwave_hummock_sdk::{HummockReadEpoch, LocalSstableInfo}; use risingwave_hummock_trace::{ TracedInitOptions, TracedNewLocalOptions, TracedPrefetchOptions, TracedReadOptions, @@ -33,7 +33,6 @@ use crate::error::{StorageError, StorageResult}; use crate::hummock::CachePolicy; use crate::monitor::{MonitoredStateStore, MonitoredStorageMetrics}; use crate::storage_value::StorageValue; -use crate::write_batch::WriteBatch; pub trait StaticSendSync = Send + Sync + 'static; @@ -70,8 +69,6 @@ pub type StateStoreIterItem = (FullKey, Bytes); pub trait StateStoreIterItemStream = Stream> + Send; pub trait StateStoreReadIterStream = StateStoreIterItemStream + 'static; -pub type IterKeyRange = (Bound, Bound); - pub trait StateStoreRead: StaticSendSync { type IterStream: StateStoreReadIterStream; @@ -79,7 +76,7 @@ pub trait StateStoreRead: StaticSendSync { /// The result is based on a snapshot corresponding to the given `epoch`. fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> impl Future>> + Send + '_; @@ -91,7 +88,7 @@ pub trait StateStoreRead: StaticSendSync { /// corresponding to the given `epoch`. fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> impl Future> + Send + '_; @@ -107,7 +104,7 @@ pub trait StateStoreReadExt: StaticSendSync { /// By default, this simply calls `StateStore::iter` to fetch elements. fn scan( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, limit: Option, read_options: ReadOptions, @@ -117,7 +114,7 @@ pub trait StateStoreReadExt: StaticSendSync { impl StateStoreReadExt for S { async fn scan( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, limit: Option, mut read_options: ReadOptions, @@ -151,18 +148,10 @@ pub trait StateStoreWrite: StaticSendSync { /// per-key modification history (e.g. in compaction), not across different keys. fn ingest_batch( &self, - kv_pairs: Vec<(Bytes, StorageValue)>, + kv_pairs: Vec<(TableKey, StorageValue)>, delete_ranges: Vec<(Bound, Bound)>, write_options: WriteOptions, ) -> impl Future> + Send + '_; - - /// Creates a `WriteBatch` associated with this state store. - fn start_write_batch(&self, write_options: WriteOptions) -> WriteBatch<'_, Self> - where - Self: Sized, - { - WriteBatch::new(self, write_options) - } } #[derive(Default, Debug)] @@ -213,7 +202,7 @@ pub trait LocalStateStore: StaticSendSync { /// The result is based on the latest written snapshot. fn get( &self, - key: Bytes, + key: TableKey, read_options: ReadOptions, ) -> impl Future>> + Send + '_; @@ -224,16 +213,21 @@ pub trait LocalStateStore: StaticSendSync { /// snapshot. fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future>> + Send + '_; /// Inserts a key-value entry associated with a given `epoch` into the state store. - fn insert(&mut self, key: Bytes, new_val: Bytes, old_val: Option) -> StorageResult<()>; + fn insert( + &mut self, + key: TableKey, + new_val: Bytes, + old_val: Option, + ) -> StorageResult<()>; /// Deletes a key-value entry from the state store. Only the key-value entry with epoch smaller /// than the given `epoch` will be deleted. - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()>; + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()>; fn flush( &mut self, @@ -268,7 +262,7 @@ pub trait LocalStateStore: StaticSendSync { /// - true: `key_range` may or may not exist in storage. fn may_exist( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future> + Send + '_; } diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index 54d79c11d741f..7c6220a8cc868 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -230,6 +230,7 @@ pub mod verify { use bytes::Bytes; use futures::{pin_mut, TryStreamExt}; use futures_async_stream::try_stream; + use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; use risingwave_hummock_sdk::HummockReadEpoch; use tracing::log::warn; @@ -274,7 +275,7 @@ pub mod verify { async fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> StorageResult> { @@ -294,7 +295,7 @@ pub mod verify { #[allow(clippy::manual_async_fn)] fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> impl Future> + '_ { @@ -340,7 +341,7 @@ pub mod verify { impl StateStoreWrite for VerifyStateStore { async fn ingest_batch( &self, - kv_pairs: Vec<(Bytes, StorageValue)>, + kv_pairs: Vec<(TableKey, StorageValue)>, delete_ranges: Vec<(Bound, Bound)>, write_options: WriteOptions, ) -> StorageResult { @@ -379,13 +380,17 @@ pub mod verify { // be consistent across different state store backends. fn may_exist( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future> + Send + '_ { self.actual.may_exist(key_range, read_options) } - async fn get(&self, key: Bytes, read_options: ReadOptions) -> StorageResult> { + async fn get( + &self, + key: TableKey, + read_options: ReadOptions, + ) -> StorageResult> { let actual = self.actual.get(key.clone(), read_options.clone()).await; if let Some(expected) = &self.expected { let expected = expected.get(key, read_options).await; @@ -397,7 +402,7 @@ pub mod verify { #[allow(clippy::manual_async_fn)] fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { async move { @@ -417,7 +422,7 @@ pub mod verify { fn insert( &mut self, - key: Bytes, + key: TableKey, new_val: Bytes, old_val: Option, ) -> StorageResult<()> { @@ -429,7 +434,7 @@ pub mod verify { Ok(()) } - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()> { + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()> { if let Some(expected) = &mut self.expected { expected.delete(key.clone(), old_val.clone())?; } @@ -737,6 +742,7 @@ pub mod boxed_state_store { use bytes::Bytes; use futures::stream::BoxStream; use futures::StreamExt; + use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; use risingwave_hummock_sdk::HummockReadEpoch; use crate::error::StorageResult; @@ -752,14 +758,14 @@ pub mod boxed_state_store { pub trait DynamicDispatchedStateStoreRead: StaticSendSync { async fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> StorageResult>; async fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> StorageResult; @@ -769,7 +775,7 @@ pub mod boxed_state_store { impl DynamicDispatchedStateStoreRead for S { async fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> StorageResult> { @@ -778,7 +784,7 @@ pub mod boxed_state_store { async fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> StorageResult { @@ -792,26 +798,30 @@ pub mod boxed_state_store { pub trait DynamicDispatchedLocalStateStore: StaticSendSync { async fn may_exist( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> StorageResult; - async fn get(&self, key: Bytes, read_options: ReadOptions) -> StorageResult>; + async fn get( + &self, + key: TableKey, + read_options: ReadOptions, + ) -> StorageResult>; async fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> StorageResult>; fn insert( &mut self, - key: Bytes, + key: TableKey, new_val: Bytes, old_val: Option, ) -> StorageResult<()>; - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()>; + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()>; async fn flush( &mut self, @@ -831,19 +841,23 @@ pub mod boxed_state_store { impl DynamicDispatchedLocalStateStore for S { async fn may_exist( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> StorageResult { self.may_exist(key_range, read_options).await } - async fn get(&self, key: Bytes, read_options: ReadOptions) -> StorageResult> { + async fn get( + &self, + key: TableKey, + read_options: ReadOptions, + ) -> StorageResult> { self.get(key, read_options).await } async fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> StorageResult> { Ok(self.iter(key_range, read_options).await?.boxed()) @@ -851,14 +865,14 @@ pub mod boxed_state_store { fn insert( &mut self, - key: Bytes, + key: TableKey, new_val: Bytes, old_val: Option, ) -> StorageResult<()> { self.insert(key, new_val, old_val) } - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()> { + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()> { self.delete(key, old_val) } @@ -893,7 +907,7 @@ pub mod boxed_state_store { fn may_exist( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future> + Send + '_ { self.deref().may_exist(key_range, read_options) @@ -901,7 +915,7 @@ pub mod boxed_state_store { fn get( &self, - key: Bytes, + key: TableKey, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { self.deref().get(key, read_options) @@ -909,7 +923,7 @@ pub mod boxed_state_store { fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { self.deref().iter(key_range, read_options) @@ -917,14 +931,14 @@ pub mod boxed_state_store { fn insert( &mut self, - key: Bytes, + key: TableKey, new_val: Bytes, old_val: Option, ) -> StorageResult<()> { self.deref_mut().insert(key, new_val, old_val) } - fn delete(&mut self, key: Bytes, old_val: Bytes) -> StorageResult<()> { + fn delete(&mut self, key: TableKey, old_val: Bytes) -> StorageResult<()> { self.deref_mut().delete(key, old_val) } @@ -1006,7 +1020,7 @@ pub mod boxed_state_store { fn get( &self, - key: Bytes, + key: TableKey, epoch: u64, read_options: ReadOptions, ) -> impl Future>> + Send + '_ { @@ -1015,7 +1029,7 @@ pub mod boxed_state_store { fn iter( &self, - key_range: IterKeyRange, + key_range: TableKeyRange, epoch: u64, read_options: ReadOptions, ) -> impl Future> + '_ { diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index dc386a719ec22..e4eb65b8b9f42 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -33,7 +33,9 @@ use risingwave_common::util::row_serde::*; use risingwave_common::util::sort_util::OrderType; use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; use risingwave_common::util::value_encoding::{BasicSerde, EitherSerde}; -use risingwave_hummock_sdk::key::{end_bound_of_prefix, next_key, prefixed_range}; +use risingwave_hummock_sdk::key::{ + end_bound_of_prefix, map_table_key_range, next_key, prefixed_range, TableKeyRange, +}; use risingwave_hummock_sdk::HummockReadEpoch; use tracing::trace; @@ -448,6 +450,7 @@ impl StorageTableInner { // For each key range, construct an iterator. let iterators: Vec<_> = try_join_all(raw_key_ranges.map(|raw_key_range| { + let table_key_range = map_table_key_range(raw_key_range); let prefix_hint = prefix_hint.clone(); let read_backup = matches!(wait_epoch, HummockReadEpoch::Backup(_)); async move { @@ -473,7 +476,7 @@ impl StorageTableInner { self.value_output_indices.clone(), self.output_row_in_key_indices.clone(), self.row_serde.clone(), - raw_key_range, + table_key_range, read_options, wait_epoch, ) @@ -680,13 +683,13 @@ impl StorageTableInnerIterInner { value_output_indices: Vec, output_row_in_key_indices: Vec, row_deserializer: Arc, - raw_key_range: (Bound, Bound), + table_key_range: TableKeyRange, read_options: ReadOptions, epoch: HummockReadEpoch, ) -> StorageResult { let raw_epoch = epoch.get_epoch(); store.try_wait_epoch(epoch).await?; - let iter = store.iter(raw_key_range, raw_epoch, read_options).await?; + let iter = store.iter(table_key_range, raw_epoch, read_options).await?; // For `HummockStorage`, a cluster recovery will clear storage data and make subsequent // `HummockReadEpoch::Current` read incomplete. // `validate_read_epoch` is a safeguard against that incorrect read. It rejects the read diff --git a/src/stream/src/common/log_store/kv_log_store/reader.rs b/src/stream/src/common/log_store/kv_log_store/reader.rs index f31235c42da0b..9c2a5f005bf55 100644 --- a/src/stream/src/common/log_store/kv_log_store/reader.rs +++ b/src/stream/src/common/log_store/kv_log_store/reader.rs @@ -21,6 +21,7 @@ use futures::stream::select_all; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; use risingwave_common::hash::VnodeBitmapExt; +use risingwave_hummock_sdk::key::TableKey; use risingwave_storage::hummock::CachePolicy; use risingwave_storage::store::{PrefetchOptions, ReadOptions}; use risingwave_storage::StateStore; @@ -77,7 +78,7 @@ impl LogReader for KvLogStoreReader { async fn init(&mut self) -> LogStoreResult<()> { let first_write_epoch = self.rx.init().await; let streams = try_join_all(self.serde.vnodes().iter_vnodes().map(|vnode| { - let range_start = Bytes::from(Vec::from(vnode.to_be_bytes())); + let range_start = TableKey(Bytes::from(Vec::from(vnode.to_be_bytes()))); let range_end = self.serde.serialize_epoch(vnode, first_write_epoch); let table_id = self.table_id; let state_store = self.state_store.clone(); diff --git a/src/stream/src/common/log_store/kv_log_store/serde.rs b/src/stream/src/common/log_store/kv_log_store/serde.rs index 15825e9f275e6..ebe4f2a578dd3 100644 --- a/src/stream/src/common/log_store/kv_log_store/serde.rs +++ b/src/stream/src/common/log_store/kv_log_store/serde.rs @@ -38,7 +38,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_common::util::value_encoding::{ BasicSerde, ValueRowDeserializer, ValueRowSerializer, }; -use risingwave_hummock_sdk::key::next_key; +use risingwave_hummock_sdk::key::{next_key, TableKey}; use risingwave_pb::catalog::Table; use risingwave_storage::row_serde::row_serde_util::serialize_pk_with_vnode; use risingwave_storage::row_serde::value_serde::ValueRowSerdeNew; @@ -174,7 +174,7 @@ impl LogStoreRowSerde { seq_id: SeqIdType, op: Op, row: impl Row, - ) -> (VirtualNode, Bytes, Bytes) { + ) -> (VirtualNode, TableKey, Bytes) { let pk = [ Some(ScalarImpl::Int64(Self::encode_epoch(epoch))), Some(ScalarImpl::Int32(seq_id)), @@ -200,7 +200,7 @@ impl LogStoreRowSerde { epoch: u64, vnode: VirtualNode, is_checkpoint: bool, - ) -> (Bytes, Bytes) { + ) -> (TableKey, Bytes) { let pk = [Some(ScalarImpl::Int64(Self::encode_epoch(epoch))), None]; let op_code = if is_checkpoint { @@ -218,7 +218,7 @@ impl LogStoreRowSerde { (key_bytes, value_bytes) } - pub(crate) fn serialize_epoch(&self, vnode: VirtualNode, epoch: u64) -> Bytes { + pub(crate) fn serialize_epoch(&self, vnode: VirtualNode, epoch: u64) -> TableKey { serialize_pk_with_vnode( [Some(ScalarImpl::Int64(Self::encode_epoch(epoch)))], &self.epoch_serde, @@ -231,7 +231,7 @@ impl LogStoreRowSerde { vnode: VirtualNode, epoch: u64, seq_id: SeqIdType, - ) -> Bytes { + ) -> TableKey { serialize_pk_with_vnode( [ Some(ScalarImpl::Int64(Self::encode_epoch(epoch))), @@ -580,7 +580,7 @@ mod tests { use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::DataType; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; - use risingwave_hummock_sdk::key::{FullKey, TableKey}; + use risingwave_hummock_sdk::key::FullKey; use risingwave_storage::store::StateStoreReadIterStream; use risingwave_storage::table::DEFAULT_VNODE; use tokio::sync::oneshot; @@ -622,7 +622,7 @@ mod tests { for (op, row) in stream_chunk.rows() { let (_, key, value) = serde.serialize_data_row(epoch, seq_id, op, row); - assert!(key < delete_range_right1); + assert!(key.as_ref() < delete_range_right1); serialized_keys.push(key); let (decoded_epoch, row_op) = serde.deserialize(value).unwrap(); assert_eq!(decoded_epoch, epoch); @@ -647,7 +647,7 @@ mod tests { } _ => unreachable!(), } - assert!(key < delete_range_right1); + assert!(key.as_ref() < delete_range_right1); serialized_keys.push(key); seq_id = 1; @@ -657,8 +657,8 @@ mod tests { for (op, row) in stream_chunk.rows() { let (_, key, value) = serde.serialize_data_row(epoch, seq_id, op, row); - assert!(key >= delete_range_right1); - assert!(key < delete_range_right2); + assert!(key.as_ref() >= delete_range_right1); + assert!(key.as_ref() < delete_range_right2); serialized_keys.push(key); let (decoded_epoch, row_op) = serde.deserialize(value).unwrap(); assert_eq!(decoded_epoch, epoch); @@ -683,8 +683,8 @@ mod tests { } _ => unreachable!(), } - assert!(key >= delete_range_right1); - assert!(key < delete_range_right2); + assert!(key.as_ref() >= delete_range_right1); + assert!(key.as_ref() < delete_range_right2); serialized_keys.push(key); assert_eq!(serialized_keys.len(), 2 * rows.len() + 2); @@ -761,7 +761,7 @@ mod tests { .map(|(op, row)| { let (_, key, value) = serde.serialize_data_row(epoch, *seq_id, op, row); *seq_id += 1; - Ok((FullKey::new(TEST_TABLE_ID, TableKey(key), epoch), value)) + Ok((FullKey::new(TEST_TABLE_ID, key, epoch), value)) }) .collect_vec(); ( @@ -790,14 +790,14 @@ mod tests { let serde = serde.clone(); async move { let (key, value) = serde.serialize_barrier(EPOCH1, DEFAULT_VNODE, false); - Ok((FullKey::new(TEST_TABLE_ID, TableKey(key), EPOCH1), value)) + Ok((FullKey::new(TEST_TABLE_ID, key, EPOCH1), value)) } })); let (row_stream, tx2) = gen_row_stream(serde.clone(), ops, rows, EPOCH2, seq_id); let stream = stream.chain(row_stream).chain(stream::once({ async move { let (key, value) = serde.serialize_barrier(EPOCH2, DEFAULT_VNODE, true); - Ok((FullKey::new(TEST_TABLE_ID, TableKey(key), EPOCH2), value)) + Ok((FullKey::new(TEST_TABLE_ID, key, EPOCH2), value)) } })); (stream, tx1, tx2) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 045fb1fdaeba9..2d06a3721a8c3 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -35,7 +35,8 @@ use risingwave_common::util::row_serde::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; use risingwave_common::util::value_encoding::BasicSerde; use risingwave_hummock_sdk::key::{ - end_bound_of_prefix, next_key, prefixed_range, range_of_prefix, start_bound_of_excluded_prefix, + end_bound_of_prefix, map_table_key_range, next_key, prefixed_range, range_of_prefix, + start_bound_of_excluded_prefix, TableKey, }; use risingwave_pb::catalog::Table; use risingwave_storage::error::{StorageError, StorageResult}; @@ -717,21 +718,21 @@ where } } - fn insert_inner(&mut self, key_bytes: Bytes, value_bytes: Bytes) { + fn insert_inner(&mut self, key: TableKey, value_bytes: Bytes) { self.local_store - .insert(key_bytes, value_bytes, None) + .insert(key, value_bytes, None) .unwrap_or_else(|e| self.handle_mem_table_error(e)); } - fn delete_inner(&mut self, key_bytes: Bytes, value_bytes: Bytes) { + fn delete_inner(&mut self, key: TableKey, value_bytes: Bytes) { self.local_store - .delete(key_bytes, value_bytes) + .delete(key, value_bytes) .unwrap_or_else(|e| self.handle_mem_table_error(e)); } fn update_inner( &mut self, - key_bytes: Bytes, + key_bytes: TableKey, old_value_bytes: Option, new_value_bytes: Bytes, ) { @@ -855,13 +856,13 @@ where if USE_WATERMARK_CACHE && let Some(ref pk) = key { self.watermark_cache.insert(pk); } - self.insert_inner(key_bytes, value); + self.insert_inner(TableKey(key_bytes), value); } Op::Delete | Op::UpdateDelete => { if USE_WATERMARK_CACHE && let Some(ref pk) = key { self.watermark_cache.delete(pk); } - self.delete_inner(key_bytes, value); + self.delete_inner(TableKey(key_bytes), value); } } } @@ -874,13 +875,13 @@ where if USE_WATERMARK_CACHE && let Some(ref pk) = key { self.watermark_cache.insert(pk); } - self.insert_inner(key_bytes, value); + self.insert_inner(TableKey(key_bytes), value); } Op::Delete | Op::UpdateDelete => { if USE_WATERMARK_CACHE && let Some(ref pk) = key { self.watermark_cache.delete(pk); } - self.delete_inner(key_bytes, value); + self.delete_inner(TableKey(key_bytes), value); } } } @@ -1147,8 +1148,9 @@ where prefetch_options, cache_policy: CachePolicy::Fill(CachePriority::High), }; + let table_key_range = map_table_key_range(key_range); - Ok(self.local_store.iter(key_range, read_options).await?) + Ok(self.local_store.iter(table_key_range, read_options).await?) } /// This function scans raw key-values from the relational table with specific `pk_prefix`. @@ -1255,7 +1257,7 @@ where // If this assertion fails, then something must be wrong with the operator implementation or // the distribution derivation from the optimizer. let vnode = self.compute_prefix_vnode(&pk_prefix).to_be_bytes(); - let encoded_key_range_with_vnode = prefixed_range(encoded_key_range, &vnode); + let table_key_range = map_table_key_range(prefixed_range(encoded_key_range, &vnode)); // Construct prefix hint for prefix bloom filter. if self.prefix_hint_len != 0 { @@ -1284,7 +1286,7 @@ where }; self.local_store - .may_exist(encoded_key_range_with_vnode, read_options) + .may_exist(table_key_range, read_options) .await .map_err(Into::into) } diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index db248ad788fba..3e2f993cf9613 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -31,6 +31,7 @@ use risingwave_common::config::{ }; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_hummock_sdk::key::TableKey; use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, FIRST_VERSION_ID}; use risingwave_pb::common::WorkerType; use risingwave_pb::hummock::{HummockVersion, HummockVersionDelta}; @@ -619,10 +620,11 @@ async fn open_hummock_iters( buf.put_u32(table_id); let b = buf.freeze(); let range = ( - Bound::Included(b.clone()), + Bound::Included(b.clone()).map(TableKey), Bound::Excluded(Bytes::from(risingwave_hummock_sdk::key::next_key( b.as_ref(), - ))), + ))) + .map(TableKey), ); for &epoch in snapshots { diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 258fb62d3b740..5b53ba70b86bd 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -29,6 +29,7 @@ use risingwave_common::catalog::hummock::PROPERTIES_RETENTION_SECOND_KEY; use risingwave_common::catalog::TableId; use risingwave_common::config::{extract_storage_memory_config, load_config, NoOverride, RwConfig}; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_hummock_sdk::key::TableKey; use risingwave_hummock_test::get_notification_client_for_test; use risingwave_hummock_test::local_state_store_test_utils::LocalStateStoreTestExt; use risingwave_meta::hummock::compaction::compaction_config::CompactionConfigBuilder; @@ -420,7 +421,7 @@ impl NormalState { async fn get_impl(&self, key: &[u8], ignore_range_tombstone: bool) -> Option { self.storage .get( - Bytes::copy_from_slice(key), + TableKey(Bytes::copy_from_slice(key)), ReadOptions { prefix_hint: None, ignore_range_tombstone, @@ -445,8 +446,8 @@ impl NormalState { .storage .iter( ( - Bound::Included(Bytes::copy_from_slice(left)), - Bound::Excluded(Bytes::copy_from_slice(right)), + Bound::Included(TableKey(Bytes::copy_from_slice(left))), + Bound::Excluded(TableKey(Bytes::copy_from_slice(right))), ), ReadOptions { prefix_hint: None, @@ -477,8 +478,8 @@ impl CheckState for NormalState { self.storage .iter( ( - Bound::Included(Bytes::copy_from_slice(left)), - Bound::Excluded(Bytes::copy_from_slice(right)), + Bound::Included(Bytes::copy_from_slice(left)).map(TableKey), + Bound::Excluded(Bytes::copy_from_slice(right)).map(TableKey), ), ReadOptions { prefix_hint: None, @@ -496,7 +497,7 @@ impl CheckState for NormalState { let mut delete_item = Vec::new(); while let Some(item) = iter.next().await { let (full_key, value) = item.unwrap(); - delete_item.push((full_key.user_key.table_key.0, value)); + delete_item.push((full_key.user_key.table_key, value)); } drop(iter); for (key, value) in delete_item { @@ -506,7 +507,11 @@ impl CheckState for NormalState { fn insert(&mut self, key: &[u8], val: &[u8]) { self.storage - .insert(Bytes::from(key.to_vec()), Bytes::copy_from_slice(val), None) + .insert( + TableKey(Bytes::from(key.to_vec())), + Bytes::copy_from_slice(val), + None, + ) .unwrap(); } diff --git a/src/tests/compaction_test/src/lib.rs b/src/tests/compaction_test/src/lib.rs index bbd1174d0c7fe..0bb1a990eebb7 100644 --- a/src/tests/compaction_test/src/lib.rs +++ b/src/tests/compaction_test/src/lib.rs @@ -23,6 +23,7 @@ #![warn(clippy::no_effect_underscore_binding)] #![warn(clippy::await_holding_lock)] #![deny(rustdoc::broken_intra_doc_links)] +#![feature(bound_map)] mod compaction_test_runner; mod delete_range_runner; From cfcc266011c2a5b49c6f3192bc7ffdaea17466c2 Mon Sep 17 00:00:00 2001 From: Patrick Huang Date: Fri, 8 Sep 2023 14:29:32 +0800 Subject: [PATCH 2/6] rebase rebase --- src/storage/src/hummock/mod.rs | 5 ----- src/storage/src/hummock/store/hummock_storage.rs | 5 +++-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/storage/src/hummock/mod.rs b/src/storage/src/hummock/mod.rs index f0f9f981c393f..5634fbd56086a 100644 --- a/src/storage/src/hummock/mod.rs +++ b/src/storage/src/hummock/mod.rs @@ -16,7 +16,6 @@ use std::ops::Bound; use std::sync::Arc; -use std::time::Duration; use bytes::Bytes; use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKeyRangeRef}; @@ -26,10 +25,6 @@ use risingwave_pb::hummock::SstableInfo; mod block_cache; pub use block_cache::*; -use crate::filter_key_extractor::RpcFilterKeyExtractorManager; -use crate::hummock::store::state_store::LocalHummockStorage; -use crate::opts::StorageOpts; - mod file_cache; pub use file_cache::*; diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index e5b3c95867d3c..a67bbe8cfc513 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -37,10 +37,11 @@ use tracing::log::error; use super::local_hummock_storage::{HummockStorageIterator, LocalHummockStorage}; use super::version::{CommittedVersion, HummockVersionReader}; use crate::error::StorageResult; -use crate::filter_key_extractor::{FilterKeyExtractorManager, FilterKeyExtractorManagerRef}; +use crate::filter_key_extractor::{FilterKeyExtractorManager, RpcFilterKeyExtractorManager}; use crate::hummock::backup_reader::{BackupReader, BackupReaderRef}; use crate::hummock::compactor::CompactorContext; use crate::hummock::event_handler::hummock_event_handler::BufferTracker; +use crate::hummock::event_handler::refiller::CacheRefillConfig; use crate::hummock::event_handler::{HummockEvent, HummockEventHandler, ReadVersionMappingType}; use crate::hummock::local_version::pinned_version::{start_pinned_version_worker, PinnedVersion}; use crate::hummock::observer_manager::HummockObserverNode; @@ -545,7 +546,7 @@ impl HummockStorage { sstable_store, hummock_meta_client, notification_client, - Arc::new(FilterKeyExtractorManager::default()), + Arc::new(RpcFilterKeyExtractorManager::default()), Arc::new(HummockStateStoreMetrics::unused()), Arc::new(CompactorMetrics::unused()), ) From 2eda42fdb6acd33829715647ea0aec5a7b43a904 Mon Sep 17 00:00:00 2001 From: Patrick Huang Date: Fri, 8 Sep 2023 15:42:22 +0800 Subject: [PATCH 3/6] fix tests --- src/rpc_client/src/meta_client.rs | 2 +- .../hummock_test/src/hummock_storage_tests.rs | 72 +++++++------------ .../hummock_test/src/state_store_tests.rs | 30 +++----- 3 files changed, 35 insertions(+), 69 deletions(-) diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 50cab50c92682..2b695f9c045b0 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -1489,7 +1489,7 @@ impl GrpcMetaClient { force_refresh_receiver: Receiver>>, meta_config: MetaConfig, ) -> Result<()> { - let core_ref = self.core.clone(); + let core_ref: Arc> = self.core.clone(); let current_leader = init_leader_addr; let enable_period_tick = matches!(members, Either::Right(_)); diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 8fb1b93e2d45e..ce91c109a814b 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -256,11 +256,9 @@ async fn test_storage_basic() { futures::pin_mut!(iter); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch1 ), Bytes::copy_from_slice(&b"111"[..]) @@ -269,11 +267,9 @@ async fn test_storage_basic() { ); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1 ), Bytes::copy_from_slice(&b"222"[..]) @@ -337,11 +333,9 @@ async fn test_storage_basic() { futures::pin_mut!(iter); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch2 ), Bytes::copy_from_slice(&b"111111"[..]) @@ -350,11 +344,9 @@ async fn test_storage_basic() { ); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1 ), Bytes::copy_from_slice(&b"222"[..]) @@ -363,11 +355,9 @@ async fn test_storage_basic() { ); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "cc"), epoch2 ), Bytes::copy_from_slice(&b"333"[..]) @@ -396,11 +386,9 @@ async fn test_storage_basic() { futures::pin_mut!(iter); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1 ), Bytes::copy_from_slice(&b"222"[..]) @@ -409,11 +397,9 @@ async fn test_storage_basic() { ); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "cc"), epoch2 ), Bytes::copy_from_slice(&b"333"[..]) @@ -422,11 +408,9 @@ async fn test_storage_basic() { ); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"dd".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "dd"), epoch3 ), Bytes::copy_from_slice(&b"444"[..]) @@ -435,11 +419,9 @@ async fn test_storage_basic() { ); assert_eq!( Some(( - FullKey::for_test( + FullKey::new( TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"ee".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "ee"), epoch3 ), Bytes::copy_from_slice(&b"555"[..]) @@ -684,11 +666,11 @@ async fn test_state_store_sync() { futures::pin_mut!(iter); let kv_map = [ - (b"aaaa", "1111", epoch1), - (b"bbbb", "2222", epoch1), - (b"cccc", "3333", epoch1), - (b"dddd", "4444", epoch1), - (b"eeee", "6666", epoch2), + ("aaaa", "1111", epoch1), + ("bbbb", "2222", epoch1), + ("cccc", "3333", epoch1), + ("dddd", "4444", epoch1), + ("eeee", "6666", epoch2), ]; for (k, v, e) in kv_map { @@ -696,13 +678,7 @@ async fn test_state_store_sync() { assert_eq!( result, Some(( - FullKey::for_test( - TEST_TABLE_ID, - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), k.as_slice()].concat() - ), - e - ), + FullKey::new(TEST_TABLE_ID, gen_key_from_str(VirtualNode::ZERO, k), e), Bytes::from(v) )) ); diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 21d132f689d2b..d938d87f227e0 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -795,11 +795,9 @@ async fn test_write_anytime_inner( futures::pin_mut!(iter); assert_eq!( ( - FullKey::for_test( + FullKey::new( TableId::default(), - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch ), Bytes::from("111") @@ -808,11 +806,9 @@ async fn test_write_anytime_inner( ); assert_eq!( ( - FullKey::for_test( + FullKey::new( TableId::default(), - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"bb".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "bb"), epoch ), Bytes::from("222") @@ -821,11 +817,9 @@ async fn test_write_anytime_inner( ); assert_eq!( ( - FullKey::for_test( + FullKey::new( TableId::default(), - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "cc"), epoch ), Bytes::from("333") @@ -931,11 +925,9 @@ async fn test_write_anytime_inner( futures::pin_mut!(iter); assert_eq!( ( - FullKey::for_test( + FullKey::new( TableId::default(), - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"aa".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "aa"), epoch ), Bytes::from("111_new") @@ -944,11 +936,9 @@ async fn test_write_anytime_inner( ); assert_eq!( ( - FullKey::for_test( + FullKey::new( TableId::default(), - Bytes::from( - [VirtualNode::ZERO.to_be_bytes().as_slice(), b"cc".as_slice()].concat() - ), + gen_key_from_str(VirtualNode::ZERO, "cc"), epoch ), Bytes::from("333") From bb7468443208d1d157b6180b8c9eb1de8c3be217 Mon Sep 17 00:00:00 2001 From: Patrick Huang Date: Fri, 8 Sep 2023 17:17:51 +0800 Subject: [PATCH 4/6] fix tests --- src/storage/hummock_sdk/src/key.rs | 2 +- .../hummock_test/src/sync_point_tests.rs | 38 +++++++++---------- src/storage/src/mem_table.rs | 1 + 3 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/storage/hummock_sdk/src/key.rs b/src/storage/hummock_sdk/src/key.rs index 34a28f51aa95f..d635bb4518a4d 100644 --- a/src/storage/hummock_sdk/src/key.rs +++ b/src/storage/hummock_sdk/src/key.rs @@ -414,7 +414,7 @@ impl> TableKey { impl EstimateSize for TableKey { fn estimated_heap_size(&self) -> usize { - self.0.estimated_size() + self.0.estimated_heap_size() } } diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index f8388e66343a5..c5e30bd489377 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -310,7 +310,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { let val1 = Bytes::from(b"1"[..].repeat(1 << 10)); // 1024 Byte value local.init_for_test(100).await.unwrap(); - let mut start_key = b"\0\0aaa".to_vec(); + let mut start_key = b"aaa".to_vec(); for _ in 0..10 { local .insert( @@ -323,21 +323,21 @@ async fn test_syncpoints_get_in_delete_range_boundary() { } local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0ggg"), + gen_key_from_bytes(VirtualNode::ZERO, b"ggg"), val0.clone(), None, ) .unwrap(); local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0hhh"), + gen_key_from_bytes(VirtualNode::ZERO, b"hhh"), val0.clone(), None, ) .unwrap(); local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0kkk"), + gen_key_from_bytes(VirtualNode::ZERO, b"kkk"), val0.clone(), None, ) @@ -354,22 +354,22 @@ async fn test_syncpoints_get_in_delete_range_boundary() { local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0aaa"), + gen_key_from_bytes(VirtualNode::ZERO, b"aaa"), val1.clone(), None, ) .unwrap(); local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0bbb"), + gen_key_from_bytes(VirtualNode::ZERO, b"bbb"), val1.clone(), None, ) .unwrap(); local .flush(vec![( - Bound::Included(Bytes::from(b"\0\0ggg".as_slice())), - Bound::Excluded(Bytes::from(b"\0\0hhh".as_slice())), + Bound::Included(Bytes::from(b"ggg".as_slice())), + Bound::Excluded(Bytes::from(b"hhh".as_slice())), )]) .await .unwrap(); @@ -384,22 +384,22 @@ async fn test_syncpoints_get_in_delete_range_boundary() { local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0hhh"), + gen_key_from_bytes(VirtualNode::ZERO, b"hhh"), val1.clone(), None, ) .unwrap(); local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0iii"), + gen_key_from_bytes(VirtualNode::ZERO, b"iii"), val1.clone(), None, ) .unwrap(); local .flush(vec![( - Bound::Included(Bytes::from(b"\0\0jjj".as_slice())), - Bound::Excluded(Bytes::from(b"\0\0kkk".as_slice())), + Bound::Included(Bytes::from(b"jjj".as_slice())), + Bound::Excluded(Bytes::from(b"kkk".as_slice())), )]) .await .unwrap(); @@ -415,14 +415,14 @@ async fn test_syncpoints_get_in_delete_range_boundary() { local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0lll"), + gen_key_from_bytes(VirtualNode::ZERO, b"lll"), val1.clone(), None, ) .unwrap(); local .insert( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0mmm"), + gen_key_from_bytes(VirtualNode::ZERO, b"mmm"), val1.clone(), None, ) @@ -463,7 +463,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { }; let get_result = storage .get( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0hhh"), + gen_key_from_bytes(VirtualNode::ZERO, b"hhh"), 120, read_options.clone(), ) @@ -472,7 +472,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { assert_eq!(get_result.unwrap(), val1); let get_result = storage .get( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0ggg"), + gen_key_from_bytes(VirtualNode::ZERO, b"ggg"), 120, read_options.clone(), ) @@ -481,7 +481,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { assert!(get_result.is_none()); let get_result = storage .get( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0aaa"), + gen_key_from_bytes(VirtualNode::ZERO, b"aaa"), 120, read_options.clone(), ) @@ -490,7 +490,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { assert_eq!(get_result.unwrap(), val1); let get_result = storage .get( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0aab"), + gen_key_from_bytes(VirtualNode::ZERO, b"aab"), 120, read_options.clone(), ) @@ -507,7 +507,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { }); let get_result = storage .get( - gen_key_from_bytes(VirtualNode::ZERO, b"\0\0kkk"), + gen_key_from_bytes(VirtualNode::ZERO, b"kkk"), 120, read_options.clone(), ) diff --git a/src/storage/src/mem_table.rs b/src/storage/src/mem_table.rs index daa810afd96e8..34508a730a5fb 100644 --- a/src/storage/src/mem_table.rs +++ b/src/storage/src/mem_table.rs @@ -702,6 +702,7 @@ mod tests { mem_table .insert(TableKey("key1".into()), "value1".into()) .unwrap(); + assert_eq!( mem_table.kv_size.size(), std::mem::size_of::() From c5f5e0456bb6cb1baf46a524c089a0f716248267 Mon Sep 17 00:00:00 2001 From: Patrick Huang Date: Tue, 12 Sep 2023 13:49:44 +0800 Subject: [PATCH 5/6] fix tests --- src/storage/hummock_test/src/state_store_tests.rs | 2 +- src/storage/hummock_test/src/sync_point_tests.rs | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index d938d87f227e0..3b9752c6c6fe4 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -912,7 +912,7 @@ async fn test_write_anytime_inner( .iter( ( Bound::Included(gen_key_from_str(VirtualNode::ZERO, "aa")), - Bound::Included(gen_key_from_str(VirtualNode::ZERO, "bb")), + Bound::Included(gen_key_from_str(VirtualNode::ZERO, "cc")), ), epoch, ReadOptions { diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index c5e30bd489377..d28db261785c0 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -368,8 +368,8 @@ async fn test_syncpoints_get_in_delete_range_boundary() { .unwrap(); local .flush(vec![( - Bound::Included(Bytes::from(b"ggg".as_slice())), - Bound::Excluded(Bytes::from(b"hhh".as_slice())), + Bound::Included(Bytes::from(b"\0\0ggg".as_slice())), + Bound::Excluded(Bytes::from(b"\0\0hhh".as_slice())), )]) .await .unwrap(); @@ -398,8 +398,8 @@ async fn test_syncpoints_get_in_delete_range_boundary() { .unwrap(); local .flush(vec![( - Bound::Included(Bytes::from(b"jjj".as_slice())), - Bound::Excluded(Bytes::from(b"kkk".as_slice())), + Bound::Included(Bytes::from(b"\0\0jjj".as_slice())), + Bound::Excluded(Bytes::from(b"\0\0kkk".as_slice())), )]) .await .unwrap(); From d6a0d95464c18cff3fa377e515798d367b3bbcb6 Mon Sep 17 00:00:00 2001 From: Patrick Huang Date: Tue, 12 Sep 2023 14:14:19 +0800 Subject: [PATCH 6/6] clippy --- src/storage/src/hummock/store/hummock_storage.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index a67bbe8cfc513..a4bafcdb99c07 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -296,7 +296,7 @@ impl HummockStorage { .cloned() .collect_vec() }) - .unwrap_or(Vec::new()) + .unwrap_or_default() }; // When the system has just started and no state has been created, the memory state