From 86f0db9fb8fcd9ac4e9ef8afa092c53c00285846 Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 23 Oct 2024 00:29:46 +0800 Subject: [PATCH 1/8] refactor(storage): refactor filter_key_extractor to compaction_catalog_manager --- src/compute/src/server.rs | 2 +- src/meta/src/hummock/test_utils.rs | 1 + src/storage/benches/bench_compactor.rs | 20 +- src/storage/benches/bench_multi_builder.rs | 8 +- .../compactor_observer/observer_manager.rs | 27 +- src/storage/compactor/src/server.rs | 14 +- .../hummock_test/src/bin/replay/main.rs | 12 +- .../hummock_test/src/compactor_tests.rs | 311 +++++++++--------- .../hummock_test/src/sync_point_tests.rs | 28 +- src/storage/hummock_test/src/test_utils.rs | 58 ++-- ...actor.rs => compaction_catalog_manager.rs} | 236 ++++++------- .../src/hummock/compactor/compaction_utils.rs | 6 +- .../src/hummock/compactor/compactor_runner.rs | 76 +---- .../compactor/fast_compactor_runner.rs | 6 +- src/storage/src/hummock/compactor/mod.rs | 72 ++-- .../compactor/shared_buffer_compact.rs | 30 +- .../event_handler/hummock_event_handler.rs | 12 +- src/storage/src/hummock/observer_manager.rs | 25 +- src/storage/src/hummock/sstable/builder.rs | 58 +++- .../src/hummock/sstable/multi_builder.rs | 9 +- src/storage/src/hummock/sstable/xor_filter.rs | 16 +- .../src/hummock/store/hummock_storage.rs | 27 +- src/storage/src/hummock/test_utils.rs | 23 +- src/storage/src/lib.rs | 2 +- src/storage/src/store_impl.rs | 12 +- 25 files changed, 531 insertions(+), 560 deletions(-) rename src/storage/src/{filter_key_extractor.rs => compaction_catalog_manager.rs} (75%) diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 909b4f96b7a14..da3328b0b2ced 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -256,7 +256,7 @@ pub async fn compute_node_serve( compactor_context, hummock_meta_client.clone(), storage.sstable_object_id_manager().clone(), - storage.filter_key_extractor_manager().clone(), + storage.compaction_catalog_manager_ref().clone(), ); sub_tasks.push((handle, shutdown_sender)); } diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 006b31475461d..b6631a672e385 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -336,6 +336,7 @@ pub async fn setup_compute_env_with_metric( compactor_streams_change_tx, ) .await; + let fake_host_address = HostAddress { host: "127.0.0.1".to_string(), port, diff --git a/src/storage/benches/bench_compactor.rs b/src/storage/benches/bench_compactor.rs index 75ab989559d92..e46951202e0f5 100644 --- a/src/storage/benches/bench_compactor.rs +++ b/src/storage/benches/bench_compactor.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::ops::Range; use std::sync::Arc; @@ -133,8 +134,13 @@ async fn build_table( policy: CachePolicy::Fill(CacheContext::Default), }, ); - let mut builder = - SstableBuilder::<_, Xor16FilterBuilder>::for_test(sstable_object_id, writer, opt); + let table_id_to_vnode = HashMap::from_iter(vec![(0, VirtualNode::COUNT_FOR_TEST)]); + let mut builder = SstableBuilder::<_, Xor16FilterBuilder>::for_test( + sstable_object_id, + writer, + opt, + table_id_to_vnode, + ); let value = b"1234567890123456789"; let mut full_key = test_key_of(0, epoch, TableId::new(0)); let table_key_len = full_key.user_key.table_key.len(); @@ -177,8 +183,14 @@ async fn build_table_2( policy: CachePolicy::Fill(CacheContext::Default), }, ); - let mut builder = - SstableBuilder::<_, Xor16FilterBuilder>::for_test(sstable_object_id, writer, opt); + + let table_id_to_vnode = HashMap::from_iter(vec![(table_id, VirtualNode::COUNT_FOR_TEST)]); + let mut builder = SstableBuilder::<_, Xor16FilterBuilder>::for_test( + sstable_object_id, + writer, + opt, + table_id_to_vnode, + ); let mut full_key = test_key_of(0, epoch, TableId::new(table_id)); let table_key_len = full_key.user_key.table_key.len(); diff --git a/src/storage/benches/bench_multi_builder.rs b/src/storage/benches/bench_multi_builder.rs index 2a7b379a55269..d37e667e75ea9 100644 --- a/src/storage/benches/bench_multi_builder.rs +++ b/src/storage/benches/bench_multi_builder.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::env; use std::ops::Range; use std::sync::atomic::AtomicU64; @@ -24,6 +25,7 @@ use foyer::{Engine, HybridCacheBuilder}; use rand::random; use risingwave_common::catalog::TableId; use risingwave_common::config::{MetricLevel, ObjectStoreConfig}; +use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::key::{FullKey, UserKey}; use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_object_store::object::{ @@ -83,7 +85,11 @@ impl TableBuilderFactory for LocalTableBuilderFactory Self { Self { - filter_key_extractor_manager, + compaction_catalog_manager, system_params_manager, version: 0, } } fn handle_catalog_snapshot(&mut self, tables: Vec) { - let all_filter_key_extractors: HashMap> = tables - .iter() - .map(|t| (t.id, Arc::new(FilterKeyExtractorImpl::from_table(t)))) - .collect(); - self.filter_key_extractor_manager - .sync(all_filter_key_extractors); + self.compaction_catalog_manager + .sync(tables.into_iter().map(|t| (t.id, Arc::new(t))).collect()); } fn handle_catalog_notification(&mut self, operation: Operation, table_catalog: Table) { match operation { Operation::Add | Operation::Update => { - self.filter_key_extractor_manager.update( - table_catalog.id, - Arc::new(FilterKeyExtractorImpl::from_table(&table_catalog)), - ); + self.compaction_catalog_manager + .update(table_catalog.id, Arc::new(table_catalog)); } Operation::Delete => { - self.filter_key_extractor_manager.remove(table_catalog.id); + self.compaction_catalog_manager.remove(table_catalog.id); } _ => panic!("receive an unsupported notify {:?}", operation), diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index e604cd34a2a1f..72ae1542f116a 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -36,8 +36,8 @@ use risingwave_pb::common::WorkerType; use risingwave_pb::compactor::compactor_service_server::CompactorServiceServer; use risingwave_pb::monitor_service::monitor_service_server::MonitorServiceServer; use risingwave_rpc_client::{GrpcCompactorProxyClient, MetaClient}; -use risingwave_storage::filter_key_extractor::{ - FilterKeyExtractorManager, RemoteTableAccessor, RpcFilterKeyExtractorManager, +use risingwave_storage::compaction_catalog_manager::{ + CompactionCatalogManager, RemoteTableAccessor, }; use risingwave_storage::hummock::compactor::{ new_compaction_await_tree_reg_ref, CompactionAwaitTreeRegRef, CompactionExecutor, @@ -212,12 +212,13 @@ pub async fn compactor_serve( compactor_metrics, ) = prepare_start_parameters(config.clone(), system_params_reader.clone()).await; - let filter_key_extractor_manager = Arc::new(RpcFilterKeyExtractorManager::new(Box::new( + let compaction_catalog_manager_ref = Arc::new(CompactionCatalogManager::new(Box::new( RemoteTableAccessor::new(meta_client.clone()), ))); + let system_params_manager = Arc::new(LocalSystemParamsManager::new(system_params_reader)); let compactor_observer_node = CompactorObserverNode::new( - filter_key_extractor_manager.clone(), + compaction_catalog_manager_ref.clone(), system_params_manager.clone(), ); let observer_manager = @@ -234,9 +235,6 @@ pub async fn compactor_serve( hummock_meta_client.clone(), storage_opts.sstable_id_remote_fetch_number, )); - let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - filter_key_extractor_manager.clone(), - ); let compaction_executor = Arc::new(CompactionExecutor::new( opts.compaction_worker_threads_number, @@ -263,7 +261,7 @@ pub async fn compactor_serve( compactor_context.clone(), hummock_meta_client.clone(), sstable_object_id_manager.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_manager_ref, ), ]; diff --git a/src/storage/hummock_test/src/bin/replay/main.rs b/src/storage/hummock_test/src/bin/replay/main.rs index 4dce34153f8a5..98ee3ba448ee7 100644 --- a/src/storage/hummock_test/src/bin/replay/main.rs +++ b/src/storage/hummock_test/src/bin/replay/main.rs @@ -40,8 +40,8 @@ use risingwave_hummock_trace::{ use risingwave_meta::hummock::test_utils::setup_compute_env; use risingwave_meta::hummock::MockHummockMetaClient; use risingwave_object_store::object::build_remote_object_store; -use risingwave_storage::filter_key_extractor::{ - FakeRemoteTableAccessor, RpcFilterKeyExtractorManager, +use risingwave_storage::compaction_catalog_manager::{ + CompactionCatalogManager, FakeRemoteTableAccessor, }; use risingwave_storage::hummock::{HummockStorage, SstableStore, SstableStoreConfig}; use risingwave_storage::monitor::{CompactorMetrics, HummockStateStoreMetrics, ObjectStoreMetrics}; @@ -166,16 +166,14 @@ async fn create_replay_hummock(r: Record, args: &Args) -> Result, @@ -111,7 +114,7 @@ pub(crate) mod tests { .unwrap(); register_tables_with_id_for_test( - hummock.filter_key_extractor_manager(), + hummock.compaction_catalog_manager_ref(), hummock_manager_ref, table_ids, ) @@ -197,7 +200,7 @@ pub(crate) mod tests { } } - fn get_compactor_context(storage: &HummockStorage) -> CompactorContext { + pub fn get_compactor_context(storage: &HummockStorage) -> CompactorContext { get_compactor_context_impl(storage.storage_opts().clone(), storage.sstable_store()) } @@ -240,16 +243,6 @@ pub(crate) mod tests { ) .await; - let rpc_filter_key_extractor_manager = match storage.filter_key_extractor_manager().clone() - { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ) => rpc_filter_key_extractor_manager, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager(_) => unreachable!(), - }; - let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ); let compact_ctx = get_compactor_context(&storage); let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( hummock_meta_client.clone(), @@ -276,8 +269,13 @@ pub(crate) mod tests { ) .await; - // 2. get compact task + let compaction_catalog_agent_ref = storage + .compaction_catalog_manager_ref() + .acquire(vec![table_id]) + .await + .unwrap(); + // 2. get compact task let compaction_group_id = get_compaction_group_id_by_table_id(hummock_manager_ref.clone(), table_id).await; while let Some(compact_task) = hummock_manager_ref @@ -292,7 +290,7 @@ pub(crate) mod tests { compact_task.clone(), rx, Box::new(sstable_object_id_manager.clone()), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), ) .await; @@ -404,29 +402,6 @@ pub(crate) mod tests { } } - pub fn prepare_compactor_and_filter( - storage: &HummockStorage, - existing_table_id: u32, - ) -> (CompactorContext, FilterKeyExtractorManager) { - let rpc_filter_key_extractor_manager = match storage.filter_key_extractor_manager().clone() - { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ) => rpc_filter_key_extractor_manager, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager(_) => unreachable!(), - }; - rpc_filter_key_extractor_manager.update( - existing_table_id, - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), - ); - - let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ); - - (get_compactor_context(storage), filter_key_extractor_manager) - } - #[tokio::test] async fn test_compaction_drop_all_key() { let (env, hummock_manager_ref, cluster_ctl_ref, worker_id) = setup_compute_env(8080).await; @@ -435,16 +410,18 @@ pub(crate) mod tests { worker_id as _, )); + let notification_client = get_notification_client_for_test( + env, + hummock_manager_ref.clone(), + cluster_ctl_ref, + worker_id, + ) + .await; + let existing_table_id: u32 = 1; let storage_existing_table_id = get_hummock_storage( hummock_meta_client.clone(), - get_notification_client_for_test( - env, - hummock_manager_ref.clone(), - cluster_ctl_ref, - worker_id, - ) - .await, + notification_client, &hummock_manager_ref, &[existing_table_id], ) @@ -483,8 +460,6 @@ pub(crate) mod tests { .get_sst_ids_by_group_id(compaction_group_id) .collect_vec() .is_empty()); - - // assert_eq!(0, current_version.num_levels(compaction_group_id)); } #[tokio::test] @@ -519,26 +494,17 @@ pub(crate) mod tests { let table_id_2 = storage_2.table_id(); let table_id_set = HashSet::from_iter([table_id_1, table_id_2]); - let rpc_filter_key_extractor_manager = - match global_storage.filter_key_extractor_manager().clone() { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ) => rpc_filter_key_extractor_manager, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager(_) => unreachable!(), - }; - - rpc_filter_key_extractor_manager.update( - table_id_1.table_id(), - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + update_filter_key_extractor_for_table_ids( + global_storage.compaction_catalog_manager_ref(), + &[table_id_1.table_id(), table_id_2.table_id()], ); - rpc_filter_key_extractor_manager.update( - table_id_2.table_id(), - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), - ); - let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ); + let compaction_catalog_agent_ref = global_storage + .compaction_catalog_manager_ref() + .acquire(vec![table_id_1.table_id(), table_id_2.table_id()]) + .await + .unwrap(); + let compact_ctx = get_compactor_context_impl( global_storage.storage_opts().clone(), global_storage.sstable_store(), @@ -644,7 +610,7 @@ pub(crate) mod tests { compact_task.clone(), rx, Box::new(sstable_object_id_manager.clone()), - filter_key_extractor_manager, + compaction_catalog_agent_ref.clone(), ) .await; hummock_manager_ref @@ -738,14 +704,6 @@ pub(crate) mod tests { ) .await; - let rpc_filter_key_extractor_manager = match storage.filter_key_extractor_manager().clone() - { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ) => rpc_filter_key_extractor_manager, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager(_) => unreachable!(), - }; - let compact_ctx = get_compactor_context(&storage); let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( hummock_meta_client.clone(), @@ -754,13 +712,12 @@ pub(crate) mod tests { .clone() .sstable_id_remote_fetch_number, )); - rpc_filter_key_extractor_manager.update( - existing_table_id, - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), - ); - let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ); + + let compaction_catalog_agent_ref = storage + .compaction_catalog_manager_ref() + .acquire(vec![existing_table_id]) + .await + .unwrap(); // 1. add sstables let val = Bytes::from(b"0"[..].to_vec()); // 1 Byte value @@ -849,7 +806,7 @@ pub(crate) mod tests { compact_task.clone(), rx, Box::new(sstable_object_id_manager.clone()), - filter_key_extractor_manager, + compaction_catalog_agent_ref.clone(), ) .await; @@ -950,23 +907,25 @@ pub(crate) mod tests { ) .await; - let rpc_filter_key_extractor_manager = match storage.filter_key_extractor_manager().clone() - { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ) => rpc_filter_key_extractor_manager, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager(_) => unreachable!(), - }; - - rpc_filter_key_extractor_manager.update( + let mut multi_filter_key_extractor = MultiFilterKeyExtractor::default(); + multi_filter_key_extractor.register( existing_table_id, Arc::new(FilterKeyExtractorImpl::FixedLength( FixedLengthFilterKeyExtractor::new(TABLE_PREFIX_LEN + key_prefix.len()), )), ); - let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ); + + let table_id_to_vnode = + HashMap::from_iter([(existing_table_id, VirtualNode::COUNT_FOR_TEST)]); + + let filter_key_extractor = + Arc::new(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)); + + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + filter_key_extractor, + table_id_to_vnode, + )); + let compact_ctx = get_compactor_context(&storage); let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( hummock_meta_client.clone(), @@ -1048,7 +1007,7 @@ pub(crate) mod tests { compact_task.clone(), rx, Box::new(sstable_object_id_manager.clone()), - filter_key_extractor_manager, + compaction_catalog_agent_ref.clone(), ) .await; @@ -1156,8 +1115,7 @@ pub(crate) mod tests { &[existing_table_id], ) .await; - let (compact_ctx, filter_key_extractor_manager) = - prepare_compactor_and_filter(&storage, existing_table_id); + let compact_ctx = get_compactor_context(&storage); let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( hummock_meta_client.clone(), storage @@ -1221,6 +1179,12 @@ pub(crate) mod tests { 129 ); + let compaction_catalog_agent_ref = storage + .compaction_catalog_manager_ref() + .acquire(vec![existing_table_id]) + .await + .unwrap(); + // 3. compact let (_tx, rx) = tokio::sync::oneshot::channel(); let ((result_task, task_stats, object_timestamps), _) = compact( @@ -1228,7 +1192,7 @@ pub(crate) mod tests { compact_task.clone(), rx, Box::new(sstable_object_id_manager.clone()), - filter_key_extractor_manager, + compaction_catalog_agent_ref.clone(), ) .await; @@ -1325,12 +1289,25 @@ pub(crate) mod tests { } } + fn build_full_filter_key_extractor_with_table_ids( + table_ids: Vec, + ) -> Arc { + let mut multi_filter_key_extractor = MultiFilterKeyExtractor::default(); + for table_id in table_ids { + multi_filter_key_extractor.register( + table_id, + Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + ); + } + + Arc::new(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)) + } + async fn run_fast_and_normal_runner( compact_ctx: CompactorContext, task: CompactTask, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, ) -> (Vec, Vec) { - let multi_filter_key_extractor = - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); let compaction_filter = DummyCompactionFilter {}; let slow_compact_runner = CompactorRunner::new( 0, @@ -1340,10 +1317,11 @@ pub(crate) mod tests { VecDeque::from_iter([5, 6, 7, 8, 9, 10, 11, 12, 13]), )), ); + let fast_compact_runner = FastCompactorRunner::new( compact_ctx.clone(), task.clone(), - multi_filter_key_extractor.clone(), + compaction_catalog_agent_ref.clone(), Box::new(SharedComapctorObjectIdManager::for_test( VecDeque::from_iter([22, 23, 24, 25, 26, 27, 28, 29]), )), @@ -1352,7 +1330,7 @@ pub(crate) mod tests { let (_, ret1, _) = slow_compact_runner .run( compaction_filter, - multi_filter_key_extractor, + compaction_catalog_agent_ref, Arc::new(TaskProgress::default()), ) .await @@ -1384,7 +1362,17 @@ pub(crate) mod tests { ) .await; hummock_manager_ref.get_new_sst_ids(10).await.unwrap(); - let (compact_ctx, _) = prepare_compactor_and_filter(&storage, existing_table_id); + let compact_ctx = get_compactor_context(&storage); + + let multi_filter_key_extractor = + build_full_filter_key_extractor_with_table_ids(vec![existing_table_id]); + let table_id_to_vnode = + HashMap::from_iter(vec![(existing_table_id, VirtualNode::COUNT_FOR_TEST)]); + + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + multi_filter_key_extractor, + table_id_to_vnode, + )); let sstable_store = compact_ctx.sstable_store.clone(); let capacity = 256 * 1024; @@ -1433,7 +1421,9 @@ pub(crate) mod tests { gc_delete_keys: true, ..Default::default() }; - let (ret, fast_ret) = run_fast_and_normal_runner(compact_ctx.clone(), task).await; + let (ret, fast_ret) = + run_fast_and_normal_runner(compact_ctx.clone(), task, compaction_catalog_agent_ref) + .await; check_compaction_result(compact_ctx.sstable_store, ret, fast_ret, capacity).await; } @@ -1563,7 +1553,17 @@ pub(crate) mod tests { ) .await; hummock_manager_ref.get_new_sst_ids(10).await.unwrap(); - let (compact_ctx, _) = prepare_compactor_and_filter(&storage, existing_table_id); + // let (compact_ctx, _) = prepare_compactor_and_filter(&storage, existing_table_id); + let compact_ctx = get_compactor_context(&storage); + let multi_filter_key_extractor = + build_full_filter_key_extractor_with_table_ids(vec![existing_table_id]); + let table_id_to_vnode = + HashMap::from_iter(vec![(existing_table_id, VirtualNode::COUNT_FOR_TEST)]); + + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + multi_filter_key_extractor, + table_id_to_vnode, + )); let sstable_store = compact_ctx.sstable_store.clone(); let capacity = 256 * 1024; @@ -1585,7 +1585,6 @@ pub(crate) mod tests { ); let mut sst_infos = vec![]; let mut max_sst_file_size = 0; - for object_id in 1..3 { let mut builder = SstableBuilder::<_, BlockedXor16FilterBuilder>::new( object_id, @@ -1594,7 +1593,7 @@ pub(crate) mod tests { .create_sst_writer(object_id, SstableWriterOptions::default()), BlockedXor16FilterBuilder::create(opts.bloom_false_positive, opts.capacity / 16), opts.clone(), - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + compaction_catalog_agent_ref.clone(), None, ); let mut last_k: u64 = 1; @@ -1659,7 +1658,12 @@ pub(crate) mod tests { gc_delete_keys: true, ..Default::default() }; - let (ret, fast_ret) = run_fast_and_normal_runner(compact_ctx.clone(), task).await; + let (ret, fast_ret) = run_fast_and_normal_runner( + compact_ctx.clone(), + task, + compaction_catalog_agent_ref.clone(), + ) + .await; check_compaction_result(compact_ctx.sstable_store, ret, fast_ret, target_file_size).await; } @@ -1685,7 +1689,16 @@ pub(crate) mod tests { ) .await; hummock_manager_ref.get_new_sst_ids(10).await.unwrap(); - let (compact_ctx, _) = prepare_compactor_and_filter(&storage, existing_table_id); + let compact_ctx = get_compactor_context(&storage); + let multi_filter_key_extractor = + build_full_filter_key_extractor_with_table_ids(vec![existing_table_id]); + let table_id_to_vnode = + HashMap::from_iter(vec![(existing_table_id, VirtualNode::COUNT_FOR_TEST)]); + + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + multi_filter_key_extractor, + table_id_to_vnode, + )); let sstable_store = compact_ctx.sstable_store.clone(); let capacity = 256 * 1024; @@ -1707,7 +1720,6 @@ pub(crate) mod tests { ); let mut sst_infos = vec![]; let mut max_sst_file_size = 0; - for object_id in 1..3 { let mut builder = SstableBuilder::<_, BlockedXor16FilterBuilder>::new( object_id, @@ -1716,7 +1728,7 @@ pub(crate) mod tests { .create_sst_writer(object_id, SstableWriterOptions::default()), BlockedXor16FilterBuilder::create(opts.bloom_false_positive, opts.capacity / 16), opts.clone(), - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + compaction_catalog_agent_ref.clone(), None, ); let key_count = KEY_COUNT / VirtualNode::COUNT_FOR_TEST * 2; @@ -1799,7 +1811,7 @@ pub(crate) mod tests { table_infos: sst_infos, }, ], - existing_table_ids: vec![1], + existing_table_ids: vec![existing_table_id], task_id: 1, splits: vec![KeyRange::inf()], target_level: 6, @@ -1810,7 +1822,12 @@ pub(crate) mod tests { table_watermarks, ..Default::default() }; - let (ret, fast_ret) = run_fast_and_normal_runner(compact_ctx.clone(), task).await; + let (ret, fast_ret) = run_fast_and_normal_runner( + compact_ctx.clone(), + task, + compaction_catalog_agent_ref.clone(), + ) + .await; let mut fast_tables = Vec::with_capacity(fast_ret.len()); let mut normal_tables = Vec::with_capacity(ret.len()); let mut stats = StoreLocalStatistic::default(); @@ -1889,35 +1906,25 @@ pub(crate) mod tests { .await; // basic cg2 -> [1, 2] - let rpc_filter_key_extractor_manager = match storage.filter_key_extractor_manager().clone() - { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ) => rpc_filter_key_extractor_manager, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager(_) => unreachable!(), - }; - let mut key = BytesMut::default(); key.put_u16(1); key.put_slice(b"key_prefix"); let key_prefix = key.freeze(); - rpc_filter_key_extractor_manager.update( + let multi_filter_key_extractor = build_full_filter_key_extractor_with_table_ids(vec![ table_id_1.table_id(), - Arc::new(FilterKeyExtractorImpl::FixedLength( - FixedLengthFilterKeyExtractor::new(TABLE_PREFIX_LEN + key_prefix.len()), - )), - ); - rpc_filter_key_extractor_manager.update( table_id_2.table_id(), - Arc::new(FilterKeyExtractorImpl::FixedLength( - FixedLengthFilterKeyExtractor::new(TABLE_PREFIX_LEN + key_prefix.len()), - )), - ); + ]); + let table_id_to_vnode = HashMap::from_iter(vec![ + (table_id_1.table_id(), VirtualNode::COUNT_FOR_TEST), + (table_id_2.table_id(), VirtualNode::COUNT_FOR_TEST), + ]); + + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + multi_filter_key_extractor, + table_id_to_vnode, + )); - let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ); let compact_ctx = get_compactor_context(&storage); let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( hummock_meta_client.clone(), @@ -2027,7 +2034,7 @@ pub(crate) mod tests { level: usize, hummock_manager_ref: HummockManagerRef, compact_ctx: CompactorContext, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, sstable_object_id_manager: Arc, ) { // compact left group @@ -2059,7 +2066,7 @@ pub(crate) mod tests { compact_task.clone(), rx, Box::new(sstable_object_id_manager.clone()), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), ) .await; @@ -2101,7 +2108,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -2111,7 +2118,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -2133,7 +2140,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -2143,7 +2150,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -2177,7 +2184,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -2234,7 +2241,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -2244,7 +2251,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -2254,7 +2261,7 @@ pub(crate) mod tests { level: usize, hummock_manager_ref: HummockManagerRef, compact_ctx: CompactorContext, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, sstable_object_id_manager: Arc, ) { loop { @@ -2284,7 +2291,7 @@ pub(crate) mod tests { compact_task.clone(), rx, Box::new(sstable_object_id_manager.clone()), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), ) .await; @@ -2328,7 +2335,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -2338,7 +2345,7 @@ pub(crate) mod tests { 0, hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 1d21b74e15738..3d1981ab0003f 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -31,7 +31,7 @@ use risingwave_meta::hummock::compaction::selector::ManualCompactionOption; use risingwave_meta::hummock::test_utils::{setup_compute_env, setup_compute_env_with_config}; use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_rpc_client::HummockMetaClient; -use risingwave_storage::filter_key_extractor::FilterKeyExtractorManager; +use risingwave_storage::compaction_catalog_manager::CompactionCatalogAgentRef; use risingwave_storage::hummock::compactor::compactor_runner::compact; use risingwave_storage::hummock::compactor::CompactorContext; use risingwave_storage::hummock::{CachePolicy, GetObjectId, SstableObjectIdManager}; @@ -39,8 +39,8 @@ use risingwave_storage::store::{LocalStateStore, NewLocalOptions, ReadOptions, S use risingwave_storage::StateStore; use serial_test::serial; -use super::compactor_tests::tests::{get_hummock_storage, prepare_compactor_and_filter}; -use crate::compactor_tests::tests::flush_and_commit; +use super::compactor_tests::tests::get_hummock_storage; +use crate::compactor_tests::tests::{flush_and_commit, get_compactor_context}; use crate::get_notification_client_for_test; use crate::local_state_store_test_utils::LocalStateStoreTestExt; use crate::test_utils::gen_key_from_bytes; @@ -178,7 +178,7 @@ async fn test_syncpoints_test_failpoints_fetch_ids() { pub async fn compact_once( hummock_manager_ref: HummockManagerRef, compact_ctx: CompactorContext, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, sstable_object_id_manager: Arc, ) { // 2. get compact task @@ -206,7 +206,7 @@ pub async fn compact_once( compact_task.clone(), rx, Box::new(sstable_object_id_manager), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), ) .await; @@ -252,8 +252,13 @@ async fn test_syncpoints_get_in_delete_range_boundary() { &[existing_table_id], ) .await; - let (compact_ctx, filter_key_extractor_manager) = - prepare_compactor_and_filter(&storage, existing_table_id); + + let compact_ctx = get_compactor_context(&storage); + let compaction_catalog_agent_ref = storage + .compaction_catalog_manager_ref() + .acquire(vec![existing_table_id]) + .await + .unwrap(); let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( hummock_meta_client.clone(), @@ -316,10 +321,11 @@ async fn test_syncpoints_get_in_delete_range_boundary() { local.table_id(), ) .await; + compact_once( hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -360,7 +366,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { compact_once( hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -402,7 +408,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { compact_once( hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; @@ -437,7 +443,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { compact_once( hummock_manager_ref.clone(), compact_ctx.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref.clone(), sstable_object_id_manager.clone(), ) .await; diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index 849e77a659f7a..874f416c81df1 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use bytes::Bytes; use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_common::hash::VirtualNode; use risingwave_common_service::ObserverManager; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::TableKey; @@ -29,11 +30,10 @@ use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_meta::manager::MetaSrvEnv; use risingwave_pb::catalog::{PbTable, Table}; use risingwave_rpc_client::HummockMetaClient; -use risingwave_storage::error::StorageResult; -use risingwave_storage::filter_key_extractor::{ - FilterKeyExtractorImpl, FilterKeyExtractorManager, FullKeyFilterKeyExtractor, - RpcFilterKeyExtractorManager, +use risingwave_storage::compaction_catalog_manager::{ + CompactionCatalogManager, CompactionCatalogManagerRef, }; +use risingwave_storage::error::StorageResult; use risingwave_storage::hummock::backup_reader::BackupReader; use risingwave_storage::hummock::event_handler::HummockVersionUpdate; use risingwave_storage::hummock::iterator::test_utils::mock_sstable_store; @@ -71,7 +71,7 @@ pub async fn prepare_first_valid_version( let observer_manager = ObserverManager::new( notification_client, HummockObserverNode::new( - Arc::new(RpcFilterKeyExtractorManager::default()), + Arc::new(CompactionCatalogManager::default()), backup_manager, tx.clone(), write_limiter, @@ -145,7 +145,7 @@ pub async fn with_hummock_storage_v2( .unwrap(); register_tables_with_id_for_test( - hummock_storage.filter_key_extractor_manager(), + hummock_storage.compaction_catalog_manager_ref(), &hummock_manager_ref, &[table_id.table_id()], ) @@ -153,31 +153,28 @@ pub async fn with_hummock_storage_v2( (hummock_storage, meta_client) } + pub fn update_filter_key_extractor_for_table_ids( - filter_key_extractor_manager_ref: &FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, table_ids: &[u32], ) { - let rpc_filter_key_extractor_manager = match filter_key_extractor_manager_ref { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ) => rpc_filter_key_extractor_manager, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager(_) => unreachable!(), - }; - for table_id in table_ids { - rpc_filter_key_extractor_manager.update( - *table_id, - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), - ) + let mock_table = Arc::new(PbTable { + id: *table_id, + read_prefix_len_hint: 0, + maybe_vnode_count: Some(VirtualNode::COUNT_FOR_TEST as u32), + ..Default::default() + }); + compaction_catalog_manager_ref.update(*table_id, mock_table); } } pub async fn register_tables_with_id_for_test( - filter_key_extractor_manager: &FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, hummock_manager_ref: &HummockManagerRef, table_ids: &[u32], ) { - update_filter_key_extractor_for_table_ids(filter_key_extractor_manager, table_ids); + update_filter_key_extractor_for_table_ids(compaction_catalog_manager_ref, table_ids); register_table_ids_to_compaction_group( hummock_manager_ref, table_ids, @@ -187,28 +184,19 @@ pub async fn register_tables_with_id_for_test( } pub fn update_filter_key_extractor_for_tables( - filter_key_extractor_manager: &FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, tables: &[PbTable], ) { - let rpc_filter_key_extractor_manager = match filter_key_extractor_manager { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_extractor_manager, - ) => rpc_filter_key_extractor_manager, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager(_) => unreachable!(), - }; for table in tables { - rpc_filter_key_extractor_manager.update( - table.id, - Arc::new(FilterKeyExtractorImpl::from_table(table)), - ) + compaction_catalog_manager_ref.update(table.id, Arc::new(table.clone())) } } pub async fn register_tables_with_catalog_for_test( - filter_key_extractor_manager: &FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, hummock_manager_ref: &HummockManagerRef, tables: &[Table], ) { - update_filter_key_extractor_for_tables(filter_key_extractor_manager, tables); + update_filter_key_extractor_for_tables(compaction_catalog_manager_ref, tables); let table_ids = tables.iter().map(|t| t.id).collect_vec(); register_table_ids_to_compaction_group( hummock_manager_ref, @@ -233,7 +221,7 @@ impl HummockTestEnv { pub async fn register_table_id(&self, table_id: TableId) { register_tables_with_id_for_test( - self.storage.filter_key_extractor_manager(), + self.storage.compaction_catalog_manager_ref(), &self.manager, &[table_id.table_id()], ) @@ -243,7 +231,7 @@ impl HummockTestEnv { pub async fn register_table(&self, table: PbTable) { register_tables_with_catalog_for_test( - self.storage.filter_key_extractor_manager(), + self.storage.compaction_catalog_manager_ref(), &self.manager, &[table], ) diff --git a/src/storage/src/filter_key_extractor.rs b/src/storage/src/compaction_catalog_manager.rs similarity index 75% rename from src/storage/src/filter_key_extractor.rs rename to src/storage/src/compaction_catalog_manager.rs index e9326d37dcd8c..51591ed8a7107 100644 --- a/src/storage/src/filter_key_extractor.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -16,13 +16,12 @@ use std::collections::{HashMap, HashSet}; use std::fmt::Debug; use std::sync::Arc; -use itertools::Itertools; use parking_lot::RwLock; use risingwave_common::catalog::ColumnDesc; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::{VirtualNode, VnodeCountCompat}; use risingwave_common::util::row_serde::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; -use risingwave_hummock_sdk::info_in_release; +use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::key::{get_table_id, TABLE_PREFIX_LEN}; use risingwave_pb::catalog::Table; use risingwave_rpc_client::error::{Result as RpcResult, RpcError}; @@ -51,7 +50,7 @@ impl FilterKeyExtractorImpl { if read_prefix_len == 0 || read_prefix_len > table_catalog.get_pk().len() { // for now frontend had not infer the table_id_to_filter_key_extractor, so we // use FullKeyFilterKeyExtractor - FilterKeyExtractorImpl::Dummy(DummyFilterKeyExtractor) + FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor) } else { FilterKeyExtractorImpl::Schema(SchemaFilterKeyExtractor::new(table_catalog)) } @@ -187,8 +186,6 @@ impl SchemaFilterKeyExtractor { #[derive(Default)] pub struct MultiFilterKeyExtractor { id_to_filter_key_extractor: HashMap>, - // cached state - // last_filter_key_extractor_state: Mutex)>>, } impl MultiFilterKeyExtractor { @@ -259,48 +256,67 @@ impl StateTableAccessor for FakeRemoteTableAccessor { ))) } } -struct FilterKeyExtractorManagerInner { - table_id_to_filter_key_extractor: RwLock>>, + +pub struct CompactionCatalogManager { + table_id_to_catalog: RwLock>>, table_accessor: Box, } -impl FilterKeyExtractorManagerInner { - fn update(&self, table_id: u32, filter_key_extractor: Arc) { - self.table_id_to_filter_key_extractor - .write() - .insert(table_id, filter_key_extractor); +impl Default for CompactionCatalogManager { + fn default() -> Self { + Self::new(Box::::default()) + } +} + +impl CompactionCatalogManager { + pub fn new(table_accessor: Box) -> Self { + Self { + table_id_to_catalog: Default::default(), + table_accessor, + } } +} - fn sync(&self, filter_key_extractor_map: HashMap>) { - let mut guard = self.table_id_to_filter_key_extractor.write(); +impl CompactionCatalogManager { + pub fn update(&self, table_id: u32, catalog: Arc
) { + self.table_id_to_catalog.write().insert(table_id, catalog); + } + + pub fn sync(&self, catalog_map: HashMap>) { + let mut guard = self.table_id_to_catalog.write(); guard.clear(); - guard.extend(filter_key_extractor_map); + guard.extend(catalog_map); } - fn remove(&self, table_id: u32) { - self.table_id_to_filter_key_extractor - .write() - .remove(&table_id); + pub fn remove(&self, table_id: u32) { + self.table_id_to_catalog.write().remove(&table_id); } - async fn acquire( + pub async fn acquire( &self, - mut table_id_set: HashSet, - ) -> HummockResult { - if table_id_set.is_empty() { + mut table_ids: Vec, + ) -> HummockResult { + if table_ids.is_empty() { // table_id_set is empty // the table in sst has been deleted // use full key as default - return Ok(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); + // return Ok(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); + return Err(HummockError::other("table_id_set is empty")); } let mut multi_filter_key_extractor = MultiFilterKeyExtractor::default(); + let mut table_id_to_vnode = HashMap::new(); { - let guard = self.table_id_to_filter_key_extractor.read(); - table_id_set.retain(|table_id| match guard.get(table_id) { - Some(filter_key_extractor) => { - multi_filter_key_extractor.register(*table_id, filter_key_extractor.clone()); + let guard = self.table_id_to_catalog.read(); + table_ids.retain(|table_id| match guard.get(table_id) { + Some(table_catalog) => { + multi_filter_key_extractor.register( + *table_id, + Arc::new(FilterKeyExtractorImpl::from_table(table_catalog.as_ref())), + ); + table_id_to_vnode.insert(*table_id, table_catalog.vnode_count()); + false } @@ -308,8 +324,7 @@ impl FilterKeyExtractorManagerInner { }); } - if !table_id_set.is_empty() { - let table_ids = table_id_set.iter().cloned().collect_vec(); + if !table_ids.is_empty() { let mut state_tables = self.table_accessor .get_tables(&table_ids) @@ -320,120 +335,82 @@ impl FilterKeyExtractorManagerInner { e.as_report() )) })?; - let mut guard = self.table_id_to_filter_key_extractor.write(); + let mut guard = self.table_id_to_catalog.write(); for table_id in table_ids { if let Some(table) = state_tables.remove(&table_id) { + let table_id = table.id; let key_extractor = Arc::new(FilterKeyExtractorImpl::from_table(&table)); - guard.insert(table_id, key_extractor.clone()); + let vnode = table.vnode_count(); + guard.insert(table_id, Arc::new(table)); multi_filter_key_extractor.register(table_id, key_extractor); + table_id_to_vnode.insert(table_id, vnode); } } } - Ok(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)) - } -} - -/// `RpcFilterKeyExtractorManager` is a wrapper for inner, and provide a protected read and write -/// interface, its thread safe -pub struct RpcFilterKeyExtractorManager { - inner: FilterKeyExtractorManagerInner, -} - -impl Default for RpcFilterKeyExtractorManager { - fn default() -> Self { - Self::new(Box::::default()) + Ok(Arc::new(CompactionCatalogAgent::new( + Arc::new(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)), + table_id_to_vnode, + ))) } -} -impl RpcFilterKeyExtractorManager { - pub fn new(table_accessor: Box) -> Self { - Self { - inner: FilterKeyExtractorManagerInner { - table_id_to_filter_key_extractor: Default::default(), - table_accessor, - }, + pub fn build_compaction_catalog_agent( + table_catalogs: HashMap>, + ) -> CompactionCatalogAgentRef { + let mut multi_filter_key_extractor = MultiFilterKeyExtractor::default(); + let mut table_id_to_vnode = HashMap::new(); + for (table_id, table_catalog) in table_catalogs { + multi_filter_key_extractor.register( + table_id, + Arc::new(FilterKeyExtractorImpl::from_table(table_catalog.as_ref())), + ); + table_id_to_vnode.insert(table_id, table_catalog.vnode_count()); } - } - - /// Insert (`table_id`, `filter_key_extractor`) as mapping to `HashMap` for `acquire` - pub fn update(&self, table_id: u32, filter_key_extractor: Arc) { - info_in_release!("update key extractor of {}", table_id); - self.inner.update(table_id, filter_key_extractor); - } - - /// Remove a mapping by `table_id` - pub fn remove(&self, table_id: u32) { - info_in_release!("remove key extractor of {}", table_id); - self.inner.remove(table_id); - } - - /// Sync all filter key extractors by snapshot - pub fn sync(&self, filter_key_extractor_map: HashMap>) { - self.inner.sync(filter_key_extractor_map) - } - /// Acquire a `MultiFilterKeyExtractor` by `table_id_set` - /// Internally, try to get all `filter_key_extractor` from `hashmap`. Will block the caller if - /// `table_id` does not util version update (notify), and retry to get - async fn acquire(&self, table_id_set: HashSet) -> HummockResult { - self.inner.acquire(table_id_set).await + Arc::new(CompactionCatalogAgent::new( + Arc::new(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)), + table_id_to_vnode, + )) } } -#[derive(Clone)] -pub enum FilterKeyExtractorManager { - RpcFilterKeyExtractorManager(Arc), - StaticFilterKeyExtractorManager(Arc), +/// build from `CompactionCatalogManager` +pub struct CompactionCatalogAgent { + filter_key_extractor_manager: Arc, + table_id_to_vnode: HashMap, } -impl FilterKeyExtractorManager { - pub async fn acquire( - &self, - table_id_set: HashSet, - ) -> HummockResult { - match self { - FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - rpc_filter_key_exactor_manager, - ) => rpc_filter_key_exactor_manager.acquire(table_id_set).await, - FilterKeyExtractorManager::StaticFilterKeyExtractorManager( - static_filter_key_extractor_manager, - ) => static_filter_key_extractor_manager.acquire(table_id_set), +impl CompactionCatalogAgent { + pub fn new( + filter_key_extractor_manager: Arc, + table_id_to_vnode: HashMap, + ) -> Self { + Self { + filter_key_extractor_manager, + table_id_to_vnode, } } } -#[derive(Clone)] -pub struct StaticFilterKeyExtractorManager { - id_to_table: HashMap, -} +impl CompactionCatalogAgent { + pub fn extract<'a>(&self, full_key: &'a [u8]) -> &'a [u8] { + self.filter_key_extractor_manager.extract(full_key) + } -impl StaticFilterKeyExtractorManager { - pub fn new(id_to_table: HashMap) -> Self { - Self { id_to_table } + pub fn vnode_count(&self, table_id: StateTableId) -> usize { + *self.table_id_to_vnode.get(&table_id).unwrap() } - fn acquire(&self, table_id_set: HashSet) -> HummockResult { - let mut multi_filter_key_extractor = MultiFilterKeyExtractor::default(); - for table_id in table_id_set { - if let Some(table) = self.id_to_table.get(&table_id) { - let key_extractor = Arc::new(FilterKeyExtractorImpl::from_table(table)); - multi_filter_key_extractor.register(table_id, key_extractor); - } else { - return Err(HummockError::other(format!( - "table {} is absent in id_to_table, need to request rpc list_tables to get the schema", table_id, - ))); - } - } - Ok(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)) + pub fn table_id_to_vnode_ref(&self) -> &HashMap { + &self.table_id_to_vnode } } -pub type FilterKeyExtractorManagerRef = Arc; +pub type CompactionCatalogManagerRef = Arc; +pub type CompactionCatalogAgentRef = Arc; #[cfg(test)] mod tests { - use std::collections::HashSet; use std::mem; use std::sync::Arc; @@ -453,9 +430,8 @@ mod tests { use risingwave_pb::plan_common::PbColumnCatalog; use super::{DummyFilterKeyExtractor, FilterKeyExtractor, SchemaFilterKeyExtractor}; - use crate::filter_key_extractor::{ + use crate::compaction_catalog_manager::{ FilterKeyExtractorImpl, FullKeyFilterKeyExtractor, MultiFilterKeyExtractor, - RpcFilterKeyExtractorManager, }; const fn dummy_vnode() -> [u8; VirtualNode::SIZE] { VirtualNode::from_index(233).to_be_bytes() @@ -664,30 +640,4 @@ mod tests { assert_eq!(pk_prefix_len, output_key.len()); } } - - #[tokio::test] - async fn test_filter_key_extractor_manager() { - let filter_key_extractor_manager = Arc::new(RpcFilterKeyExtractorManager::default()); - - filter_key_extractor_manager.update( - 1, - Arc::new(FilterKeyExtractorImpl::Dummy(DummyFilterKeyExtractor)), - ); - - let remaining_table_id_set = HashSet::from([1]); - let multi_filter_key_extractor = filter_key_extractor_manager - .acquire(remaining_table_id_set) - .await - .unwrap(); - - match multi_filter_key_extractor { - FilterKeyExtractorImpl::Multi(multi_filter_key_extractor) => { - assert_eq!(1, multi_filter_key_extractor.size()); - } - - _ => { - unreachable!() - } - } - } } diff --git a/src/storage/src/hummock/compactor/compaction_utils.rs b/src/storage/src/hummock/compactor/compaction_utils.rs index 3b032123f426a..22d864df8d2b3 100644 --- a/src/storage/src/hummock/compactor/compaction_utils.rs +++ b/src/storage/src/hummock/compactor/compaction_utils.rs @@ -33,7 +33,7 @@ use risingwave_pb::hummock::{BloomFilterType, PbLevelType, PbTableSchema}; use tokio::time::Instant; pub use super::context::CompactorContext; -use crate::filter_key_extractor::FilterKeyExtractorImpl; +use crate::compaction_catalog_manager::CompactionCatalogAgentRef; use crate::hummock::compactor::{ ConcatSstableIterator, MultiCompactionFilter, StateCleanUpCompactionFilter, TaskProgress, TtlCompactionFilter, @@ -55,7 +55,7 @@ pub struct RemoteBuilderFactory { pub options: SstableBuilderOptions, pub policy: CachePolicy, pub remote_rpc_cost: Arc, - pub filter_key_extractor: Arc, + pub compaction_catalog_agent_ref: CompactionCatalogAgentRef, pub sstable_writer_factory: W, pub _phantom: PhantomData, } @@ -87,7 +87,7 @@ impl TableBuilderFactory for RemoteBu self.options.capacity / DEFAULT_ENTRY_SIZE + 1, ), self.options.clone(), - self.filter_key_extractor.clone(), + self.compaction_catalog_agent_ref.clone(), Some(self.limiter.clone()), ); Ok(builder) diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 57b2b929db411..9e00568154532 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -40,7 +40,7 @@ use tokio::sync::oneshot::Receiver; use super::iterator::MonitoredCompactorIterator; use super::task_progress::TaskProgress; use super::{CompactionStatistics, TaskConfig}; -use crate::filter_key_extractor::{FilterKeyExtractorImpl, FilterKeyExtractorManager}; +use crate::compaction_catalog_manager::CompactionCatalogAgentRef; use crate::hummock::compactor::compaction_utils::{ build_multi_compaction_filter, estimate_task_output_capacity, generate_splits_for_task, metrics_report_for_task, optimize_by_copy_block, @@ -134,7 +134,7 @@ impl CompactorRunner { pub async fn run( &self, compaction_filter: impl CompactionFilter, - filter_key_extractor: Arc, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, task_progress: Arc, ) -> HummockResult { let iter = self.build_sst_iter(task_progress.clone())?; @@ -143,7 +143,7 @@ impl CompactorRunner { .compact_key_range( iter, compaction_filter, - filter_key_extractor, + compaction_catalog_agent_ref, Some(task_progress), Some(self.compact_task.task_id), Some(self.split_index), @@ -307,7 +307,7 @@ pub async fn compact( mut compact_task: CompactTask, mut shutdown_rx: Receiver<()>, object_id_getter: Box, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, ) -> ( ( CompactTask, @@ -330,35 +330,6 @@ pub async fn compact( .start_timer(); let multi_filter = build_multi_compaction_filter(&compact_task); - - let existing_table_ids: HashSet = - HashSet::from_iter(compact_task.existing_table_ids.clone()); - let compact_table_ids = HashSet::from_iter( - compact_task - .input_ssts - .iter() - .flat_map(|level| level.table_infos.iter()) - .flat_map(|sst| sst.table_ids.clone()) - .filter(|table_id| existing_table_ids.contains(table_id)), - ); - - let multi_filter_key_extractor = match build_filter_key_extractor( - &compact_task, - filter_key_extractor_manager, - &compact_table_ids, - ) - .await - { - Some(multi_filter_key_extractor) => multi_filter_key_extractor, - None => { - let task_status = TaskStatus::ExecuteFailed; - return ( - compact_done(compact_task, context.clone(), vec![], task_status), - None, - ); - } - }; - let mut task_status = TaskStatus::Success; let optimize_by_copy_block = optimize_by_copy_block(&compact_task, &context); @@ -446,7 +417,7 @@ pub async fn compact( let runner = fast_compactor_runner::CompactorRunner::new( context.clone(), compact_task.clone(), - multi_filter_key_extractor.clone(), + compaction_catalog_agent_ref.clone(), object_id_getter.clone(), task_progress_guard.progress.clone(), ); @@ -490,7 +461,7 @@ pub async fn compact( } for (split_index, _) in compact_task.splits.iter().enumerate() { let filter = multi_filter.clone(); - let multi_filter_key_extractor = multi_filter_key_extractor.clone(); + let compaction_catalog_agent_ref = compaction_catalog_agent_ref.clone(); let compactor_runner = CompactorRunner::new( split_index, compactor_context.clone(), @@ -500,7 +471,7 @@ pub async fn compact( let task_progress = task_progress_guard.progress.clone(); let runner = async move { compactor_runner - .run(filter, multi_filter_key_extractor, task_progress) + .run(filter, compaction_catalog_agent_ref, task_progress) .await }; let traced = match context.await_tree_reg.as_ref() { @@ -810,39 +781,6 @@ where Ok(compaction_statistics) } -async fn build_filter_key_extractor( - compact_task: &CompactTask, - filter_key_extractor_manager: FilterKeyExtractorManager, - compact_table_ids: &HashSet, -) -> Option> { - let multi_filter_key_extractor = match filter_key_extractor_manager - .acquire(compact_table_ids.clone()) - .await - { - Err(e) => { - tracing::error!(error = %e.as_report(), "Failed to fetch filter key extractor tables [{:?}], it may caused by some RPC error", compact_task.existing_table_ids); - return None; - } - Ok(extractor) => extractor, - }; - - if let FilterKeyExtractorImpl::Multi(multi) = &multi_filter_key_extractor { - let found_tables = multi.get_existing_table_ids(); - let removed_tables = compact_table_ids - .iter() - .filter(|table_id| !found_tables.contains(table_id)) - .collect_vec(); - if !removed_tables.is_empty() { - tracing::error!("Failed to fetch filter key extractor tables [{:?}. [{:?}] may be removed by meta-service. ", compact_table_ids, removed_tables); - return None; - } - } - - let multi_filter_key_extractor = Arc::new(multi_filter_key_extractor); - - Some(multi_filter_key_extractor) -} - #[cfg(test)] pub mod tests { use risingwave_hummock_sdk::can_concat; diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 641d866e544b5..619c5f858fa9f 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -30,7 +30,7 @@ use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_stats::TableStats; use risingwave_hummock_sdk::{can_concat, compact_task_to_string, EpochWithGap, LocalSstableInfo}; -use crate::filter_key_extractor::FilterKeyExtractorImpl; +use crate::compaction_catalog_manager::CompactionCatalogAgentRef; use crate::hummock::block_stream::BlockDataStream; use crate::hummock::compactor::task_progress::TaskProgress; use crate::hummock::compactor::{ @@ -359,7 +359,7 @@ impl CompactorRunner { pub fn new( context: CompactorContext, task: CompactTask, - filter_key_extractor: Arc, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, object_id_getter: Box, task_progress: Arc, ) -> Self { @@ -391,7 +391,7 @@ impl CompactorRunner { options, policy: task_config.cache_policy, remote_rpc_cost: get_id_time, - filter_key_extractor, + compaction_catalog_agent_ref, sstable_writer_factory: factory, _phantom: PhantomData, }; diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index 3a4c487b217a8..69d10cf1dc574 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -34,7 +34,7 @@ mod iterator; mod shared_buffer_compact; pub(super) mod task_progress; -use std::collections::{HashMap, VecDeque}; +use std::collections::{HashMap, HashSet, VecDeque}; use std::marker::PhantomData; use std::sync::atomic::{AtomicU32, AtomicU64, Ordering}; use std::sync::{Arc, Mutex}; @@ -80,8 +80,8 @@ use super::multi_builder::CapacitySplitTableBuilder; use super::{ GetObjectId, HummockResult, SstableBuilderOptions, SstableObjectIdManager, Xor16FilterBuilder, }; -use crate::filter_key_extractor::{ - FilterKeyExtractorImpl, FilterKeyExtractorManager, StaticFilterKeyExtractorManager, +use crate::compaction_catalog_manager::{ + CompactionCatalogAgentRef, CompactionCatalogManager, CompactionCatalogManagerRef, }; use crate::hummock::compactor::compaction_utils::calculate_task_parallelism; use crate::hummock::compactor::compactor_runner::{compact_and_build_sst, compact_done}; @@ -130,7 +130,7 @@ impl Compactor { &self, iter: impl HummockIterator, compaction_filter: impl CompactionFilter, - filter_key_extractor: Arc, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, task_progress: Option>, task_id: Option, split_index: Option, @@ -155,7 +155,7 @@ impl Compactor { factory, iter, compaction_filter, - filter_key_extractor, + compaction_catalog_agent_ref, task_progress.clone(), self.object_id_getter.clone(), ) @@ -166,7 +166,7 @@ impl Compactor { factory, iter, compaction_filter, - filter_key_extractor, + compaction_catalog_agent_ref, task_progress.clone(), self.object_id_getter.clone(), ) @@ -230,7 +230,7 @@ impl Compactor { writer_factory: F, iter: impl HummockIterator, compaction_filter: impl CompactionFilter, - filter_key_extractor: Arc, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, task_progress: Option>, object_id_getter: Box, ) -> HummockResult<(Vec, CompactionStatistics)> { @@ -240,7 +240,7 @@ impl Compactor { options: self.options.clone(), policy: self.task_config.cache_policy, remote_rpc_cost: self.get_id_time.clone(), - filter_key_extractor, + compaction_catalog_agent_ref, sstable_writer_factory: writer_factory, _phantom: PhantomData, }; @@ -280,7 +280,7 @@ pub fn start_compactor( compactor_context: CompactorContext, hummock_meta_client: Arc, sstable_object_id_manager: Arc, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, ) -> (JoinHandle<()>, Sender<()>) { type CompactionShutdownMap = Arc>>>; let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel(); @@ -470,7 +470,7 @@ pub fn start_compactor( let meta_client = hummock_meta_client.clone(); let sstable_object_id_manager = sstable_object_id_manager.clone(); - let filter_key_extractor_manager = filter_key_extractor_manager.clone(); + let compaction_catalog_manager_ref = compaction_catalog_manager_ref.clone(); match event { ResponseEvent::CompactTask(compact_task) => { @@ -515,14 +515,37 @@ pub fn start_compactor( let (tx, rx) = tokio::sync::oneshot::channel(); let task_id = compact_task.task_id; shutdown.lock().unwrap().insert(task_id, tx); - let ((compact_task, table_stats, object_timestamps), _memory_tracker) = - compactor_runner::compact( - context.clone(), - compact_task, - rx, - Box::new(sstable_object_id_manager.clone()), - filter_key_extractor_manager.clone(), - ).await; + let existing_table_ids: HashSet = HashSet::from_iter(compact_task.existing_table_ids.clone()); + let compact_table_ids = Vec::from_iter( + compact_task + .input_ssts + .iter() + .flat_map(|level| level.table_infos.iter()) + .flat_map(|sst| sst.table_ids.clone()) + .filter(|table_id| existing_table_ids.contains(table_id)), + ); + + + let ((compact_task, table_stats, object_timestamps), _memory_tracker) = match compaction_catalog_manager_ref.acquire(compact_table_ids).await { + Ok(compaction_catalog_agent_ref) => { + compactor_runner::compact( + context.clone(), + compact_task, + rx, + Box::new(sstable_object_id_manager.clone()), + compaction_catalog_agent_ref, + ).await + }, + Err(e) => { + tracing::error!(error = %e.as_report(), "Failed to acquire catalog"); + let task_status = TaskStatus::ExecuteFailed; + ( + compact_done(compact_task, context.clone(), vec![], task_status), + None, + ) + } + }; + shutdown.lock().unwrap().remove(&task_id); running_task_parallelism.fetch_sub(parallelism as u32, Ordering::SeqCst); @@ -703,16 +726,10 @@ pub fn start_shared_compactor( output_object_ids, task: dispatch_task, } = request.into_inner(); - let id_to_tables = tables.into_iter().fold(HashMap::new(), |mut acc, table| { - acc.insert(table.id, table); + let table_id_to_catalog = tables.into_iter().fold(HashMap::new(), |mut acc, table| { + acc.insert(table.id, Arc::new(table)); acc }); - let static_filter_key_extractor_manager: Arc = - Arc::new(StaticFilterKeyExtractorManager::new(id_to_tables)); - let filter_key_extractor_manager = - FilterKeyExtractorManager::StaticFilterKeyExtractorManager( - static_filter_key_extractor_manager, - ); let mut output_object_ids_deque: VecDeque<_> = VecDeque::new(); output_object_ids_deque.extend(output_object_ids); @@ -725,12 +742,13 @@ pub fn start_shared_compactor( let task_id = compact_task.task_id; shutdown.lock().unwrap().insert(task_id, tx); + let compaction_catalog_agent_ref = CompactionCatalogManager::build_compaction_catalog_agent(table_id_to_catalog); let ((compact_task, table_stats, object_timestamps), _memory_tracker)= compactor_runner::compact( context.clone(), compact_task, rx, Box::new(shared_compactor_object_id_manager), - filter_key_extractor_manager.clone(), + compaction_catalog_agent_ref, ) .await; shutdown.lock().unwrap().remove(&task_id); diff --git a/src/storage/src/hummock/compactor/shared_buffer_compact.rs b/src/storage/src/hummock/compactor/shared_buffer_compact.rs index 6ca42e41e3d92..022033ac26be5 100644 --- a/src/storage/src/hummock/compactor/shared_buffer_compact.rs +++ b/src/storage/src/hummock/compactor/shared_buffer_compact.rs @@ -32,7 +32,7 @@ use risingwave_pb::hummock::compact_task; use thiserror_ext::AsReport; use tracing::{error, warn}; -use crate::filter_key_extractor::{FilterKeyExtractorImpl, FilterKeyExtractorManager}; +use crate::compaction_catalog_manager::{CompactionCatalogAgentRef, CompactionCatalogManagerRef}; use crate::hummock::compactor::compaction_filter::DummyCompactionFilter; use crate::hummock::compactor::context::{await_tree_key, CompactorContext}; use crate::hummock::compactor::{check_flush_result, CompactOutput, Compactor}; @@ -57,14 +57,14 @@ pub async fn compact( context: CompactorContext, sstable_object_id_manager: SstableObjectIdManagerRef, payload: Vec, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, ) -> HummockResult { let new_value_payload = payload.clone(); let new_value_future = async { compact_shared_buffer::( context.clone(), sstable_object_id_manager.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_manager_ref.clone(), new_value_payload, ) .map_ok(move |results| results.into_iter()) @@ -84,7 +84,7 @@ pub async fn compact( compact_shared_buffer::( context.clone(), sstable_object_id_manager.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_manager_ref.clone(), old_value_payload, ) .await @@ -109,29 +109,23 @@ pub async fn compact( async fn compact_shared_buffer( context: CompactorContext, sstable_object_id_manager: SstableObjectIdManagerRef, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, mut payload: Vec, ) -> HummockResult> { if !IS_NEW_VALUE { assert!(payload.iter().all(|imm| imm.has_old_value())); } // Local memory compaction looks at all key ranges. - - let mut existing_table_ids: HashSet = payload + let existing_table_ids: HashSet = payload .iter() .map(|imm| imm.table_id.table_id) .dedup() .collect(); assert!(!existing_table_ids.is_empty()); - let multi_filter_key_extractor = filter_key_extractor_manager - .acquire(existing_table_ids.clone()) + let compaction_catalog_agent_ref = compaction_catalog_manager_ref + .acquire(existing_table_ids.iter().copied().collect()) .await?; - if let FilterKeyExtractorImpl::Multi(multi) = &multi_filter_key_extractor { - existing_table_ids = multi.get_existing_table_ids(); - } - let multi_filter_key_extractor = Arc::new(multi_filter_key_extractor); - payload.retain(|imm| { let ret = existing_table_ids.contains(&imm.table_id.table_id); if !ret { @@ -167,7 +161,7 @@ async fn compact_shared_buffer( forward_iters.push(imm.clone().into_directed_iter::()); } let compaction_executor = context.compaction_executor.clone(); - let multi_filter_key_extractor = multi_filter_key_extractor.clone(); + let compaction_catalog_agent_ref = compaction_catalog_agent_ref.clone(); let handle = compaction_executor.spawn({ static NEXT_SHARED_BUFFER_COMPACT_ID: LazyLock = LazyLock::new(|| AtomicUsize::new(0)); @@ -187,7 +181,7 @@ async fn compact_shared_buffer( }); let future = compactor.run( MergeIterator::new(forward_iters), - multi_filter_key_extractor, + compaction_catalog_agent_ref, ); if let Some(root) = tree_root { root.instrument(future).left_future() @@ -550,7 +544,7 @@ impl SharedBufferCompactRunner { pub async fn run( self, iter: impl HummockIterator, - filter_key_extractor: Arc, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, ) -> HummockResult { let dummy_compaction_filter = DummyCompactionFilter {}; let (ssts, table_stats_map) = self @@ -558,7 +552,7 @@ impl SharedBufferCompactRunner { .compact_key_range( iter, dummy_compaction_filter, - filter_key_extractor, + compaction_catalog_agent_ref, None, None, None, diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index 7a33ed81b4373..908bb45a43fc4 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -37,7 +37,7 @@ use tracing::{debug, error, info, trace, warn}; use super::refiller::{CacheRefillConfig, CacheRefiller}; use super::{LocalInstanceGuard, LocalInstanceId, ReadVersionMappingType}; -use crate::filter_key_extractor::FilterKeyExtractorManager; +use crate::compaction_catalog_manager::CompactionCatalogManagerRef; use crate::hummock::compactor::{await_tree_key, compact, CompactorContext}; use crate::hummock::event_handler::refiller::{CacheRefillerEvent, SpawnRefillTask}; use crate::hummock::event_handler::uploader::{ @@ -207,14 +207,14 @@ pub struct HummockEventHandler { async fn flush_imms( payload: Vec, compactor_context: CompactorContext, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, sstable_object_id_manager: Arc, ) -> HummockResult { compact( compactor_context, sstable_object_id_manager, payload, - filter_key_extractor_manager, + compaction_catalog_manager_ref, ) .verbose_instrument_await("shared_buffer_compact") .await @@ -225,7 +225,7 @@ impl HummockEventHandler { version_update_rx: UnboundedReceiver, pinned_version: PinnedVersion, compactor_context: CompactorContext, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, sstable_object_id_manager: Arc, state_store_metrics: Arc, ) -> Self { @@ -251,7 +251,7 @@ impl HummockEventHandler { let upload_task_latency = upload_task_latency.clone(); let wait_poll_latency = wait_poll_latency.clone(); let upload_compactor_context = upload_compactor_context.clone(); - let filter_key_extractor_manager = filter_key_extractor_manager.clone(); + let compaction_catalog_manager_ref = compaction_catalog_manager_ref.clone(); let sstable_object_id_manager = sstable_object_id_manager.clone(); spawn({ let future = async move { @@ -262,7 +262,7 @@ impl HummockEventHandler { .flat_map(|imms| imms.into_iter()) .collect(), upload_compactor_context.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_manager_ref.clone(), sstable_object_id_manager.clone(), ) .await?; diff --git a/src/storage/src/hummock/observer_manager.rs b/src/storage/src/hummock/observer_manager.rs index a9171005aeaa9..62a2842fddca9 100644 --- a/src/storage/src/hummock/observer_manager.rs +++ b/src/storage/src/hummock/observer_manager.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; use std::sync::Arc; use risingwave_common_service::ObserverState; @@ -24,13 +23,13 @@ use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::SubscribeResponse; use tokio::sync::mpsc::UnboundedSender; -use crate::filter_key_extractor::{FilterKeyExtractorImpl, FilterKeyExtractorManagerRef}; +use crate::compaction_catalog_manager::CompactionCatalogManagerRef; use crate::hummock::backup_reader::BackupReaderRef; use crate::hummock::event_handler::HummockVersionUpdate; use crate::hummock::write_limiter::WriteLimiterRef; pub struct HummockObserverNode { - filter_key_extractor_manager: FilterKeyExtractorManagerRef, + compaction_catalog_manager: CompactionCatalogManagerRef, backup_reader: BackupReaderRef, write_limiter: WriteLimiterRef, version_update_sender: UnboundedSender, @@ -140,13 +139,13 @@ impl ObserverState for HummockObserverNode { impl HummockObserverNode { pub fn new( - filter_key_extractor_manager: FilterKeyExtractorManagerRef, + compaction_catalog_manager: CompactionCatalogManagerRef, backup_reader: BackupReaderRef, version_update_sender: UnboundedSender, write_limiter: WriteLimiterRef, ) -> Self { Self { - filter_key_extractor_manager, + compaction_catalog_manager, backup_reader, version_update_sender, version: 0, @@ -155,25 +154,19 @@ impl HummockObserverNode { } fn handle_catalog_snapshot(&mut self, tables: Vec
) { - let all_filter_key_extractors: HashMap> = tables - .iter() - .map(|t| (t.id, Arc::new(FilterKeyExtractorImpl::from_table(t)))) - .collect(); - self.filter_key_extractor_manager - .sync(all_filter_key_extractors); + self.compaction_catalog_manager + .sync(tables.into_iter().map(|t| (t.id, Arc::new(t))).collect()); } fn handle_catalog_notification(&mut self, operation: Operation, table_catalog: Table) { match operation { Operation::Add | Operation::Update => { - self.filter_key_extractor_manager.update( - table_catalog.id, - Arc::new(FilterKeyExtractorImpl::from_table(&table_catalog)), - ); + self.compaction_catalog_manager + .update(table_catalog.id, Arc::new(table_catalog)); } Operation::Delete => { - self.filter_key_extractor_manager.remove(table_catalog.id); + self.compaction_catalog_manager.remove(table_catalog.id); } _ => panic!("receive an unsupported notify {:?}", operation), diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index 24e7e14e02e0f..cace26e3267d9 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeSet; +use std::collections::{BTreeSet, HashMap}; use std::sync::Arc; use std::time::SystemTime; use bytes::{Bytes, BytesMut}; +use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::key::{user_key, FullKey, MAX_KEY_LEN}; use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::sstable_info::SstableInfo; @@ -29,7 +30,10 @@ use super::{ BlockBuilder, BlockBuilderOptions, BlockMeta, SstableMeta, SstableWriter, DEFAULT_BLOCK_SIZE, DEFAULT_ENTRY_SIZE, DEFAULT_RESTART_INTERVAL, VERSION, }; -use crate::filter_key_extractor::{FilterKeyExtractorImpl, FullKeyFilterKeyExtractor}; +use crate::compaction_catalog_manager::{ + CompactionCatalogAgent, CompactionCatalogAgentRef, FilterKeyExtractorImpl, + FullKeyFilterKeyExtractor, +}; use crate::hummock::sstable::{utils, FilterBuilder}; use crate::hummock::value::HummockValue; use crate::hummock::{ @@ -98,7 +102,8 @@ pub struct SstableBuilder { writer: W, /// Current block builder. block_builder: BlockBuilder, - filter_key_extractor: Arc, + + compaction_catalog_agent_ref: CompactionCatalogAgentRef, /// Block metadata vec. block_metas: Vec, @@ -126,13 +131,23 @@ pub struct SstableBuilder { } impl SstableBuilder { - pub fn for_test(sstable_id: u64, writer: W, options: SstableBuilderOptions) -> Self { + pub fn for_test( + sstable_id: u64, + writer: W, + options: SstableBuilderOptions, + table_id_to_vnode: HashMap, + ) -> Self { + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + table_id_to_vnode, + )); + Self::new( sstable_id, writer, Xor16FilterBuilder::new(options.capacity / DEFAULT_ENTRY_SIZE + 1), options, - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + compaction_catalog_agent_ref, None, ) } @@ -144,7 +159,7 @@ impl SstableBuilder { writer: W, filter_builder: F, options: SstableBuilderOptions, - filter_key_extractor: Arc, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, memory_limiter: Option>, ) -> Self { Self { @@ -163,7 +178,7 @@ impl SstableBuilder { raw_value: BytesMut::new(), last_full_key: vec![], sstable_id, - filter_key_extractor, + compaction_catalog_agent_ref, table_stats: Default::default(), last_table_stats: Default::default(), epoch_set: BTreeSet::default(), @@ -340,7 +355,7 @@ impl SstableBuilder { let table_id = full_key.user_key.table_id.table_id(); let mut extract_key = user_key(&self.raw_key); - extract_key = self.filter_key_extractor.extract(extract_key); + extract_key = self.compaction_catalog_agent_ref.extract(extract_key); // add bloom_filter check if !extract_key.is_empty() { self.filter_builder.add_key(extract_key, table_id); @@ -689,7 +704,7 @@ impl SstableBuilderOutputStats { #[cfg(test)] pub(super) mod tests { - use std::collections::Bound; + use std::collections::{Bound, HashMap}; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; @@ -698,7 +713,9 @@ pub(super) mod tests { use super::*; use crate::assert_bytes_eq; - use crate::filter_key_extractor::{DummyFilterKeyExtractor, MultiFilterKeyExtractor}; + use crate::compaction_catalog_manager::{ + CompactionCatalogAgent, DummyFilterKeyExtractor, MultiFilterKeyExtractor, + }; use crate::hummock::iterator::test_utils::mock_sstable_store; use crate::hummock::sstable::xor_filter::BlockedXor16FilterBuilder; use crate::hummock::test_utils::{ @@ -718,7 +735,8 @@ pub(super) mod tests { ..Default::default() }; - let b = SstableBuilder::for_test(0, mock_sst_writer(&opt), opt); + let table_id_to_vnode = HashMap::from_iter(vec![(0, VirtualNode::COUNT_FOR_TEST)]); + let b = SstableBuilder::for_test(0, mock_sst_writer(&opt), opt, table_id_to_vnode); b.finish().await.unwrap(); } @@ -726,7 +744,9 @@ pub(super) mod tests { #[tokio::test] async fn test_basic() { let opt = default_builder_opt_for_test(); - let mut b = SstableBuilder::for_test(0, mock_sst_writer(&opt), opt); + + let table_id_to_vnode = HashMap::from_iter(vec![(0, VirtualNode::COUNT_FOR_TEST)]); + let mut b = SstableBuilder::for_test(0, mock_sst_writer(&opt), opt, table_id_to_vnode); for i in 0..TEST_KEYS_COUNT { b.add_for_test( @@ -827,12 +847,24 @@ pub(super) mod tests { 3, Arc::new(FilterKeyExtractorImpl::Dummy(DummyFilterKeyExtractor)), ); + + let table_id_to_vnode = HashMap::from_iter(vec![ + (1, VirtualNode::COUNT_FOR_TEST), + (2, VirtualNode::COUNT_FOR_TEST), + (3, VirtualNode::COUNT_FOR_TEST), + ]); + + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + Arc::new(FilterKeyExtractorImpl::Multi(filter)), + table_id_to_vnode, + )); + let mut builder = SstableBuilder::new( object_id, writer, BlockedXor16FilterBuilder::new(1024), opts, - Arc::new(FilterKeyExtractorImpl::Multi(filter)), + compaction_catalog_agent_ref, None, ); diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index 45870257bc04e..b70f7261e5d79 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering::SeqCst; use std::sync::Arc; @@ -21,6 +21,7 @@ use bytes::Bytes; use futures::stream::FuturesUnordered; use futures::StreamExt; use num_integer::Integer; +use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::key::{FullKey, UserKey}; use risingwave_hummock_sdk::LocalSstableInfo; @@ -366,7 +367,11 @@ impl TableBuilderFactory for LocalTableBuilderFactory { .sstable_store .clone() .create_sst_writer(id, writer_options); - let builder = SstableBuilder::for_test(id, writer, self.options.clone()); + let table_id_to_vnode = HashMap::from_iter(vec![( + TableId::default().table_id(), + VirtualNode::COUNT_FOR_TEST, + )]); + let builder = SstableBuilder::for_test(id, writer, self.options.clone(), table_id_to_vnode); Ok(builder) } diff --git a/src/storage/src/hummock/sstable/xor_filter.rs b/src/storage/src/hummock/sstable/xor_filter.rs index 1df4333fca459..a5e530b14bcb5 100644 --- a/src/storage/src/hummock/sstable/xor_filter.rs +++ b/src/storage/src/hummock/sstable/xor_filter.rs @@ -442,13 +442,18 @@ impl Clone for XorFilterReader { #[cfg(test)] mod tests { + use std::collections::HashMap; + use foyer::CacheContext; use rand::RngCore; + use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::EpochWithGap; use super::*; - use crate::filter_key_extractor::{FilterKeyExtractorImpl, FullKeyFilterKeyExtractor}; + use crate::compaction_catalog_manager::{ + CompactionCatalogAgent, FilterKeyExtractorImpl, FullKeyFilterKeyExtractor, + }; use crate::hummock::iterator::test_utils::mock_sstable_store; use crate::hummock::sstable::{SstableBuilder, SstableBuilderOptions}; use crate::hummock::test_utils::{test_user_key_of, test_value_of, TEST_KEYS_COUNT}; @@ -475,12 +480,19 @@ mod tests { let writer = sstable_store .clone() .create_sst_writer(object_id, writer_opts); + + let table_id_to_vnode = HashMap::from_iter(vec![(0, VirtualNode::COUNT_FOR_TEST)]); + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + table_id_to_vnode, + )); + let mut builder = SstableBuilder::new( object_id, writer, BlockedXor16FilterBuilder::create(0.01, 2048), opts, - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + compaction_catalog_agent_ref, None, ); let mut rng = rand::thread_rng(); diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 665b064181687..073dd9e1dc1c4 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -38,8 +38,10 @@ use tokio::sync::oneshot; use super::local_hummock_storage::LocalHummockStorage; use super::version::{read_filter_for_version, CommittedVersion, HummockVersionReader}; +use crate::compaction_catalog_manager::{ + CompactionCatalogManager, CompactionCatalogManagerRef, FakeRemoteTableAccessor, +}; use crate::error::StorageResult; -use crate::filter_key_extractor::{FilterKeyExtractorManager, RpcFilterKeyExtractorManager}; use crate::hummock::backup_reader::{BackupReader, BackupReaderRef}; use crate::hummock::compactor::{ new_compaction_await_tree_reg_ref, CompactionAwaitTreeRegRef, CompactorContext, @@ -90,7 +92,7 @@ pub struct HummockStorage { context: CompactorContext, - filter_key_extractor_manager: FilterKeyExtractorManager, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, sstable_object_id_manager: SstableObjectIdManagerRef, @@ -148,7 +150,7 @@ impl HummockStorage { sstable_store: SstableStoreRef, hummock_meta_client: Arc, notification_client: impl NotificationClient, - filter_key_extractor_manager: Arc, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, state_store_metrics: Arc, compactor_metrics: Arc, await_tree_config: Option, @@ -170,7 +172,7 @@ impl HummockStorage { let observer_manager = ObserverManager::new( notification_client, HummockObserverNode::new( - filter_key_extractor_manager.clone(), + compaction_catalog_manager_ref.clone(), backup_reader.clone(), version_update_tx.clone(), write_limiter.clone(), @@ -191,9 +193,6 @@ impl HummockStorage { hummock_meta_client.clone(), options.max_version_pinning_duration_sec, )); - let filter_key_extractor_manager = FilterKeyExtractorManager::RpcFilterKeyExtractorManager( - filter_key_extractor_manager.clone(), - ); let await_tree_reg = await_tree_config.map(new_compaction_await_tree_reg_ref); @@ -208,7 +207,7 @@ impl HummockStorage { version_update_rx, pinned_version, compactor_context.clone(), - filter_key_extractor_manager.clone(), + compaction_catalog_manager_ref.clone(), sstable_object_id_manager.clone(), state_store_metrics.clone(), ); @@ -217,7 +216,7 @@ impl HummockStorage { let instance = Self { context: compactor_context, - filter_key_extractor_manager: filter_key_extractor_manager.clone(), + compaction_catalog_manager_ref: compaction_catalog_manager_ref.clone(), sstable_object_id_manager, buffer_tracker: hummock_event_handler.buffer_tracker().clone(), version_update_notifier_tx: hummock_event_handler.version_update_notifier_tx(), @@ -536,8 +535,8 @@ impl HummockStorage { &self.sstable_object_id_manager } - pub fn filter_key_extractor_manager(&self) -> &FilterKeyExtractorManager { - &self.filter_key_extractor_manager + pub fn compaction_catalog_manager_ref(&self) -> CompactionCatalogManagerRef { + self.compaction_catalog_manager_ref.clone() } pub fn get_memory_limiter(&self) -> Arc { @@ -763,12 +762,16 @@ impl HummockStorage { hummock_meta_client: Arc, notification_client: impl NotificationClient, ) -> HummockResult { + let compaction_catalog_manager = Arc::new(CompactionCatalogManager::new(Box::new( + FakeRemoteTableAccessor {}, + ))); + Self::new( options, sstable_store, hummock_meta_client, notification_client, - Arc::new(RpcFilterKeyExtractorManager::default()), + compaction_catalog_manager, Arc::new(HummockStateStoreMetrics::unused()), Arc::new(CompactorMetrics::unused()), None, diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index ee3da3088dbd5..b3b1411742ecf 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; use bytes::Bytes; @@ -33,7 +34,9 @@ use super::iterator::test_utils::iterator_test_table_key_of; use super::{ HummockResult, InMemWriter, SstableMeta, SstableWriterOptions, DEFAULT_RESTART_INTERVAL, }; -use crate::filter_key_extractor::{FilterKeyExtractorImpl, FullKeyFilterKeyExtractor}; +use crate::compaction_catalog_manager::{ + CompactionCatalogAgent, FilterKeyExtractorImpl, FullKeyFilterKeyExtractor, +}; use crate::hummock::shared_buffer::shared_buffer_batch::{ SharedBufferBatch, SharedBufferItem, SharedBufferValue, }; @@ -153,7 +156,11 @@ pub async fn gen_test_sstable_data( opts: SstableBuilderOptions, kv_iter: impl Iterator>, HummockValue>)>, ) -> (Bytes, SstableMeta) { - let mut b = SstableBuilder::for_test(0, mock_sst_writer(&opts), opts); + let table_id_to_vnode = HashMap::from_iter(vec![( + TableId::default().table_id(), + VirtualNode::COUNT_FOR_TEST, + )]); + let mut b = SstableBuilder::for_test(0, mock_sst_writer(&opts), opts, table_id_to_vnode); for (key, value) in kv_iter { b.add_for_test(key.to_ref(), value.as_slice()) .await @@ -231,12 +238,22 @@ pub async fn gen_test_sstable_impl + Clone + Default + Eq, F: Fil let writer = sstable_store .clone() .create_sst_writer(object_id, writer_opts); + + let table_id_to_vnode = HashMap::from_iter(vec![( + TableId::default().table_id(), + VirtualNode::COUNT_FOR_TEST, + )]); + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + table_id_to_vnode, + )); + let mut b = SstableBuilder::<_, F>::new( object_id, writer, F::create(opts.bloom_false_positive, opts.capacity / 16), opts, - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + compaction_catalog_agent_ref, None, ); diff --git a/src/storage/src/lib.rs b/src/storage/src/lib.rs index 779062767c7ae..e4afdb5c00c82 100644 --- a/src/storage/src/lib.rs +++ b/src/storage/src/lib.rs @@ -52,7 +52,7 @@ pub mod opts; pub mod store_impl; pub mod table; -pub mod filter_key_extractor; +pub mod compaction_catalog_manager; pub mod mem_table; #[cfg(test)] #[cfg(feature = "failpoints")] diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index ae9f630f304cb..f63e1aaba9abc 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -26,8 +26,8 @@ use risingwave_common_service::RpcNotificationClient; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_object_store::object::build_remote_object_store; +use crate::compaction_catalog_manager::{CompactionCatalogManager, RemoteTableAccessor}; use crate::error::StorageResult; -use crate::filter_key_extractor::{RemoteTableAccessor, RpcFilterKeyExtractorManager}; use crate::hummock::hummock_meta_client::MonitoredHummockMetaClient; use crate::hummock::{ Block, BlockCacheEventListener, HummockError, HummockStorage, RecentFilter, Sstable, @@ -764,15 +764,17 @@ impl StateStoreImpl { })); let notification_client = RpcNotificationClient::new(hummock_meta_client.get_inner().clone()); - let key_filter_manager = Arc::new(RpcFilterKeyExtractorManager::new(Box::new( - RemoteTableAccessor::new(hummock_meta_client.get_inner().clone()), - ))); + let compaction_catalog_manager_ref = + Arc::new(CompactionCatalogManager::new(Box::new( + RemoteTableAccessor::new(hummock_meta_client.get_inner().clone()), + ))); + let inner = HummockStorage::new( opts.clone(), sstable_store, hummock_meta_client.clone(), notification_client, - key_filter_manager, + compaction_catalog_manager_ref, state_store_metrics.clone(), compactor_metrics.clone(), await_tree_config, From 7ed5b820e1bc90c55222ec43f937be384671386a Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 23 Oct 2024 14:59:40 +0800 Subject: [PATCH 2/8] feat(compaction): introduce per table vnode for multi builder --- src/storage/benches/bench_compactor.rs | 8 ++- src/storage/benches/bench_multi_builder.rs | 8 +++ .../hummock_test/src/compactor_tests.rs | 68 +++---------------- src/storage/src/compaction_catalog_manager.rs | 32 ++++++++- .../compactor/fast_compactor_runner.rs | 3 +- src/storage/src/hummock/compactor/mod.rs | 3 +- .../src/hummock/sstable/multi_builder.rs | 48 +++++++++---- 7 files changed, 93 insertions(+), 77 deletions(-) diff --git a/src/storage/benches/bench_compactor.rs b/src/storage/benches/bench_compactor.rs index e46951202e0f5..f09c5c3fa15b9 100644 --- a/src/storage/benches/bench_compactor.rs +++ b/src/storage/benches/bench_compactor.rs @@ -33,6 +33,7 @@ use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::{InMemObjectStore, ObjectStore, ObjectStoreImpl}; use risingwave_pb::hummock::compact_task::PbTaskType; use risingwave_pb::hummock::PbTableSchema; +use risingwave_storage::compaction_catalog_manager::CompactionCatalogAgent; use risingwave_storage::hummock::compactor::compactor_runner::compact_and_build_sst; use risingwave_storage::hummock::compactor::{ ConcatSstableIterator, DummyCompactionFilter, TaskConfig, TaskProgress, @@ -285,8 +286,11 @@ async fn compact>( bloom_false_positive: 0.001, ..Default::default() }; - let mut builder = - CapacitySplitTableBuilder::for_test(LocalTableBuilderFactory::new(32, sstable_store, opt)); + let compaction_catalog_agent_ref = CompactionCatalogAgent::for_test(vec![0]); + let mut builder = CapacitySplitTableBuilder::for_test( + LocalTableBuilderFactory::new(32, sstable_store, opt), + compaction_catalog_agent_ref, + ); let task_config = task_config.unwrap_or_else(|| TaskConfig { key_range: KeyRange::inf(), diff --git a/src/storage/benches/bench_multi_builder.rs b/src/storage/benches/bench_multi_builder.rs index d37e667e75ea9..fc25674ad626b 100644 --- a/src/storage/benches/bench_multi_builder.rs +++ b/src/storage/benches/bench_multi_builder.rs @@ -31,6 +31,7 @@ use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_object_store::object::{ InMemObjectStore, ObjectStore, ObjectStoreImpl, S3ObjectStore, }; +use risingwave_storage::compaction_catalog_manager::CompactionCatalogAgent; use risingwave_storage::hummock::iterator::{ConcatIterator, ConcatIteratorInner, HummockIterator}; use risingwave_storage::hummock::multi_builder::{CapacitySplitTableBuilder, TableBuilderFactory}; use risingwave_storage::hummock::value::HummockValue; @@ -198,6 +199,8 @@ fn bench_builder( let sstable_store = runtime.block_on(async { generate_sstable_store(object_store).await }); + let compaction_catalog_agent_ref = CompactionCatalogAgent::for_test(vec![0]); + let mut group = c.benchmark_group("bench_multi_builder"); group .sample_size(SAMPLE_COUNT) @@ -211,6 +214,7 @@ fn bench_builder( StreamingSstableWriterFactory::new(sstable_store.clone()), get_builder_options(capacity_mb), ), + compaction_catalog_agent_ref.clone(), )) }) }); @@ -223,6 +227,7 @@ fn bench_builder( BatchSstableWriterFactory::new(sstable_store.clone()), get_builder_options(capacity_mb), ), + compaction_catalog_agent_ref.clone(), )) }) }); @@ -255,6 +260,8 @@ fn bench_table_scan(c: &mut Criterion) { let object_store = Arc::new(ObjectStoreImpl::InMem(store)); let sstable_store = runtime.block_on(async { generate_sstable_store(object_store).await }); + let compaction_catalog_agent_ref = CompactionCatalogAgent::for_test(vec![0]); + let ssts = runtime.block_on(async { build_tables(CapacitySplitTableBuilder::for_test( LocalTableBuilderFactory::new( @@ -262,6 +269,7 @@ fn bench_table_scan(c: &mut Criterion) { BatchSstableWriterFactory::new(sstable_store.clone()), get_builder_options(capacity_mb), ), + compaction_catalog_agent_ref.clone(), )) .await }); diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 137c0f76720d8..0e5f2c6ff0d49 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -29,7 +29,7 @@ pub(crate) mod tests { use risingwave_common::util::epoch::{test_epoch, Epoch, EpochExt}; use risingwave_common_service::NotificationClient; use risingwave_hummock_sdk::compact_task::CompactTask; - use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; + use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{ next_key, prefix_slice_with_vnode, prefixed_range_with_vnode, FullKey, TableKey, TABLE_PREFIX_LEN, @@ -55,7 +55,7 @@ pub(crate) mod tests { use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::compaction_catalog_manager::{ CompactionCatalogAgent, CompactionCatalogAgentRef, FilterKeyExtractorImpl, - FixedLengthFilterKeyExtractor, FullKeyFilterKeyExtractor, MultiFilterKeyExtractor, + FixedLengthFilterKeyExtractor, MultiFilterKeyExtractor, }; use risingwave_storage::hummock::compactor::compactor_runner::{compact, CompactorRunner}; use risingwave_storage::hummock::compactor::fast_compactor_runner::CompactorRunner as FastCompactorRunner; @@ -1289,20 +1289,6 @@ pub(crate) mod tests { } } - fn build_full_filter_key_extractor_with_table_ids( - table_ids: Vec, - ) -> Arc { - let mut multi_filter_key_extractor = MultiFilterKeyExtractor::default(); - for table_id in table_ids { - multi_filter_key_extractor.register( - table_id, - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), - ); - } - - Arc::new(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)) - } - async fn run_fast_and_normal_runner( compact_ctx: CompactorContext, task: CompactTask, @@ -1363,16 +1349,8 @@ pub(crate) mod tests { .await; hummock_manager_ref.get_new_sst_ids(10).await.unwrap(); let compact_ctx = get_compactor_context(&storage); - - let multi_filter_key_extractor = - build_full_filter_key_extractor_with_table_ids(vec![existing_table_id]); - let table_id_to_vnode = - HashMap::from_iter(vec![(existing_table_id, VirtualNode::COUNT_FOR_TEST)]); - - let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - multi_filter_key_extractor, - table_id_to_vnode, - )); + let compaction_catalog_agent_ref = + CompactionCatalogAgent::for_test(vec![existing_table_id]); let sstable_store = compact_ctx.sstable_store.clone(); let capacity = 256 * 1024; @@ -1553,17 +1531,9 @@ pub(crate) mod tests { ) .await; hummock_manager_ref.get_new_sst_ids(10).await.unwrap(); - // let (compact_ctx, _) = prepare_compactor_and_filter(&storage, existing_table_id); let compact_ctx = get_compactor_context(&storage); - let multi_filter_key_extractor = - build_full_filter_key_extractor_with_table_ids(vec![existing_table_id]); - let table_id_to_vnode = - HashMap::from_iter(vec![(existing_table_id, VirtualNode::COUNT_FOR_TEST)]); - - let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - multi_filter_key_extractor, - table_id_to_vnode, - )); + let compaction_catalog_agent_ref = + CompactionCatalogAgent::for_test(vec![existing_table_id]); let sstable_store = compact_ctx.sstable_store.clone(); let capacity = 256 * 1024; @@ -1690,15 +1660,8 @@ pub(crate) mod tests { .await; hummock_manager_ref.get_new_sst_ids(10).await.unwrap(); let compact_ctx = get_compactor_context(&storage); - let multi_filter_key_extractor = - build_full_filter_key_extractor_with_table_ids(vec![existing_table_id]); - let table_id_to_vnode = - HashMap::from_iter(vec![(existing_table_id, VirtualNode::COUNT_FOR_TEST)]); - - let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - multi_filter_key_extractor, - table_id_to_vnode, - )); + let compaction_catalog_agent_ref = + CompactionCatalogAgent::for_test(vec![existing_table_id]); let sstable_store = compact_ctx.sstable_store.clone(); let capacity = 256 * 1024; @@ -1911,19 +1874,8 @@ pub(crate) mod tests { key.put_slice(b"key_prefix"); let key_prefix = key.freeze(); - let multi_filter_key_extractor = build_full_filter_key_extractor_with_table_ids(vec![ - table_id_1.table_id(), - table_id_2.table_id(), - ]); - let table_id_to_vnode = HashMap::from_iter(vec![ - (table_id_1.table_id(), VirtualNode::COUNT_FOR_TEST), - (table_id_2.table_id(), VirtualNode::COUNT_FOR_TEST), - ]); - - let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - multi_filter_key_extractor, - table_id_to_vnode, - )); + let compaction_catalog_agent_ref = + CompactionCatalogAgent::for_test(vec![table_id_1.table_id(), table_id_2.table_id()]); let compact_ctx = get_compactor_context(&storage); let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( diff --git a/src/storage/src/compaction_catalog_manager.rs b/src/storage/src/compaction_catalog_manager.rs index 51591ed8a7107..c45bf2021205b 100644 --- a/src/storage/src/compaction_catalog_manager.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -374,7 +374,8 @@ impl CompactionCatalogManager { } } -/// build from `CompactionCatalogManager` +/// `CompactionCatalogAgent` is a wrapper of `filter_key_extractor_manager` and `table_id_to_vnode` +/// which is used to extract key and get vnode count pub struct CompactionCatalogAgent { filter_key_extractor_manager: Arc, table_id_to_vnode: HashMap, @@ -390,6 +391,30 @@ impl CompactionCatalogAgent { table_id_to_vnode, } } + + pub fn dummy() -> Self { + Self { + filter_key_extractor_manager: Arc::new(FilterKeyExtractorImpl::Dummy( + DummyFilterKeyExtractor, + )), + table_id_to_vnode: Default::default(), + } + } + + pub fn for_test(table_ids: Vec) -> Arc { + let full_key_filter_key_extractor = + Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); + + let table_id_to_vnode = table_ids + .into_iter() + .map(|table_id| (table_id, VirtualNode::COUNT_FOR_TEST)) + .collect(); + + Arc::new(CompactionCatalogAgent::new( + full_key_filter_key_extractor, + table_id_to_vnode, + )) + } } impl CompactionCatalogAgent { @@ -398,7 +423,10 @@ impl CompactionCatalogAgent { } pub fn vnode_count(&self, table_id: StateTableId) -> usize { - *self.table_id_to_vnode.get(&table_id).unwrap() + *self + .table_id_to_vnode + .get(&table_id) + .expect(&format!("table_id not found {}", table_id)) } pub fn table_id_to_vnode_ref(&self) -> &HashMap { diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 619c5f858fa9f..6cdf38261cdea 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -391,7 +391,7 @@ impl CompactorRunner { options, policy: task_config.cache_policy, remote_rpc_cost: get_id_time, - compaction_catalog_agent_ref, + compaction_catalog_agent_ref: compaction_catalog_agent_ref.clone(), sstable_writer_factory: factory, _phantom: PhantomData, }; @@ -403,6 +403,7 @@ impl CompactorRunner { context .storage_opts .compactor_concurrent_uploading_sst_count, + compaction_catalog_agent_ref, ); assert_eq!( task.input_ssts.len(), diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index 69d10cf1dc574..2e7ace2f1c05e 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -240,7 +240,7 @@ impl Compactor { options: self.options.clone(), policy: self.task_config.cache_policy, remote_rpc_cost: self.get_id_time.clone(), - compaction_catalog_agent_ref, + compaction_catalog_agent_ref: compaction_catalog_agent_ref.clone(), sstable_writer_factory: writer_factory, _phantom: PhantomData, }; @@ -253,6 +253,7 @@ impl Compactor { self.context .storage_opts .compactor_concurrent_uploading_sst_count, + compaction_catalog_agent_ref, ); let compaction_statistics = compact_and_build_sst( &mut sst_builder, diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index b70f7261e5d79..98d672ccbfc4c 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -27,6 +27,7 @@ use risingwave_hummock_sdk::key::{FullKey, UserKey}; use risingwave_hummock_sdk::LocalSstableInfo; use tokio::task::JoinHandle; +use crate::compaction_catalog_manager::CompactionCatalogAgentRef; use crate::hummock::compactor::task_progress::TaskProgress; use crate::hummock::sstable::filter::FilterBuilder; use crate::hummock::sstable_store::SstableStoreRef; @@ -79,6 +80,8 @@ where concurrent_upload_join_handle: FuturesUnordered, concurrent_uploading_sst_count: Option, + + compaction_catalog_agent_ref: CompactionCatalogAgentRef, } impl CapacitySplitTableBuilder @@ -93,6 +96,7 @@ where task_progress: Option>, table_vnode_partition: BTreeMap, concurrent_uploading_sst_count: Option, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, ) -> Self { // TODO(var-vnode): should use value from caller let vnode_count = VirtualNode::COUNT_FOR_COMPAT; @@ -110,10 +114,14 @@ where largest_vnode_in_current_partition: vnode_count - 1, concurrent_upload_join_handle: FuturesUnordered::new(), concurrent_uploading_sst_count, + compaction_catalog_agent_ref, } } - pub fn for_test(builder_factory: F) -> Self { + pub fn for_test( + builder_factory: F, + compaction_catalog_agent_ref: CompactionCatalogAgentRef, + ) -> Self { Self { builder_factory, sst_outputs: Vec::new(), @@ -127,6 +135,7 @@ where largest_vnode_in_current_partition: VirtualNode::MAX_FOR_TEST.to_index(), concurrent_upload_join_handle: FuturesUnordered::new(), concurrent_uploading_sst_count: None, + compaction_catalog_agent_ref, } } @@ -223,6 +232,11 @@ where let new_vnode_partition_count = self.table_vnode_partition.get(&user_key.table_id.table_id); + self.vnode_count = self + .compaction_catalog_agent_ref + .vnode_count(user_key.table_id.table_id); + self.largest_vnode_in_current_partition = self.vnode_count - 1; + if new_vnode_partition_count.is_some() || self.table_vnode_partition.contains_key(&self.last_table_id) { @@ -383,6 +397,7 @@ mod tests { use risingwave_common::util::epoch::{test_epoch, EpochExt}; use super::*; + use crate::compaction_catalog_manager::CompactionCatalogAgent; use crate::hummock::iterator::test_utils::mock_sstable_store; use crate::hummock::test_utils::{default_builder_opt_for_test, test_key_of, test_user_key_of}; use crate::hummock::DEFAULT_RESTART_INTERVAL; @@ -399,7 +414,9 @@ mod tests { ..Default::default() }; let builder_factory = LocalTableBuilderFactory::new(1001, mock_sstable_store().await, opts); - let builder = CapacitySplitTableBuilder::for_test(builder_factory); + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::dummy()); + let builder = + CapacitySplitTableBuilder::for_test(builder_factory, compaction_catalog_agent_ref); let results = builder.finish().await.unwrap(); assert!(results.is_empty()); } @@ -415,8 +432,11 @@ mod tests { bloom_false_positive: 0.1, ..Default::default() }; + let compaction_catalog_agent_ref = CompactionCatalogAgent::for_test(vec![0]); + let builder_factory = LocalTableBuilderFactory::new(1001, mock_sstable_store().await, opts); - let mut builder = CapacitySplitTableBuilder::for_test(builder_factory); + let mut builder = + CapacitySplitTableBuilder::for_test(builder_factory, compaction_catalog_agent_ref); for i in 0..table_capacity { builder @@ -439,11 +459,11 @@ mod tests { #[tokio::test] async fn test_table_seal() { let opts = default_builder_opt_for_test(); - let mut builder = CapacitySplitTableBuilder::for_test(LocalTableBuilderFactory::new( - 1001, - mock_sstable_store().await, - opts, - )); + let compaction_catalog_agent_ref = CompactionCatalogAgent::for_test(vec![0]); + let mut builder = CapacitySplitTableBuilder::for_test( + LocalTableBuilderFactory::new(1001, mock_sstable_store().await, opts), + compaction_catalog_agent_ref, + ); let mut epoch = test_epoch(100); macro_rules! add { @@ -483,11 +503,11 @@ mod tests { #[tokio::test] async fn test_initial_not_allowed_split() { let opts = default_builder_opt_for_test(); - let mut builder = CapacitySplitTableBuilder::for_test(LocalTableBuilderFactory::new( - 1001, - mock_sstable_store().await, - opts, - )); + let compaction_catalog_agent_ref = CompactionCatalogAgent::for_test(vec![0]); + let mut builder = CapacitySplitTableBuilder::for_test( + LocalTableBuilderFactory::new(1001, mock_sstable_store().await, opts), + compaction_catalog_agent_ref, + ); builder .add_full_key_for_test(test_key_of(0).to_ref(), HummockValue::put(b"v"), false) .await @@ -509,12 +529,14 @@ mod tests { let table_partition_vnode = BTreeMap::from([(1_u32, 4_u32), (2_u32, 4_u32), (3_u32, 4_u32)]); + let compaction_catalog_agent_ref = CompactionCatalogAgent::for_test(vec![0, 1, 2, 3, 4, 5]); let mut builder = CapacitySplitTableBuilder::new( LocalTableBuilderFactory::new(1001, mock_sstable_store().await, opts), Arc::new(CompactorMetrics::unused()), None, table_partition_vnode, None, + compaction_catalog_agent_ref, ); let mut table_key = VirtualNode::from_index(0).to_be_bytes().to_vec(); From 982c6ddfed2c850f2dc62077726af97dae6b037d Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 23 Oct 2024 15:17:26 +0800 Subject: [PATCH 3/8] doc --- src/storage/src/compaction_catalog_manager.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/storage/src/compaction_catalog_manager.rs b/src/storage/src/compaction_catalog_manager.rs index c45bf2021205b..ddd992dc8488f 100644 --- a/src/storage/src/compaction_catalog_manager.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -257,8 +257,11 @@ impl StateTableAccessor for FakeRemoteTableAccessor { } } +/// `CompactionCatalogManager` is a manager to manage all `Table` which used in compaction pub struct CompactionCatalogManager { + // `table_id_to_catalog` is a map to store all `Table` which used in compaction table_id_to_catalog: RwLock>>, + // `table_accessor` is a accessor to fetch `Table` from meta when the table not found table_accessor: Box, } @@ -278,20 +281,25 @@ impl CompactionCatalogManager { } impl CompactionCatalogManager { + /// `update` is used to update `Table` in `table_id_to_catalog` from notification pub fn update(&self, table_id: u32, catalog: Arc
) { self.table_id_to_catalog.write().insert(table_id, catalog); } + /// `sync` is used to sync all `Table` in `table_id_to_catalog` from notification whole snapshot pub fn sync(&self, catalog_map: HashMap>) { let mut guard = self.table_id_to_catalog.write(); guard.clear(); guard.extend(catalog_map); } + /// `remove` is used to remove `Table` in `table_id_to_catalog` by `table_id` pub fn remove(&self, table_id: u32) { self.table_id_to_catalog.write().remove(&table_id); } + /// `acquire` is used to acquire `CompactionCatalogAgent` by `table_ids` + /// if the table not found in `table_id_to_catalog`, it will fetch from meta pub async fn acquire( &self, mut table_ids: Vec, @@ -301,7 +309,6 @@ impl CompactionCatalogManager { // the table in sst has been deleted // use full key as default - // return Ok(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); return Err(HummockError::other("table_id_set is empty")); } @@ -354,6 +361,7 @@ impl CompactionCatalogManager { ))) } + /// `build_compaction_catalog_agent` is used to build `CompactionCatalogAgent` by `table_catalogs` pub fn build_compaction_catalog_agent( table_catalogs: HashMap>, ) -> CompactionCatalogAgentRef { @@ -375,7 +383,8 @@ impl CompactionCatalogManager { } /// `CompactionCatalogAgent` is a wrapper of `filter_key_extractor_manager` and `table_id_to_vnode` -/// which is used to extract key and get vnode count +/// The `CompactionCatalogAgent` belongs to a compaction task call, which we will build from the `table_ids` contained in a compact task and use it during the compaction. +/// The `CompactionCatalogAgent` can act as a agent for the `CompactionCatalogManager`, providing `extract` and `vnode_count` capabilities. pub struct CompactionCatalogAgent { filter_key_extractor_manager: Arc, table_id_to_vnode: HashMap, @@ -426,7 +435,7 @@ impl CompactionCatalogAgent { *self .table_id_to_vnode .get(&table_id) - .expect(&format!("table_id not found {}", table_id)) + .unwrap_or_else(|| panic!("table_id not found {}", table_id)) } pub fn table_id_to_vnode_ref(&self) -> &HashMap { From f989c695872ed84d66f5c9d8565af72c16914db8 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 24 Oct 2024 14:21:05 +0800 Subject: [PATCH 4/8] fix(compaction): fix table id not found panic --- src/storage/src/compaction_catalog_manager.rs | 20 ++++++++++++++----- src/storage/src/hummock/compactor/mod.rs | 6 ++++-- 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/storage/src/compaction_catalog_manager.rs b/src/storage/src/compaction_catalog_manager.rs index ddd992dc8488f..11277cc405727 100644 --- a/src/storage/src/compaction_catalog_manager.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -323,7 +323,6 @@ impl CompactionCatalogManager { Arc::new(FilterKeyExtractorImpl::from_table(table_catalog.as_ref())), ); table_id_to_vnode.insert(*table_id, table_catalog.vnode_count()); - false } @@ -342,6 +341,14 @@ impl CompactionCatalogManager { e.as_report() )) })?; + + if state_tables.len() != table_ids.len() { + return Err(HummockError::other(format!( + "table_ids not found in meta {:?}", + table_ids + ))); + } + let mut guard = self.table_id_to_catalog.write(); for table_id in table_ids { if let Some(table) = state_tables.remove(&table_id) { @@ -432,10 +439,13 @@ impl CompactionCatalogAgent { } pub fn vnode_count(&self, table_id: StateTableId) -> usize { - *self - .table_id_to_vnode - .get(&table_id) - .unwrap_or_else(|| panic!("table_id not found {}", table_id)) + *self.table_id_to_vnode.get(&table_id).unwrap_or_else(|| { + panic!( + "table_id not found {} all_table_ids {:?}", + table_id, + self.table_id_to_vnode.keys() + ) + }) } pub fn table_id_to_vnode_ref(&self) -> &HashMap { diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index 2e7ace2f1c05e..ff4c802c31876 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -15,6 +15,7 @@ mod compaction_executor; mod compaction_filter; pub mod compaction_utils; +use itertools::Itertools; use risingwave_hummock_sdk::compact_task::{CompactTask, ValidationTask}; use risingwave_pb::compactor::{dispatch_compaction_task_request, DispatchCompactionTaskRequest}; use risingwave_pb::hummock::report_compaction_task_request::{ @@ -523,10 +524,11 @@ pub fn start_compactor( .iter() .flat_map(|level| level.table_infos.iter()) .flat_map(|sst| sst.table_ids.clone()) - .filter(|table_id| existing_table_ids.contains(table_id)), + .filter(|table_id| existing_table_ids.contains(table_id)) + .sorted() + .unique(), ); - let ((compact_task, table_stats, object_timestamps), _memory_tracker) = match compaction_catalog_manager_ref.acquire(compact_table_ids).await { Ok(compaction_catalog_agent_ref) => { compactor_runner::compact( From 421624f77b7d146004e4458fbc98e2e17fbd7fb1 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 24 Oct 2024 14:53:42 +0800 Subject: [PATCH 5/8] add ut --- src/storage/src/compaction_catalog_manager.rs | 27 ++- .../src/hummock/sstable/multi_builder.rs | 171 ++++++++++++++---- 2 files changed, 165 insertions(+), 33 deletions(-) diff --git a/src/storage/src/compaction_catalog_manager.rs b/src/storage/src/compaction_catalog_manager.rs index 11277cc405727..5b25ca90b10ee 100644 --- a/src/storage/src/compaction_catalog_manager.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -344,7 +344,7 @@ impl CompactionCatalogManager { if state_tables.len() != table_ids.len() { return Err(HummockError::other(format!( - "table_ids not found in meta {:?}", + "table_ids not found in catalog {:?}", table_ids ))); } @@ -687,4 +687,29 @@ mod tests { assert_eq!(pk_prefix_len, output_key.len()); } } + + #[tokio::test] + async fn test_compaction_catalog_manager_exception() { + let compaction_catalog_manager = super::CompactionCatalogManager::default(); + + { + let ret = compaction_catalog_manager.acquire(vec![]).await; + assert!(ret.is_err()); + if let Err(e) = ret { + assert_eq!(e.to_string(), "Other error: table_id_set is empty"); + } + } + + { + // network error with FakeRemoteTableAccessor + let ret = compaction_catalog_manager.acquire(vec![1]).await; + assert!(ret.is_err()); + if let Err(e) = ret { + assert_eq!( + e.to_string(), + "Other error: request rpc list_tables for meta failed: fake accessor does not support fetch remote table" + ); + } + } + } } diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index 98d672ccbfc4c..6228ae2c053dd 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -397,7 +397,9 @@ mod tests { use risingwave_common::util::epoch::{test_epoch, EpochExt}; use super::*; - use crate::compaction_catalog_manager::CompactionCatalogAgent; + use crate::compaction_catalog_manager::{ + CompactionCatalogAgent, FilterKeyExtractorImpl, FullKeyFilterKeyExtractor, + }; use crate::hummock::iterator::test_utils::mock_sstable_store; use crate::hummock::test_utils::{default_builder_opt_for_test, test_key_of, test_user_key_of}; use crate::hummock::DEFAULT_RESTART_INTERVAL; @@ -526,27 +528,28 @@ mod tests { ..Default::default() }; - let table_partition_vnode = - BTreeMap::from([(1_u32, 4_u32), (2_u32, 4_u32), (3_u32, 4_u32)]); - - let compaction_catalog_agent_ref = CompactionCatalogAgent::for_test(vec![0, 1, 2, 3, 4, 5]); - let mut builder = CapacitySplitTableBuilder::new( - LocalTableBuilderFactory::new(1001, mock_sstable_store().await, opts), - Arc::new(CompactorMetrics::unused()), - None, - table_partition_vnode, - None, - compaction_catalog_agent_ref, - ); - - let mut table_key = VirtualNode::from_index(0).to_be_bytes().to_vec(); - table_key.extend_from_slice("a".as_bytes()); + { + let table_partition_vnode = + BTreeMap::from([(1_u32, 4_u32), (2_u32, 4_u32), (3_u32, 4_u32)]); + + let compaction_catalog_agent_ref = + CompactionCatalogAgent::for_test(vec![0, 1, 2, 3, 4, 5]); + let mut builder = CapacitySplitTableBuilder::new( + LocalTableBuilderFactory::new(1001, mock_sstable_store().await, opts.clone()), + Arc::new(CompactorMetrics::unused()), + None, + table_partition_vnode, + None, + compaction_catalog_agent_ref, + ); + + let mut table_key = VirtualNode::from_index(0).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); - let switch_builder = - builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); - assert!(switch_builder); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); + assert!(switch_builder); - { let mut table_key = VirtualNode::from_index(62).to_be_bytes().to_vec(); table_key.extend_from_slice("a".as_bytes()); let switch_builder = @@ -564,19 +567,123 @@ mod tests { let switch_builder = builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); assert!(switch_builder); + + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(2), &table_key)); + assert!(switch_builder); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(3), &table_key)); + assert!(switch_builder); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(4), &table_key)); + assert!(switch_builder); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(5), &table_key)); + assert!(!switch_builder); } - let switch_builder = - builder.check_switch_builder(&UserKey::for_test(TableId::from(2), &table_key)); - assert!(switch_builder); - let switch_builder = - builder.check_switch_builder(&UserKey::for_test(TableId::from(3), &table_key)); - assert!(switch_builder); - let switch_builder = - builder.check_switch_builder(&UserKey::for_test(TableId::from(4), &table_key)); - assert!(switch_builder); - let switch_builder = - builder.check_switch_builder(&UserKey::for_test(TableId::from(5), &table_key)); - assert!(!switch_builder); + { + // Test different table vnode count + + let full_key_filter_key_extractor = + Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); + + let table_partition_vnode = + BTreeMap::from([(1_u32, 4_u32), (2_u32, 4_u32), (3_u32, 4_u32)]); + + let table_id_to_vnode = HashMap::from_iter(vec![(1, 64), (2, 128), (3, 256)]); + let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( + full_key_filter_key_extractor, + table_id_to_vnode, + )); + + let mut builder = CapacitySplitTableBuilder::new( + LocalTableBuilderFactory::new(1001, mock_sstable_store().await, opts), + Arc::new(CompactorMetrics::unused()), + None, + table_partition_vnode, + None, + compaction_catalog_agent_ref, + ); + + let mut table_key = VirtualNode::from_index(0).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); + assert!(switch_builder); + + let mut table_key = VirtualNode::from_index(15).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); + assert!(!switch_builder); + + let mut table_key = VirtualNode::from_index(16).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); + assert!(switch_builder); + + let mut table_key = VirtualNode::from_index(0).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(2), &table_key)); + assert!(switch_builder); + + let mut table_key = VirtualNode::from_index(16).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(2), &table_key)); + assert!(!switch_builder); + + let mut table_key = VirtualNode::from_index(31).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(2), &table_key)); + assert!(!switch_builder); + + let mut table_key = VirtualNode::from_index(32).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(2), &table_key)); + assert!(switch_builder); + + let mut table_key = VirtualNode::from_index(64).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(2), &table_key)); + assert!(switch_builder); + + let mut table_key = VirtualNode::from_index(0).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(3), &table_key)); + assert!(switch_builder); + + let mut table_key = VirtualNode::from_index(16).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(3), &table_key)); + assert!(!switch_builder); + + let mut table_key = VirtualNode::from_index(32).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(3), &table_key)); + assert!(!switch_builder); + + let mut table_key = VirtualNode::from_index(63).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(3), &table_key)); + assert!(!switch_builder); + + let mut table_key = VirtualNode::from_index(64).to_be_bytes().to_vec(); + table_key.extend_from_slice("a".as_bytes()); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(3), &table_key)); + assert!(switch_builder); + } } } From 065530b755313925748d8d699318938e5d379132 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 24 Oct 2024 17:57:00 +0800 Subject: [PATCH 6/8] fix panic --- src/storage/src/compaction_catalog_manager.rs | 11 +++--- src/storage/src/hummock/compactor/mod.rs | 34 ++++++++++++++----- .../compactor/shared_buffer_compact.rs | 3 ++ 3 files changed, 33 insertions(+), 15 deletions(-) diff --git a/src/storage/src/compaction_catalog_manager.rs b/src/storage/src/compaction_catalog_manager.rs index 5b25ca90b10ee..3b72b7769bfc4 100644 --- a/src/storage/src/compaction_catalog_manager.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -342,13 +342,6 @@ impl CompactionCatalogManager { )) })?; - if state_tables.len() != table_ids.len() { - return Err(HummockError::other(format!( - "table_ids not found in catalog {:?}", - table_ids - ))); - } - let mut guard = self.table_id_to_catalog.write(); for table_id in table_ids { if let Some(table) = state_tables.remove(&table_id) { @@ -451,6 +444,10 @@ impl CompactionCatalogAgent { pub fn table_id_to_vnode_ref(&self) -> &HashMap { &self.table_id_to_vnode } + + pub fn table_ids(&self) -> impl Iterator + '_ { + self.table_id_to_vnode.keys().cloned() + } } pub type CompactionCatalogManagerRef = Arc; diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index ff4c802c31876..fee5bd32fbbc2 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -17,6 +17,7 @@ mod compaction_filter; pub mod compaction_utils; use itertools::Itertools; use risingwave_hummock_sdk::compact_task::{CompactTask, ValidationTask}; +use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_pb::compactor::{dispatch_compaction_task_request, DispatchCompactionTaskRequest}; use risingwave_pb::hummock::report_compaction_task_request::{ Event as ReportCompactionTaskEvent, HeartBeat as SharedHeartBeat, @@ -529,15 +530,32 @@ pub fn start_compactor( .unique(), ); - let ((compact_task, table_stats, object_timestamps), _memory_tracker) = match compaction_catalog_manager_ref.acquire(compact_table_ids).await { + let ((compact_task, table_stats, object_timestamps), _memory_tracker) = match compaction_catalog_manager_ref.acquire(compact_table_ids.clone()).await { Ok(compaction_catalog_agent_ref) => { - compactor_runner::compact( - context.clone(), - compact_task, - rx, - Box::new(sstable_object_id_manager.clone()), - compaction_catalog_agent_ref, - ).await + let acquire_table_ids: HashSet = compaction_catalog_agent_ref.table_ids().collect(); + if acquire_table_ids.len() != compact_table_ids.len() { + let diff = compact_table_ids.into_iter().collect::>() + .symmetric_difference(&acquire_table_ids) + .cloned() + .collect::>(); + tracing::warn!( + dif= ?diff, + "Some table ids are not acquired." + ); + let task_status = TaskStatus::ExecuteFailed; + ( + compact_done(compact_task, context.clone(), vec![], task_status), + None, + ) + } else { + compactor_runner::compact( + context.clone(), + compact_task, + rx, + Box::new(sstable_object_id_manager.clone()), + compaction_catalog_agent_ref, + ).await + } }, Err(e) => { tracing::error!(error = %e.as_report(), "Failed to acquire catalog"); diff --git a/src/storage/src/hummock/compactor/shared_buffer_compact.rs b/src/storage/src/hummock/compactor/shared_buffer_compact.rs index 022033ac26be5..b7ae13b2c1067 100644 --- a/src/storage/src/hummock/compactor/shared_buffer_compact.rs +++ b/src/storage/src/hummock/compactor/shared_buffer_compact.rs @@ -126,6 +126,9 @@ async fn compact_shared_buffer( let compaction_catalog_agent_ref = compaction_catalog_manager_ref .acquire(existing_table_ids.iter().copied().collect()) .await?; + let existing_table_ids = compaction_catalog_agent_ref + .table_ids() + .collect::>(); payload.retain(|imm| { let ret = existing_table_ids.contains(&imm.table_id.table_id); if !ret { From 48e04d14930e0dd994a394e64cc20448b0113fca Mon Sep 17 00:00:00 2001 From: Li0k Date: Tue, 29 Oct 2024 16:45:46 +0800 Subject: [PATCH 7/8] address comments --- .../hummock_test/src/compactor_tests.rs | 28 +++--- .../hummock_test/src/sync_point_tests.rs | 4 +- src/storage/src/compaction_catalog_manager.rs | 29 +++--- .../src/hummock/compactor/compactor_runner.rs | 92 ++++++++++++++++++- src/storage/src/hummock/compactor/mod.rs | 61 ++---------- src/storage/src/hummock/sstable/builder.rs | 16 +--- .../src/hummock/sstable/multi_builder.rs | 6 +- src/storage/src/hummock/sstable/xor_filter.rs | 2 +- src/storage/src/hummock/test_utils.rs | 2 +- 9 files changed, 136 insertions(+), 104 deletions(-) diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 0e5f2c6ff0d49..fe2299a63683a 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -57,7 +57,9 @@ pub(crate) mod tests { CompactionCatalogAgent, CompactionCatalogAgentRef, FilterKeyExtractorImpl, FixedLengthFilterKeyExtractor, MultiFilterKeyExtractor, }; - use risingwave_storage::hummock::compactor::compactor_runner::{compact, CompactorRunner}; + use risingwave_storage::hummock::compactor::compactor_runner::{ + compact_with_agent, CompactorRunner, + }; use risingwave_storage::hummock::compactor::fast_compactor_runner::CompactorRunner as FastCompactorRunner; use risingwave_storage::hummock::compactor::{ CompactionExecutor, CompactorContext, DummyCompactionFilter, TaskProgress, @@ -285,7 +287,7 @@ pub(crate) mod tests { { // 3. compact let (_tx, rx) = tokio::sync::oneshot::channel(); - let ((result_task, task_stats, object_timestamps), _) = compact( + let ((result_task, task_stats, object_timestamps), _) = compact_with_agent( compact_ctx.clone(), compact_task.clone(), rx, @@ -605,7 +607,7 @@ pub(crate) mod tests { // 4. compact let (_tx, rx) = tokio::sync::oneshot::channel(); - let ((result_task, task_stats, object_timestamps), _) = compact( + let ((result_task, task_stats, object_timestamps), _) = compact_with_agent( compact_ctx, compact_task.clone(), rx, @@ -801,7 +803,7 @@ pub(crate) mod tests { // 3. compact let (_tx, rx) = tokio::sync::oneshot::channel(); - let ((result_task, task_stats, object_timestamps), _) = compact( + let ((result_task, task_stats, object_timestamps), _) = compact_with_agent( compact_ctx, compact_task.clone(), rx, @@ -910,19 +912,15 @@ pub(crate) mod tests { let mut multi_filter_key_extractor = MultiFilterKeyExtractor::default(); multi_filter_key_extractor.register( existing_table_id, - Arc::new(FilterKeyExtractorImpl::FixedLength( - FixedLengthFilterKeyExtractor::new(TABLE_PREFIX_LEN + key_prefix.len()), + FilterKeyExtractorImpl::FixedLength(FixedLengthFilterKeyExtractor::new( + TABLE_PREFIX_LEN + key_prefix.len(), )), ); let table_id_to_vnode = HashMap::from_iter([(existing_table_id, VirtualNode::COUNT_FOR_TEST)]); - - let filter_key_extractor = - Arc::new(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)); - let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - filter_key_extractor, + FilterKeyExtractorImpl::Multi(multi_filter_key_extractor), table_id_to_vnode, )); @@ -1002,7 +1000,7 @@ pub(crate) mod tests { // 3. compact let (_tx, rx) = tokio::sync::oneshot::channel(); - let ((result_task, task_stats, object_timestamps), _) = compact( + let ((result_task, task_stats, object_timestamps), _) = compact_with_agent( compact_ctx, compact_task.clone(), rx, @@ -1187,7 +1185,7 @@ pub(crate) mod tests { // 3. compact let (_tx, rx) = tokio::sync::oneshot::channel(); - let ((result_task, task_stats, object_timestamps), _) = compact( + let ((result_task, task_stats, object_timestamps), _) = compact_with_agent( compact_ctx, compact_task.clone(), rx, @@ -2013,7 +2011,7 @@ pub(crate) mod tests { // 3. compact let (_tx, rx) = tokio::sync::oneshot::channel(); - let ((result_task, task_stats, object_timestamps), _) = compact( + let ((result_task, task_stats, object_timestamps), _) = compact_with_agent( compact_ctx, compact_task.clone(), rx, @@ -2238,7 +2236,7 @@ pub(crate) mod tests { // 3. compact let (_tx, rx) = tokio::sync::oneshot::channel(); - let ((result_task, task_stats, object_timestamps), _) = compact( + let ((result_task, task_stats, object_timestamps), _) = compact_with_agent( compact_ctx.clone(), compact_task.clone(), rx, diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 3d1981ab0003f..065c9536006bb 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -32,7 +32,7 @@ use risingwave_meta::hummock::test_utils::{setup_compute_env, setup_compute_env_ use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::compaction_catalog_manager::CompactionCatalogAgentRef; -use risingwave_storage::hummock::compactor::compactor_runner::compact; +use risingwave_storage::hummock::compactor::compactor_runner::compact_with_agent; use risingwave_storage::hummock::compactor::CompactorContext; use risingwave_storage::hummock::{CachePolicy, GetObjectId, SstableObjectIdManager}; use risingwave_storage::store::{LocalStateStore, NewLocalOptions, ReadOptions, StateStoreRead}; @@ -201,7 +201,7 @@ pub async fn compact_once( compact_task.compaction_filter_mask = compaction_filter_flag.bits(); // 3. compact let (_tx, rx) = tokio::sync::oneshot::channel(); - let ((result_task, task_stats, object_timestamps), _) = compact( + let ((result_task, task_stats, object_timestamps), _) = compact_with_agent( compact_ctx, compact_task.clone(), rx, diff --git a/src/storage/src/compaction_catalog_manager.rs b/src/storage/src/compaction_catalog_manager.rs index 3b72b7769bfc4..bf183f220fecd 100644 --- a/src/storage/src/compaction_catalog_manager.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -185,11 +185,11 @@ impl SchemaFilterKeyExtractor { #[derive(Default)] pub struct MultiFilterKeyExtractor { - id_to_filter_key_extractor: HashMap>, + id_to_filter_key_extractor: HashMap, } impl MultiFilterKeyExtractor { - pub fn register(&mut self, table_id: u32, filter_key_extractor: Arc) { + pub fn register(&mut self, table_id: u32, filter_key_extractor: FilterKeyExtractorImpl) { self.id_to_filter_key_extractor .insert(table_id, filter_key_extractor); } @@ -320,7 +320,7 @@ impl CompactionCatalogManager { Some(table_catalog) => { multi_filter_key_extractor.register( *table_id, - Arc::new(FilterKeyExtractorImpl::from_table(table_catalog.as_ref())), + FilterKeyExtractorImpl::from_table(table_catalog.as_ref()), ); table_id_to_vnode.insert(*table_id, table_catalog.vnode_count()); false @@ -346,7 +346,7 @@ impl CompactionCatalogManager { for table_id in table_ids { if let Some(table) = state_tables.remove(&table_id) { let table_id = table.id; - let key_extractor = Arc::new(FilterKeyExtractorImpl::from_table(&table)); + let key_extractor = FilterKeyExtractorImpl::from_table(&table); let vnode = table.vnode_count(); guard.insert(table_id, Arc::new(table)); multi_filter_key_extractor.register(table_id, key_extractor); @@ -356,7 +356,7 @@ impl CompactionCatalogManager { } Ok(Arc::new(CompactionCatalogAgent::new( - Arc::new(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)), + FilterKeyExtractorImpl::Multi(multi_filter_key_extractor), table_id_to_vnode, ))) } @@ -370,13 +370,13 @@ impl CompactionCatalogManager { for (table_id, table_catalog) in table_catalogs { multi_filter_key_extractor.register( table_id, - Arc::new(FilterKeyExtractorImpl::from_table(table_catalog.as_ref())), + FilterKeyExtractorImpl::from_table(table_catalog.as_ref()), ); table_id_to_vnode.insert(table_id, table_catalog.vnode_count()); } Arc::new(CompactionCatalogAgent::new( - Arc::new(FilterKeyExtractorImpl::Multi(multi_filter_key_extractor)), + FilterKeyExtractorImpl::Multi(multi_filter_key_extractor), table_id_to_vnode, )) } @@ -386,13 +386,13 @@ impl CompactionCatalogManager { /// The `CompactionCatalogAgent` belongs to a compaction task call, which we will build from the `table_ids` contained in a compact task and use it during the compaction. /// The `CompactionCatalogAgent` can act as a agent for the `CompactionCatalogManager`, providing `extract` and `vnode_count` capabilities. pub struct CompactionCatalogAgent { - filter_key_extractor_manager: Arc, + filter_key_extractor_manager: FilterKeyExtractorImpl, table_id_to_vnode: HashMap, } impl CompactionCatalogAgent { pub fn new( - filter_key_extractor_manager: Arc, + filter_key_extractor_manager: FilterKeyExtractorImpl, table_id_to_vnode: HashMap, ) -> Self { Self { @@ -403,16 +403,14 @@ impl CompactionCatalogAgent { pub fn dummy() -> Self { Self { - filter_key_extractor_manager: Arc::new(FilterKeyExtractorImpl::Dummy( - DummyFilterKeyExtractor, - )), + filter_key_extractor_manager: FilterKeyExtractorImpl::Dummy(DummyFilterKeyExtractor), table_id_to_vnode: Default::default(), } } pub fn for_test(table_ids: Vec) -> Arc { let full_key_filter_key_extractor = - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); + FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor); let table_id_to_vnode = table_ids .into_iter() @@ -456,7 +454,6 @@ pub type CompactionCatalogAgentRef = Arc; #[cfg(test)] mod tests { use std::mem; - use std::sync::Arc; use bytes::{BufMut, BytesMut}; use itertools::Itertools; @@ -615,7 +612,7 @@ mod tests { let schema_filter_key_extractor = SchemaFilterKeyExtractor::new(&prost_table); multi_filter_key_extractor.register( 1, - Arc::new(FilterKeyExtractorImpl::Schema(schema_filter_key_extractor)), + FilterKeyExtractorImpl::Schema(schema_filter_key_extractor), ); let order_types: Vec = vec![OrderType::ascending(), OrderType::ascending()]; let schema = vec![DataType::Int64, DataType::Varchar]; @@ -652,7 +649,7 @@ mod tests { let schema_filter_key_extractor = SchemaFilterKeyExtractor::new(&prost_table); multi_filter_key_extractor.register( 2, - Arc::new(FilterKeyExtractorImpl::Schema(schema_filter_key_extractor)), + FilterKeyExtractorImpl::Schema(schema_filter_key_extractor), ); let order_types: Vec = vec![OrderType::ascending(), OrderType::ascending()]; let schema = vec![DataType::Int64, DataType::Varchar]; diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 9e00568154532..f6b1ee832375f 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -24,6 +24,7 @@ use risingwave_hummock_sdk::compact::{ compact_task_to_string, estimate_memory_for_compact_task, statistics_compact_task, }; use risingwave_hummock_sdk::compact_task::CompactTask; +use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::key::{FullKey, FullKeyTracker}; use risingwave_hummock_sdk::key_range::{KeyRange, KeyRangeCommon}; use risingwave_hummock_sdk::sstable_info::SstableInfo; @@ -40,7 +41,7 @@ use tokio::sync::oneshot::Receiver; use super::iterator::MonitoredCompactorIterator; use super::task_progress::TaskProgress; use super::{CompactionStatistics, TaskConfig}; -use crate::compaction_catalog_manager::CompactionCatalogAgentRef; +use crate::compaction_catalog_manager::{CompactionCatalogAgentRef, CompactionCatalogManagerRef}; use crate::hummock::compactor::compaction_utils::{ build_multi_compaction_filter, estimate_task_output_capacity, generate_splits_for_task, metrics_report_for_task, optimize_by_copy_block, @@ -302,7 +303,7 @@ pub fn partition_overlapping_sstable_infos( /// Handles a compaction task and reports its status to hummock manager. /// Always return `Ok` and let hummock manager handle errors. -pub async fn compact( +pub async fn compact_with_agent( compactor_context: CompactorContext, mut compact_task: CompactTask, mut shutdown_rx: Receiver<()>, @@ -558,6 +559,93 @@ pub async fn compact( ) } +/// Handles a compaction task and reports its status to hummock manager. +/// Always return `Ok` and let hummock manager handle errors. +pub async fn compact( + compactor_context: CompactorContext, + compact_task: CompactTask, + shutdown_rx: Receiver<()>, + object_id_getter: Box, + compaction_catalog_manager_ref: CompactionCatalogManagerRef, +) -> ( + ( + CompactTask, + HashMap, + HashMap, + ), + Option, +) { + let existing_table_ids: HashSet = + HashSet::from_iter(compact_task.existing_table_ids.clone()); + let compact_table_ids = Vec::from_iter( + compact_task + .input_ssts + .iter() + .flat_map(|level| level.table_infos.iter()) + .flat_map(|sst| sst.table_ids.clone()) + .filter(|table_id| existing_table_ids.contains(table_id)) + .sorted() + .unique(), + ); + + let compaction_catalog_agent_ref = match compaction_catalog_manager_ref + .acquire(compact_table_ids.clone()) + .await + { + Ok(compaction_catalog_agent_ref) => { + let acquire_table_ids: HashSet = + compaction_catalog_agent_ref.table_ids().collect(); + if acquire_table_ids.len() != compact_table_ids.len() { + let diff = compact_table_ids + .into_iter() + .collect::>() + .symmetric_difference(&acquire_table_ids) + .cloned() + .collect::>(); + tracing::warn!( + dif= ?diff, + "Some table ids are not acquired." + ); + return ( + compact_done( + compact_task, + compactor_context.clone(), + vec![], + TaskStatus::ExecuteFailed, + ), + None, + ); + } + + compaction_catalog_agent_ref + } + Err(e) => { + tracing::warn!( + error = %e.as_report(), + "Failed to acquire compaction catalog agent" + ); + return ( + compact_done( + compact_task, + compactor_context.clone(), + vec![], + TaskStatus::ExecuteFailed, + ), + None, + ); + } + }; + + compact_with_agent( + compactor_context, + compact_task, + shutdown_rx, + object_id_getter, + compaction_catalog_agent_ref, + ) + .await +} + /// Fills in the compact task and tries to report the task result to meta node. pub(crate) fn compact_done( mut compact_task: CompactTask, diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index fee5bd32fbbc2..ec2bb260d8342 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -15,9 +15,7 @@ mod compaction_executor; mod compaction_filter; pub mod compaction_utils; -use itertools::Itertools; use risingwave_hummock_sdk::compact_task::{CompactTask, ValidationTask}; -use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_pb::compactor::{dispatch_compaction_task_request, DispatchCompactionTaskRequest}; use risingwave_pb::hummock::report_compaction_task_request::{ Event as ReportCompactionTaskEvent, HeartBeat as SharedHeartBeat, @@ -36,7 +34,7 @@ mod iterator; mod shared_buffer_compact; pub(super) mod task_progress; -use std::collections::{HashMap, HashSet, VecDeque}; +use std::collections::{HashMap, VecDeque}; use std::marker::PhantomData; use std::sync::atomic::{AtomicU32, AtomicU64, Ordering}; use std::sync::{Arc, Mutex}; @@ -518,54 +516,15 @@ pub fn start_compactor( let (tx, rx) = tokio::sync::oneshot::channel(); let task_id = compact_task.task_id; shutdown.lock().unwrap().insert(task_id, tx); - let existing_table_ids: HashSet = HashSet::from_iter(compact_task.existing_table_ids.clone()); - let compact_table_ids = Vec::from_iter( - compact_task - .input_ssts - .iter() - .flat_map(|level| level.table_infos.iter()) - .flat_map(|sst| sst.table_ids.clone()) - .filter(|table_id| existing_table_ids.contains(table_id)) - .sorted() - .unique(), - ); - let ((compact_task, table_stats, object_timestamps), _memory_tracker) = match compaction_catalog_manager_ref.acquire(compact_table_ids.clone()).await { - Ok(compaction_catalog_agent_ref) => { - let acquire_table_ids: HashSet = compaction_catalog_agent_ref.table_ids().collect(); - if acquire_table_ids.len() != compact_table_ids.len() { - let diff = compact_table_ids.into_iter().collect::>() - .symmetric_difference(&acquire_table_ids) - .cloned() - .collect::>(); - tracing::warn!( - dif= ?diff, - "Some table ids are not acquired." - ); - let task_status = TaskStatus::ExecuteFailed; - ( - compact_done(compact_task, context.clone(), vec![], task_status), - None, - ) - } else { - compactor_runner::compact( - context.clone(), - compact_task, - rx, - Box::new(sstable_object_id_manager.clone()), - compaction_catalog_agent_ref, - ).await - } - }, - Err(e) => { - tracing::error!(error = %e.as_report(), "Failed to acquire catalog"); - let task_status = TaskStatus::ExecuteFailed; - ( - compact_done(compact_task, context.clone(), vec![], task_status), - None, - ) - } - }; + let ((compact_task, table_stats, object_timestamps), _memory_tracker)= compactor_runner::compact( + context.clone(), + compact_task, + rx, + Box::new(sstable_object_id_manager.clone()), + compaction_catalog_manager_ref.clone(), + ) + .await; shutdown.lock().unwrap().remove(&task_id); running_task_parallelism.fetch_sub(parallelism as u32, Ordering::SeqCst); @@ -764,7 +723,7 @@ pub fn start_shared_compactor( shutdown.lock().unwrap().insert(task_id, tx); let compaction_catalog_agent_ref = CompactionCatalogManager::build_compaction_catalog_agent(table_id_to_catalog); - let ((compact_task, table_stats, object_timestamps), _memory_tracker)= compactor_runner::compact( + let ((compact_task, table_stats, object_timestamps), _memory_tracker)= compactor_runner::compact_with_agent( context.clone(), compact_task, rx, diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index cace26e3267d9..0ad46e5e000a2 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -138,7 +138,7 @@ impl SstableBuilder { table_id_to_vnode: HashMap, ) -> Self { let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor), table_id_to_vnode, )); @@ -835,18 +835,12 @@ pub(super) mod tests { .clone() .create_sst_writer(object_id, writer_opts); let mut filter = MultiFilterKeyExtractor::default(); - filter.register( - 1, - Arc::new(FilterKeyExtractorImpl::Dummy(DummyFilterKeyExtractor)), - ); + filter.register(1, FilterKeyExtractorImpl::Dummy(DummyFilterKeyExtractor)); filter.register( 2, - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), - ); - filter.register( - 3, - Arc::new(FilterKeyExtractorImpl::Dummy(DummyFilterKeyExtractor)), + FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor), ); + filter.register(3, FilterKeyExtractorImpl::Dummy(DummyFilterKeyExtractor)); let table_id_to_vnode = HashMap::from_iter(vec![ (1, VirtualNode::COUNT_FOR_TEST), @@ -855,7 +849,7 @@ pub(super) mod tests { ]); let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - Arc::new(FilterKeyExtractorImpl::Multi(filter)), + FilterKeyExtractorImpl::Multi(filter), table_id_to_vnode, )); diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index 6228ae2c053dd..f03bcf057c81f 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -584,16 +584,12 @@ mod tests { { // Test different table vnode count - - let full_key_filter_key_extractor = - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)); - let table_partition_vnode = BTreeMap::from([(1_u32, 4_u32), (2_u32, 4_u32), (3_u32, 4_u32)]); let table_id_to_vnode = HashMap::from_iter(vec![(1, 64), (2, 128), (3, 256)]); let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - full_key_filter_key_extractor, + FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor), table_id_to_vnode, )); diff --git a/src/storage/src/hummock/sstable/xor_filter.rs b/src/storage/src/hummock/sstable/xor_filter.rs index a5e530b14bcb5..8120184b17247 100644 --- a/src/storage/src/hummock/sstable/xor_filter.rs +++ b/src/storage/src/hummock/sstable/xor_filter.rs @@ -483,7 +483,7 @@ mod tests { let table_id_to_vnode = HashMap::from_iter(vec![(0, VirtualNode::COUNT_FOR_TEST)]); let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor), table_id_to_vnode, )); diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index b3b1411742ecf..45d18027c1ebe 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -244,7 +244,7 @@ pub async fn gen_test_sstable_impl + Clone + Default + Eq, F: Fil VirtualNode::COUNT_FOR_TEST, )]); let compaction_catalog_agent_ref = Arc::new(CompactionCatalogAgent::new( - Arc::new(FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor)), + FilterKeyExtractorImpl::FullKey(FullKeyFilterKeyExtractor), table_id_to_vnode, )); From fab9a06a388af759381b0e2bba18bee5259f2fe3 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 31 Oct 2024 14:03:03 +0800 Subject: [PATCH 8/8] revmoe Arc
--- .../compactor_observer/observer_manager.rs | 6 ++--- src/storage/hummock_test/src/test_utils.rs | 6 ++--- src/storage/src/compaction_catalog_manager.rs | 22 ++++++++----------- src/storage/src/hummock/compactor/mod.rs | 2 +- src/storage/src/hummock/observer_manager.rs | 6 ++--- 5 files changed, 17 insertions(+), 25 deletions(-) diff --git a/src/storage/compactor/src/compactor_observer/observer_manager.rs b/src/storage/compactor/src/compactor_observer/observer_manager.rs index d3c36dab82dd9..0a04341ed0de9 100644 --- a/src/storage/compactor/src/compactor_observer/observer_manager.rs +++ b/src/storage/compactor/src/compactor_observer/observer_manager.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::ObserverState; use risingwave_pb::catalog::Table; @@ -92,14 +90,14 @@ impl CompactorObserverNode { fn handle_catalog_snapshot(&mut self, tables: Vec
) { self.compaction_catalog_manager - .sync(tables.into_iter().map(|t| (t.id, Arc::new(t))).collect()); + .sync(tables.into_iter().map(|t| (t.id, t)).collect()); } fn handle_catalog_notification(&mut self, operation: Operation, table_catalog: Table) { match operation { Operation::Add | Operation::Update => { self.compaction_catalog_manager - .update(table_catalog.id, Arc::new(table_catalog)); + .update(table_catalog.id, table_catalog); } Operation::Delete => { diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index 874f416c81df1..c403917938fb5 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -159,12 +159,12 @@ pub fn update_filter_key_extractor_for_table_ids( table_ids: &[u32], ) { for table_id in table_ids { - let mock_table = Arc::new(PbTable { + let mock_table = PbTable { id: *table_id, read_prefix_len_hint: 0, maybe_vnode_count: Some(VirtualNode::COUNT_FOR_TEST as u32), ..Default::default() - }); + }; compaction_catalog_manager_ref.update(*table_id, mock_table); } } @@ -188,7 +188,7 @@ pub fn update_filter_key_extractor_for_tables( tables: &[PbTable], ) { for table in tables { - compaction_catalog_manager_ref.update(table.id, Arc::new(table.clone())) + compaction_catalog_manager_ref.update(table.id, table.clone()) } } pub async fn register_tables_with_catalog_for_test( diff --git a/src/storage/src/compaction_catalog_manager.rs b/src/storage/src/compaction_catalog_manager.rs index bf183f220fecd..3133cae023300 100644 --- a/src/storage/src/compaction_catalog_manager.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -260,7 +260,7 @@ impl StateTableAccessor for FakeRemoteTableAccessor { /// `CompactionCatalogManager` is a manager to manage all `Table` which used in compaction pub struct CompactionCatalogManager { // `table_id_to_catalog` is a map to store all `Table` which used in compaction - table_id_to_catalog: RwLock>>, + table_id_to_catalog: RwLock>, // `table_accessor` is a accessor to fetch `Table` from meta when the table not found table_accessor: Box, } @@ -282,12 +282,12 @@ impl CompactionCatalogManager { impl CompactionCatalogManager { /// `update` is used to update `Table` in `table_id_to_catalog` from notification - pub fn update(&self, table_id: u32, catalog: Arc
) { + pub fn update(&self, table_id: u32, catalog: Table) { self.table_id_to_catalog.write().insert(table_id, catalog); } /// `sync` is used to sync all `Table` in `table_id_to_catalog` from notification whole snapshot - pub fn sync(&self, catalog_map: HashMap>) { + pub fn sync(&self, catalog_map: HashMap) { let mut guard = self.table_id_to_catalog.write(); guard.clear(); guard.extend(catalog_map); @@ -318,10 +318,8 @@ impl CompactionCatalogManager { let guard = self.table_id_to_catalog.read(); table_ids.retain(|table_id| match guard.get(table_id) { Some(table_catalog) => { - multi_filter_key_extractor.register( - *table_id, - FilterKeyExtractorImpl::from_table(table_catalog.as_ref()), - ); + multi_filter_key_extractor + .register(*table_id, FilterKeyExtractorImpl::from_table(table_catalog)); table_id_to_vnode.insert(*table_id, table_catalog.vnode_count()); false } @@ -348,7 +346,7 @@ impl CompactionCatalogManager { let table_id = table.id; let key_extractor = FilterKeyExtractorImpl::from_table(&table); let vnode = table.vnode_count(); - guard.insert(table_id, Arc::new(table)); + guard.insert(table_id, table); multi_filter_key_extractor.register(table_id, key_extractor); table_id_to_vnode.insert(table_id, vnode); } @@ -363,15 +361,13 @@ impl CompactionCatalogManager { /// `build_compaction_catalog_agent` is used to build `CompactionCatalogAgent` by `table_catalogs` pub fn build_compaction_catalog_agent( - table_catalogs: HashMap>, + table_catalogs: HashMap, ) -> CompactionCatalogAgentRef { let mut multi_filter_key_extractor = MultiFilterKeyExtractor::default(); let mut table_id_to_vnode = HashMap::new(); for (table_id, table_catalog) in table_catalogs { - multi_filter_key_extractor.register( - table_id, - FilterKeyExtractorImpl::from_table(table_catalog.as_ref()), - ); + multi_filter_key_extractor + .register(table_id, FilterKeyExtractorImpl::from_table(&table_catalog)); table_id_to_vnode.insert(table_id, table_catalog.vnode_count()); } diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index ec2bb260d8342..772df4050b66e 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -707,7 +707,7 @@ pub fn start_shared_compactor( task: dispatch_task, } = request.into_inner(); let table_id_to_catalog = tables.into_iter().fold(HashMap::new(), |mut acc, table| { - acc.insert(table.id, Arc::new(table)); + acc.insert(table.id, table); acc }); diff --git a/src/storage/src/hummock/observer_manager.rs b/src/storage/src/hummock/observer_manager.rs index 62a2842fddca9..7ba86a6f08977 100644 --- a/src/storage/src/hummock/observer_manager.rs +++ b/src/storage/src/hummock/observer_manager.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - use risingwave_common_service::ObserverState; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_trace::TraceSpan; @@ -155,14 +153,14 @@ impl HummockObserverNode { fn handle_catalog_snapshot(&mut self, tables: Vec
) { self.compaction_catalog_manager - .sync(tables.into_iter().map(|t| (t.id, Arc::new(t))).collect()); + .sync(tables.into_iter().map(|t| (t.id, t)).collect()); } fn handle_catalog_notification(&mut self, operation: Operation, table_catalog: Table) { match operation { Operation::Add | Operation::Update => { self.compaction_catalog_manager - .update(table_catalog.id, Arc::new(table_catalog)); + .update(table_catalog.id, table_catalog); } Operation::Delete => {