From 224ac394c25f753195eaf395b6274ee8de45c22c Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu, 16 May 2024 14:43:25 +0800 Subject: [PATCH 01/12] fix: bind source do not need pk bug introduced in 16665 (#16776) --- src/frontend/src/handler/create_source.rs | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 1dd5a879354cf..49f3126f53b96 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -503,7 +503,6 @@ pub(crate) async fn bind_columns_from_source( fn bind_columns_from_source_for_cdc( session: &SessionImpl, source_schema: &ConnectorSchema, - _with_properties: &HashMap, ) -> Result<(Option>, StreamSourceInfo)> { let format_encode_options = WithOptions::try_from(source_schema.row_options())?.into_inner(); let mut format_encode_options_to_consume = format_encode_options.clone(); @@ -1391,7 +1390,18 @@ pub async fn bind_create_source( } debug_assert!(is_column_ids_dedup(&columns)); - let (mut columns, pk_col_ids, row_id_index) = bind_pk_on_relation(columns, pk_names, true)?; + let must_need_pk = if is_create_source { + with_properties.connector_need_pk() + } else { + // For those connectors that do not need generate a `row_id`` column in the source schema such as iceberg. + // But in such case, we can not create mv or table on the source because there is not a pk. + assert!(with_properties.connector_need_pk()); + + true + }; + + let (mut columns, pk_col_ids, row_id_index) = + bind_pk_on_relation(columns, pk_names, must_need_pk)?; let watermark_descs = bind_source_watermark(session, source_name.clone(), source_watermarks, &columns)?; @@ -1471,7 +1481,7 @@ pub async fn handle_create_source( || (with_properties.is_kafka_connector() && session.config().rw_enable_shared_source()); let (columns_from_resolve_source, mut source_info) = if create_cdc_source_job { - bind_columns_from_source_for_cdc(&session, &source_schema, &with_properties)? + bind_columns_from_source_for_cdc(&session, &source_schema)? } else { bind_columns_from_source(&session, &source_schema, &with_properties).await? }; From 0febb1a8d9a98be7d76431441aaa394a4ad2c940 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 16 May 2024 16:48:24 +0800 Subject: [PATCH 02/12] fix: list subscription without filtering in streaming job (#16778) --- src/meta/src/controller/catalog.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index f8ec470a74918..27bac7048b967 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -2945,8 +2945,6 @@ impl CatalogControllerInner { async fn list_subscriptions(&self) -> MetaResult> { let subscription_objs = Subscription::find() .find_also_related(Object) - .join(JoinType::LeftJoin, object::Relation::StreamingJob.def()) - .filter(streaming_job::Column::JobStatus.eq(JobStatus::Created)) .all(&self.db) .await?; From 892d9a18560105737d38290991c90ae3cfe1a305 Mon Sep 17 00:00:00 2001 From: Huangjw <1223644280@qq.com> Date: Thu, 16 May 2024 17:55:07 +0800 Subject: [PATCH 03/12] fix: fix name kinesis-s3-source in integration test (#16784) --- integration_tests/kinesis-s3-source/create_mv.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/integration_tests/kinesis-s3-source/create_mv.sql b/integration_tests/kinesis-s3-source/create_mv.sql index cbd7bd958a363..7c1cbe1cdeac7 100644 --- a/integration_tests/kinesis-s3-source/create_mv.sql +++ b/integration_tests/kinesis-s3-source/create_mv.sql @@ -1,4 +1,4 @@ -CREATE MATERIALIZED VIEW ad_ctr_mv AS +CREATE MATERIALIZED VIEW ad_ctr AS SELECT ad_clicks.ad_id AS ad_id, ad_clicks.clicks_count :: NUMERIC / ad_impressions.impressions_count AS ctr @@ -23,7 +23,7 @@ FROM ai.ad_id ) AS ad_clicks ON ad_impressions.ad_id = ad_clicks.ad_id; -CREATE MATERIALIZED VIEW ad_ctr_5min_mv AS +CREATE MATERIALIZED VIEW ad_ctr_5min AS SELECT ac.ad_id AS ad_id, ac.clicks_count :: NUMERIC / ai.impressions_count AS ctr, From 93c221dd0352ccaf5702ed8c46660319b7dfdd76 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Thu, 16 May 2024 18:19:59 +0800 Subject: [PATCH 04/12] refactor(meta): break hummock manager Versioning into smaller structs (#16710) --- src/common/metrics/src/monitor/rwlock.rs | 25 +- src/meta/src/hummock/manager/checkpoint.rs | 22 +- src/meta/src/hummock/manager/compaction.rs | 20 +- .../manager/compaction_group_manager.rs | 67 ++-- src/meta/src/hummock/manager/context.rs | 110 ++++-- src/meta/src/hummock/manager/gc.rs | 69 ++-- src/meta/src/hummock/manager/mod.rs | 338 +++++++++--------- src/meta/src/hummock/manager/tests.rs | 20 +- src/meta/src/hummock/manager/versioning.rs | 105 +++--- src/meta/src/hummock/vacuum.rs | 6 +- src/meta/src/rpc/metrics.rs | 2 +- 11 files changed, 406 insertions(+), 378 deletions(-) diff --git a/src/common/metrics/src/monitor/rwlock.rs b/src/common/metrics/src/monitor/rwlock.rs index 46f9d5edea91c..4d65c53801106 100644 --- a/src/common/metrics/src/monitor/rwlock.rs +++ b/src/common/metrics/src/monitor/rwlock.rs @@ -16,31 +16,34 @@ use prometheus::HistogramVec; use tokio::sync::{RwLock, RwLockReadGuard, RwLockWriteGuard}; pub struct MonitoredRwLock { + // labels: [lock_name, lock_type] metrics: HistogramVec, inner: RwLock, + lock_name: &'static str, } impl MonitoredRwLock { - pub fn new(metrics: HistogramVec, val: T) -> Self { + pub fn new(metrics: HistogramVec, val: T, lock_name: &'static str) -> Self { Self { metrics, inner: RwLock::new(val), + lock_name, } } - pub async fn read<'a, 'b>( - &'a self, - label_values: &'b [&'static str], - ) -> RwLockReadGuard<'a, T> { - let _timer = self.metrics.with_label_values(label_values).start_timer(); + pub async fn read(&self) -> RwLockReadGuard<'_, T> { + let _timer = self + .metrics + .with_label_values(&[self.lock_name, "read"]) + .start_timer(); self.inner.read().await } - pub async fn write<'a, 'b>( - &'a self, - label_values: &'b [&'static str], - ) -> RwLockWriteGuard<'a, T> { - let _timer = self.metrics.with_label_values(label_values).start_timer(); + pub async fn write(&self) -> RwLockWriteGuard<'_, T> { + let _timer = self + .metrics + .with_label_values(&[self.lock_name, "write"]) + .start_timer(); self.inner.write().await } } diff --git a/src/meta/src/hummock/manager/checkpoint.rs b/src/meta/src/hummock/manager/checkpoint.rs index d20df896f0a1a..70bbef6bd3db2 100644 --- a/src/meta/src/hummock/manager/checkpoint.rs +++ b/src/meta/src/hummock/manager/checkpoint.rs @@ -17,7 +17,6 @@ use std::ops::Bound::{Excluded, Included}; use std::ops::{Deref, DerefMut}; use std::sync::atomic::Ordering; -use function_name::named; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ object_size_map, summarize_group_deltas, }; @@ -32,7 +31,6 @@ use tracing::warn; use crate::hummock::error::Result; use crate::hummock::manager::versioning::Versioning; -use crate::hummock::manager::{read_lock, write_lock}; use crate::hummock::metrics_utils::{trigger_gc_stat, trigger_split_stat}; use crate::hummock::HummockManager; @@ -122,11 +120,10 @@ impl HummockManager { /// Returns the diff between new and old checkpoint id. /// Note that this method must not be called concurrently, because internally it doesn't hold /// lock throughout the method. - #[named] pub async fn create_version_checkpoint(&self, min_delta_log_num: u64) -> Result { let timer = self.metrics.version_checkpoint_latency.start_timer(); // 1. hold read lock and create new checkpoint - let versioning_guard = read_lock!(self, versioning).await; + let versioning_guard = self.versioning.read().await; let versioning: &Versioning = versioning_guard.deref(); let current_version: &HummockVersion = &versioning.current_version; let old_checkpoint: &HummockVersionCheckpoint = &versioning.checkpoint; @@ -137,9 +134,10 @@ impl HummockManager { } if cfg!(test) && new_checkpoint_id == old_checkpoint_id { drop(versioning_guard); - let mut versioning = write_lock!(self, versioning).await; - versioning.mark_objects_for_deletion(); - let min_pinned_version_id = versioning.min_pinned_version_id(); + let versioning = self.versioning.read().await; + let context_info = self.context_info.read().await; + versioning.mark_objects_for_deletion(&context_info, &self.delete_object_tracker); + let min_pinned_version_id = context_info.min_pinned_version_id(); trigger_gc_stat(&self.metrics, &versioning.checkpoint, min_pinned_version_id); return Ok(0); } @@ -224,16 +222,17 @@ impl HummockManager { } } // 3. hold write lock and update in memory state - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); + let context_info = self.context_info.read().await; assert!(new_checkpoint.version.id > versioning.checkpoint.version.id); versioning.checkpoint = new_checkpoint; // Not delete stale objects when archive is enabled if !self.env.opts.enable_hummock_data_archive { - versioning.mark_objects_for_deletion(); + versioning.mark_objects_for_deletion(&context_info, &self.delete_object_tracker); } - let min_pinned_version_id = versioning.min_pinned_version_id(); + let min_pinned_version_id = context_info.min_pinned_version_id(); trigger_gc_stat(&self.metrics, &versioning.checkpoint, min_pinned_version_id); trigger_split_stat(&self.metrics, &versioning.current_version); drop(versioning_guard); @@ -260,9 +259,8 @@ impl HummockManager { self.pause_version_checkpoint.load(Ordering::Relaxed) } - #[named] pub async fn get_checkpoint_version(&self) -> HummockVersion { - let versioning_guard = read_lock!(self, versioning).await; + let versioning_guard = self.versioning.read().await; versioning_guard.checkpoint.version.clone() } } diff --git a/src/meta/src/hummock/manager/compaction.rs b/src/meta/src/hummock/manager/compaction.rs index ba9954607ab73..369799a8d7b04 100644 --- a/src/meta/src/hummock/manager/compaction.rs +++ b/src/meta/src/hummock/manager/compaction.rs @@ -15,7 +15,6 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use std::sync::Arc; -use function_name::named; use futures::future::Shared; use itertools::Itertools; use risingwave_hummock_sdk::{CompactionGroupId, HummockCompactionTaskId}; @@ -37,7 +36,7 @@ use tokio::sync::oneshot::Receiver as OneShotReceiver; use crate::hummock::compaction::selector::level_selector::PickerInfo; use crate::hummock::compaction::selector::DynamicLevelSelectorCore; use crate::hummock::compaction::{CompactStatus, CompactionDeveloperConfig, CompactionSelector}; -use crate::hummock::manager::{init_selectors, read_lock}; +use crate::hummock::manager::init_selectors; use crate::hummock::HummockManager; const MAX_SKIP_TIMES: usize = 8; @@ -54,17 +53,12 @@ pub struct Compaction { } impl HummockManager { - #[named] pub async fn get_assigned_compact_task_num(&self) -> u64 { - read_lock!(self, compaction) - .await - .compact_task_assignment - .len() as u64 + self.compaction.read().await.compact_task_assignment.len() as u64 } - #[named] pub async fn list_all_tasks_ids(&self) -> Vec { - let compaction = read_lock!(self, compaction).await; + let compaction = self.compaction.read().await; compaction .compaction_statuses @@ -77,11 +71,10 @@ impl HummockManager { .collect_vec() } - #[named] pub async fn list_compaction_status( &self, ) -> (Vec, Vec) { - let compaction = read_lock!(self, compaction).await; + let compaction = self.compaction.read().await; ( compaction.compaction_statuses.values().map_into().collect(), compaction @@ -92,14 +85,13 @@ impl HummockManager { ) } - #[named] pub async fn get_compaction_scores( &self, compaction_group_id: CompactionGroupId, ) -> Vec { let (status, levels, group) = { - let compaction = read_lock!(self, compaction).await; - let versioning = read_lock!(self, versioning).await; + let compaction = self.compaction.read().await; + let versioning = self.versioning.read().await; let config_manager = self.compaction_group_manager.read().await; match ( compaction.compaction_statuses.get(&compaction_group_id), diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index 2a9b4eca28fea..c98700792a881 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -16,7 +16,6 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use std::ops::DerefMut; use std::sync::Arc; -use function_name::named; use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ @@ -31,19 +30,19 @@ use risingwave_pb::hummock::group_delta::DeltaType; use risingwave_pb::hummock::hummock_version_delta::GroupDeltas; use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config::MutableConfig; use risingwave_pb::hummock::subscribe_compaction_event_request::ReportTask; +use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ compact_task, CompactionConfig, CompactionGroupInfo, CompatibilityVersion, GroupConstruct, GroupDelta, GroupDestroy, GroupMetaChange, }; use thiserror_ext::AsReport; -use tokio::sync::{OnceCell, RwLock}; +use tokio::sync::OnceCell; -use super::write_lock; use crate::hummock::compaction::compaction_config::{ validate_compaction_config, CompactionConfigBuilder, }; use crate::hummock::error::{Error, Result}; -use crate::hummock::manager::{commit_multi_var, create_trx_wrapper, read_lock, HummockManager}; +use crate::hummock::manager::{commit_multi_var, create_trx_wrapper, HummockManager}; use crate::hummock::metrics_utils::remove_compaction_group_in_sst_stat; use crate::hummock::model::CompactionGroup; use crate::hummock::sequence::{next_compaction_group_id, next_sstable_object_id}; @@ -57,7 +56,7 @@ use crate::storage::MetaStore; impl HummockManager { pub(super) async fn build_compaction_group_manager( env: &MetaSrvEnv, - ) -> Result> { + ) -> Result { let default_config = match env.opts.compaction_config.as_ref() { None => CompactionConfigBuilder::new().build(), Some(opt) => CompactionConfigBuilder::with_opt(opt).build(), @@ -68,21 +67,21 @@ impl HummockManager { pub(super) async fn build_compaction_group_manager_with_config( env: &MetaSrvEnv, default_config: CompactionConfig, - ) -> Result> { - let compaction_group_manager = RwLock::new(CompactionGroupManager { + ) -> Result { + let mut compaction_group_manager = CompactionGroupManager { compaction_groups: BTreeMap::new(), default_config, + write_limit: Default::default(), meta_store_impl: env.meta_store_ref(), - }); - compaction_group_manager.write().await.init().await?; + }; + compaction_group_manager.init().await?; Ok(compaction_group_manager) } /// Should not be called inside [`HummockManager`], because it requests locks internally. /// The implementation acquires `versioning` lock. - #[named] pub async fn compaction_group_ids(&self) -> Vec { - get_compaction_group_ids(&read_lock!(self, versioning).await.current_version).collect_vec() + get_compaction_group_ids(&self.versioning.read().await.current_version).collect_vec() } /// The implementation acquires `compaction_group_manager` lock. @@ -141,10 +140,9 @@ impl HummockManager { /// Unregisters stale members and groups /// The caller should ensure `table_fragments_list` remain unchanged during `purge`. /// Currently `purge` is only called during meta service start ups. - #[named] pub async fn purge(&self, valid_ids: &[u32]) -> Result<()> { let registered_members = - get_member_table_ids(&read_lock!(self, versioning).await.current_version); + get_member_table_ids(&self.versioning.read().await.current_version); let to_unregister = registered_members .into_iter() .filter(|table_id| !valid_ids.contains(table_id)) @@ -155,7 +153,6 @@ impl HummockManager { } /// The implementation acquires `versioning` lock. - #[named] pub async fn register_table_ids( &self, pairs: &[(StateTableId, CompactionGroupId)], @@ -163,7 +160,7 @@ impl HummockManager { if pairs.is_empty() { return Ok(()); } - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); let current_version = &versioning.current_version; @@ -250,12 +247,11 @@ impl HummockManager { Ok(()) } - #[named] pub async fn unregister_table_ids(&self, table_ids: &[StateTableId]) -> Result<()> { if table_ids.is_empty() { return Ok(()); } - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); let current_version = &versioning.current_version; let mut new_version_delta = create_trx_wrapper!( @@ -383,9 +379,8 @@ impl HummockManager { /// Gets complete compaction group info. /// It is the aggregate of `HummockVersion` and `CompactionGroupConfig` - #[named] pub async fn list_compaction_group(&self) -> Vec { - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); let current_version = &versioning.current_version; let mut compaction_groups = vec![]; @@ -431,8 +426,7 @@ impl HummockManager { /// move some table to another compaction-group. Create a new compaction group if it does not /// exist. - /// TODO: Move table_to_partition in result to compaction group - #[named] + /// TODO: Move `table_to_partition` in result to compaction group pub async fn move_state_table_to_compaction_group( &self, parent_group_id: CompactionGroupId, @@ -444,8 +438,8 @@ impl HummockManager { return Ok((parent_group_id, table_to_partition)); } let table_ids = table_ids.iter().cloned().unique().collect_vec(); - let compaction_guard = write_lock!(self, compaction).await; - let mut versioning_guard = write_lock!(self, versioning).await; + let compaction_guard = self.compaction.write().await; + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); let current_version = &versioning.current_version; // Validate parameters. @@ -604,11 +598,10 @@ impl HummockManager { Ok((target_compaction_group_id, table_to_partition)) } - #[named] pub async fn calculate_compaction_group_statistic(&self) -> Vec { let mut infos = vec![]; { - let versioning_guard = read_lock!(self, versioning).await; + let versioning_guard = self.versioning.read().await; let version = &versioning_guard.current_version; for (group_id, group) in &version.levels { let mut group_info = TableGroupInfo { @@ -649,6 +642,8 @@ impl HummockManager { pub(super) struct CompactionGroupManager { compaction_groups: BTreeMap, default_config: CompactionConfig, + /// Tables that write limit is trigger for. + pub write_limit: HashMap, meta_store_impl: MetaStoreImpl, } @@ -871,38 +866,30 @@ mod tests { #[tokio::test] async fn test_inner() { let (env, ..) = setup_compute_env(8080).await; - let inner = HummockManager::build_compaction_group_manager(&env) + let mut inner = HummockManager::build_compaction_group_manager(&env) .await .unwrap(); - assert_eq!(inner.read().await.compaction_groups.len(), 2); + assert_eq!(inner.compaction_groups.len(), 2); inner - .write() - .await .update_compaction_config(&[100, 200], &[]) .await .unwrap_err(); inner - .write() - .await .get_or_insert_compaction_group_configs(&[100, 200]) .await .unwrap(); - assert_eq!(inner.read().await.compaction_groups.len(), 4); - let inner = HummockManager::build_compaction_group_manager(&env) + assert_eq!(inner.compaction_groups.len(), 4); + let mut inner = HummockManager::build_compaction_group_manager(&env) .await .unwrap(); - assert_eq!(inner.read().await.compaction_groups.len(), 4); + assert_eq!(inner.compaction_groups.len(), 4); inner - .write() - .await .update_compaction_config(&[100, 200], &[MutableConfig::MaxSubCompaction(123)]) .await .unwrap(); - assert_eq!(inner.read().await.compaction_groups.len(), 4); + assert_eq!(inner.compaction_groups.len(), 4); assert_eq!( inner - .read() - .await .try_get_compaction_group_config(100) .unwrap() .compaction_config @@ -911,8 +898,6 @@ mod tests { ); assert_eq!( inner - .read() - .await .try_get_compaction_group_config(200) .unwrap() .compaction_config diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index 21aeb8402838b..a388f9d472137 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -12,35 +12,41 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet}; -use std::ops::DerefMut; +use std::collections::{BTreeMap, HashMap, HashSet}; use fail::fail_point; -use function_name::named; use itertools::Itertools; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ - ExtendedSstableInfo, HummockContextId, HummockEpoch, HummockSstableObjectId, + ExtendedSstableInfo, HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, }; -use risingwave_pb::hummock::ValidationTask; +use risingwave_pb::hummock::{HummockPinnedSnapshot, HummockPinnedVersion, ValidationTask}; use crate::hummock::error::{Error, Result}; use crate::hummock::manager::{ - commit_multi_var, create_trx_wrapper, read_lock, start_measure_real_process_timer, write_lock, + commit_multi_var, create_trx_wrapper, start_measure_real_process_timer, }; use crate::hummock::HummockManager; -use crate::manager::META_NODE_ID; +use crate::manager::{MetaStoreImpl, MetadataManager, META_NODE_ID}; use crate::model::{BTreeMapTransaction, BTreeMapTransactionWrapper, ValTransaction}; use crate::storage::MetaStore; -impl HummockManager { +#[derive(Default)] +pub(super) struct ContextInfo { + pub pinned_versions: BTreeMap, + pub pinned_snapshots: BTreeMap, + /// `version_safe_points` is similar to `pinned_versions` expect for being a transient state. + pub version_safe_points: Vec, +} + +impl ContextInfo { /// Release resources pinned by these contexts, including: /// - Version /// - Snapshot - #[named] - pub async fn release_contexts( - &self, + async fn release_contexts( + &mut self, context_ids: impl AsRef<[HummockContextId]>, + meta_store_ref: MetaStoreImpl, ) -> Result<()> { fail_point!("release_contexts_metastore_err", |_| Err(Error::MetaStore( anyhow::anyhow!("failpoint metastore error") @@ -49,50 +55,77 @@ impl HummockManager { anyhow::anyhow!("failpoint internal error") ))); - let mut versioning_guard = write_lock!(self, versioning).await; - let versioning = versioning_guard.deref_mut(); let mut pinned_versions = create_trx_wrapper!( - self.meta_store_ref(), + meta_store_ref, BTreeMapTransactionWrapper, - BTreeMapTransaction::new(&mut versioning.pinned_versions,) + BTreeMapTransaction::new(&mut self.pinned_versions,) ); let mut pinned_snapshots = create_trx_wrapper!( - self.meta_store_ref(), + meta_store_ref, BTreeMapTransactionWrapper, - BTreeMapTransaction::new(&mut versioning.pinned_snapshots,) + BTreeMapTransaction::new(&mut self.pinned_snapshots,) ); for context_id in context_ids.as_ref() { pinned_versions.remove(*context_id); pinned_snapshots.remove(*context_id); } - commit_multi_var!(self.meta_store_ref(), pinned_versions, pinned_snapshots)?; + commit_multi_var!(meta_store_ref, pinned_versions, pinned_snapshots)?; + Ok(()) + } +} + +impl HummockManager { + pub async fn release_contexts( + &self, + context_ids: impl AsRef<[HummockContextId]>, + ) -> Result<()> { + let mut context_info = self.context_info.write().await; + context_info + .release_contexts(context_ids, self.meta_store_ref()) + .await?; #[cfg(test)] { - drop(versioning_guard); + drop(context_info); self.check_state_consistency().await; } - Ok(()) } /// Checks whether `context_id` is valid. pub async fn check_context(&self, context_id: HummockContextId) -> Result { - Ok(self - .metadata_manager() + self.context_info + .read() + .await + .check_context(context_id, &self.metadata_manager) + .await + } +} + +impl ContextInfo { + /// Checks whether `context_id` is valid. + /// + /// Need `&self` to sync with `release_context` + pub(super) async fn check_context( + &self, + context_id: HummockContextId, + metadata_manager: &MetadataManager, + ) -> Result { + Ok(metadata_manager .get_worker_by_id(context_id) .await .map_err(|err| Error::MetaStore(err.into()))? .is_some()) } +} +impl HummockManager { /// Release invalid contexts, aka worker node ids which are no longer valid in `ClusterManager`. - #[named] pub(super) async fn release_invalid_contexts(&self) -> Result> { - let active_context_ids = { - let compaction_guard = read_lock!(self, compaction).await; - let versioning_guard = read_lock!(self, versioning).await; - let _timer = start_measure_real_process_timer!(self); + let (active_context_ids, mut context_info) = { + let compaction_guard = self.compaction.read().await; + let context_info = self.context_info.write().await; + let _timer = start_measure_real_process_timer!(self, "release_invalid_contexts"); let mut active_context_ids = HashSet::new(); active_context_ids.extend( compaction_guard @@ -100,19 +133,24 @@ impl HummockManager { .values() .map(|c| c.context_id), ); - active_context_ids.extend(versioning_guard.pinned_versions.keys()); - active_context_ids.extend(versioning_guard.pinned_snapshots.keys()); - active_context_ids + active_context_ids.extend(context_info.pinned_versions.keys()); + active_context_ids.extend(context_info.pinned_snapshots.keys()); + (active_context_ids, context_info) }; let mut invalid_context_ids = vec![]; for active_context_id in &active_context_ids { - if !self.check_context(*active_context_id).await? { + if !context_info + .check_context(*active_context_id, &self.metadata_manager) + .await? + { invalid_context_ids.push(*active_context_id); } } - self.release_contexts(&invalid_context_ids).await?; + context_info + .release_contexts(&invalid_context_ids, self.meta_store_ref()) + .await?; Ok(invalid_context_ids) } @@ -133,7 +171,13 @@ impl HummockManager { continue; } } - if !self.check_context(*context_id).await? { + if !self + .context_info + .read() + .await + .check_context(*context_id, &self.metadata_manager) + .await? + { return Err(Error::InvalidSst(*sst_id)); } } diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index dafd3231afcce..54f5e71203889 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -19,9 +19,9 @@ use std::ops::DerefMut; use std::time::Duration; use anyhow::Context; -use function_name::named; use futures::{stream, StreamExt}; use itertools::Itertools; +use parking_lot::Mutex; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_pb::common::worker_node::State::Running; use risingwave_pb::common::WorkerType; @@ -29,32 +29,54 @@ use risingwave_pb::hummock::subscribe_compaction_event_response::Event as Respon use risingwave_pb::hummock::FullScanTask; use crate::hummock::error::{Error, Result}; -use crate::hummock::manager::versioning::Versioning; -use crate::hummock::manager::{commit_multi_var, create_trx_wrapper, read_lock, write_lock}; +use crate::hummock::manager::{commit_multi_var, create_trx_wrapper}; use crate::hummock::HummockManager; use crate::manager::MetadataManager; use crate::model::{BTreeMapTransaction, BTreeMapTransactionWrapper, ValTransaction}; use crate::storage::MetaStore; +#[derive(Default)] +pub(super) struct DeleteObjectTracker { + /// Objects that waits to be deleted from object store. It comes from either compaction, or + /// full GC (listing object store). + objects_to_delete: Mutex>, +} + +impl DeleteObjectTracker { + pub(super) fn add(&self, objects: impl Iterator) { + self.objects_to_delete.lock().extend(objects) + } + + pub(super) fn current(&self) -> HashSet { + self.objects_to_delete.lock().clone() + } + + pub(super) fn clear(&self) { + self.objects_to_delete.lock().clear(); + } + + pub(super) fn ack<'a>(&self, objects: impl Iterator) { + let mut lock = self.objects_to_delete.lock(); + for object in objects { + lock.remove(object); + } + } +} + impl HummockManager { /// Gets SST objects that is safe to be deleted from object store. - #[named] - pub async fn get_objects_to_delete(&self) -> Vec { - read_lock!(self, versioning) - .await - .objects_to_delete + pub fn get_objects_to_delete(&self) -> Vec { + self.delete_object_tracker + .current() .iter() .cloned() .collect_vec() } /// Acknowledges SSTs have been deleted from object store. - #[named] pub async fn ack_deleted_objects(&self, object_ids: &[HummockSstableObjectId]) -> Result<()> { - let mut versioning_guard = write_lock!(self, versioning).await; - for object_id in object_ids { - versioning_guard.objects_to_delete.remove(object_id); - } + self.delete_object_tracker.ack(object_ids.iter()); + let mut versioning_guard = self.versioning.write().await; for stale_objects in versioning_guard.checkpoint.stale_objects.values_mut() { stale_objects.id.retain(|id| !object_ids.contains(id)); } @@ -69,10 +91,10 @@ impl HummockManager { /// Deletes at most `batch_size` deltas. /// /// Returns (number of deleted deltas, number of remain `deltas_to_delete`). - #[named] pub async fn delete_version_deltas(&self, batch_size: usize) -> Result<(usize, usize)> { - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); + let context_info = self.context_info.read().await; let deltas_to_delete = versioning .hummock_version_deltas .range(..=versioning.checkpoint.version.id) @@ -80,7 +102,7 @@ impl HummockManager { .collect_vec(); // If there is any safe point, skip this to ensure meta backup has required delta logs to // replay version. - if !versioning.version_safe_points.is_empty() { + if !context_info.version_safe_points.is_empty() { return Ok((0, deltas_to_delete.len())); } let mut hummock_version_deltas = create_trx_wrapper!( @@ -102,6 +124,7 @@ impl HummockManager { commit_multi_var!(self.meta_store_ref(), hummock_version_deltas)?; #[cfg(test)] { + drop(context_info); drop(versioning_guard); self.check_state_consistency().await; } @@ -110,16 +133,15 @@ impl HummockManager { /// Extends `objects_to_delete` according to object store full scan result. /// Caller should ensure `object_ids` doesn't include any SST objects belong to a on-going - /// version write. That's to say, these object_ids won't appear in either `commit_epoch` or + /// version write. That's to say, these `object_ids` won't appear in either `commit_epoch` or /// `report_compact_task`. - #[named] pub async fn extend_objects_to_delete_from_scan( &self, object_ids: &[HummockSstableObjectId], ) -> usize { let tracked_object_ids: HashSet = { - let versioning_guard = read_lock!(self, versioning).await; - let versioning: &Versioning = &versioning_guard; + let versioning = self.versioning.read().await; + let context_info = self.context_info.read().await; // object ids in checkpoint version let mut tracked_object_ids = versioning.checkpoint.version.get_object_ids(); @@ -131,7 +153,7 @@ impl HummockManager { tracked_object_ids.extend(delta.newly_added_object_ids()); } // add stale object ids before the checkpoint version - let min_pinned_version_id = versioning.min_pinned_version_id(); + let min_pinned_version_id = context_info.min_pinned_version_id(); tracked_object_ids.extend( versioning .checkpoint @@ -147,9 +169,8 @@ impl HummockManager { .iter() .filter(|object_id| !tracked_object_ids.contains(object_id)) .collect_vec(); - let mut versioning_guard = write_lock!(self, versioning).await; - versioning_guard.objects_to_delete.extend(to_delete.clone()); - drop(versioning_guard); + self.delete_object_tracker + .add(to_delete.iter().map(|id| **id)); to_delete.len() } diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 261b58ce0116b..d3d91fa70c59a 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::borrow::BorrowMut; use std::collections::{BTreeMap, HashMap, HashSet, VecDeque}; use std::ops::{Deref, DerefMut}; use std::sync::atomic::AtomicBool; @@ -23,7 +22,6 @@ use anyhow::Context; use arc_swap::ArcSwap; use bytes::Bytes; use fail::fail_point; -use function_name::named; use futures::future::Either; use futures::stream::{BoxStream, FuturesUnordered}; use futures::{FutureExt, StreamExt}; @@ -68,7 +66,6 @@ use rw_futures_util::{pending_on_none, select_all}; use thiserror_ext::AsReport; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; use tokio::sync::oneshot::Sender; -use tokio::sync::RwLockWriteGuard; use tokio::task::JoinHandle; use tokio_stream::wrappers::IntervalStream; use tonic::Streaming; @@ -129,13 +126,14 @@ pub struct HummockManager { pub env: MetaSrvEnv, metadata_manager: MetadataManager, - /// Lock order: compaction, versioning, `compaction_group_manager`. - /// - Lock compaction first, then versioning, and finally `compaction_group_manager`. + /// Lock order: `compaction`, `versioning`, `compaction_group_manager`, `context_info` + /// - Lock `compaction` first, then `versioning`, then `compaction_group_manager` and finally `context_info`. /// - This order should be strictly followed to prevent deadlock. compaction: MonitoredRwLock, versioning: MonitoredRwLock, /// `CompactionGroupManager` manages compaction configs for compaction groups. - compaction_group_manager: tokio::sync::RwLock, + compaction_group_manager: MonitoredRwLock, + context_info: MonitoredRwLock, latest_snapshot: Snapshot, pub metrics: Arc, @@ -143,6 +141,8 @@ pub struct HummockManager { pub compactor_manager: CompactorManagerRef, event_sender: HummockManagerEventSender, + delete_object_tracker: DeleteObjectTracker, + object_store: ObjectStoreRef, version_checkpoint_path: String, version_archive_dir: String, @@ -174,21 +174,6 @@ pub struct HummockManager { pub type HummockManagerRef = Arc; -/// Acquire read lock of the lock with `lock_name`. -/// The macro will use macro `function_name` to get the name of the function of method that calls -/// the lock, and therefore, anyone to call this macro should ensured that the caller method has the -/// macro #[named] -macro_rules! read_lock { - ($hummock_mgr:expr, $lock_name:ident) => { - async { - $hummock_mgr - .$lock_name - .read(&[function_name!(), stringify!($lock_name), "read"]) - .await - } - }; -} -pub(crate) use read_lock; use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; use risingwave_hummock_sdk::table_stats::{ add_prost_table_stats_map, purge_prost_table_stats, PbTableStatsMap, @@ -198,28 +183,12 @@ use risingwave_pb::catalog::Table; use risingwave_pb::hummock::level_handler::RunningCompactTask; use risingwave_pb::meta::relation::RelationInfo; -/// Acquire write lock of the lock with `lock_name`. -/// The macro will use macro `function_name` to get the name of the function of method that calls -/// the lock, and therefore, anyone to call this macro should ensured that the caller method has the -/// macro #[named] -macro_rules! write_lock { - ($hummock_mgr:expr, $lock_name:ident) => { - async { - $hummock_mgr - .$lock_name - .write(&[function_name!(), stringify!($lock_name), "write"]) - .await - } - }; -} -pub(crate) use write_lock; - macro_rules! start_measure_real_process_timer { - ($hummock_mgr:expr) => { + ($hummock_mgr:expr, $func_name:literal) => { $hummock_mgr .metrics .hummock_manager_real_process_time - .with_label_values(&[function_name!()]) + .with_label_values(&[$func_name]) .start_timer() }; } @@ -359,7 +328,7 @@ impl HummockManager { metadata_manager: MetadataManager, metrics: Arc, compactor_manager: CompactorManagerRef, - compaction_group_manager: tokio::sync::RwLock, + compaction_group_manager: CompactionGroupManager, compactor_streams_change_tx: UnboundedSender<( u32, Streaming, @@ -415,14 +384,25 @@ impl HummockManager { versioning: MonitoredRwLock::new( metrics.hummock_manager_lock_time.clone(), Default::default(), + "hummock_manager::versioning", ), compaction: MonitoredRwLock::new( metrics.hummock_manager_lock_time.clone(), Default::default(), + "hummock_manager::compaction", + ), + compaction_group_manager: MonitoredRwLock::new( + metrics.hummock_manager_lock_time.clone(), + compaction_group_manager, + "hummock_manager::compaction_group_manager", + ), + context_info: MonitoredRwLock::new( + metrics.hummock_manager_lock_time.clone(), + Default::default(), + "hummock_manager::context_info", ), metrics, metadata_manager, - compaction_group_manager, // compaction_request_channel: parking_lot::RwLock::new(None), compactor_manager, latest_snapshot: ArcSwap::from_pointee(HummockSnapshot { @@ -430,6 +410,7 @@ impl HummockManager { current_epoch: INVALID_EPOCH, }), event_sender: tx, + delete_object_tracker: Default::default(), object_store, version_checkpoint_path, version_archive_dir, @@ -453,13 +434,14 @@ impl HummockManager { } /// Load state from meta store. - #[named] async fn load_meta_store_state(&self) -> Result<()> { - let mut compaction_guard = write_lock!(self, compaction).await; - let mut versioning_guard = write_lock!(self, versioning).await; + let mut compaction_guard = self.compaction.write().await; + let mut versioning_guard = self.versioning.write().await; + let mut context_info_guard = self.context_info.write().await; self.load_meta_store_state_impl( - compaction_guard.borrow_mut(), - versioning_guard.borrow_mut(), + &mut compaction_guard, + &mut versioning_guard, + &mut context_info_guard, ) .await } @@ -467,8 +449,9 @@ impl HummockManager { /// Load state from meta store. async fn load_meta_store_state_impl( &self, - compaction_guard: &mut RwLockWriteGuard<'_, Compaction>, - versioning_guard: &mut RwLockWriteGuard<'_, Versioning>, + compaction_guard: &mut Compaction, + versioning_guard: &mut Versioning, + context_info: &mut ContextInfo, ) -> Result<()> { use sea_orm::EntityTrait; let meta_store = self.meta_store_ref(); @@ -582,7 +565,7 @@ impl HummockManager { versioning_guard.current_version = redo_state; versioning_guard.hummock_version_deltas = hummock_version_deltas; - versioning_guard.pinned_versions = match &meta_store { + context_info.pinned_versions = match &meta_store { MetaStoreImpl::Kv(meta_store) => HummockPinnedVersion::list(meta_store) .await? .into_iter() @@ -597,7 +580,7 @@ impl HummockManager { .collect(), }; - versioning_guard.pinned_snapshots = match &meta_store { + context_info.pinned_snapshots = match &meta_store { MetaStoreImpl::Kv(meta_store) => HummockPinnedSnapshot::list(meta_store) .await? .into_iter() @@ -612,23 +595,32 @@ impl HummockManager { .collect(), }; - versioning_guard.objects_to_delete.clear(); + self.delete_object_tracker.clear(); // Not delete stale objects when archive is enabled if !self.env.opts.enable_hummock_data_archive { - versioning_guard.mark_objects_for_deletion(); + versioning_guard.mark_objects_for_deletion(context_info, &self.delete_object_tracker); } - self.initial_compaction_group_config_after_load(versioning_guard) - .await?; + self.initial_compaction_group_config_after_load( + versioning_guard, + self.compaction_group_manager.write().await.deref_mut(), + ) + .await?; Ok(()) } - /// Caller should hold `versioning` lock, to sync with `HummockManager::release_contexts`. - async fn check_context_with_meta_node(&self, context_id: HummockContextId) -> Result<()> { + async fn check_context_with_meta_node( + &self, + context_id: HummockContextId, + context_info: &ContextInfo, + ) -> Result<()> { if context_id == META_NODE_ID { // Using the preserved meta id is allowed. - } else if !self.check_context(context_id).await? { + } else if !context_info + .check_context(context_id, &self.metadata_manager) + .await? + { // The worker is not found in cluster. return Err(Error::InvalidContext(context_id)); } @@ -637,16 +629,16 @@ impl HummockManager { /// Pin the current greatest hummock version. The pin belongs to `context_id` /// and will be unpinned when `context_id` is invalidated. - #[named] pub async fn pin_version(&self, context_id: HummockContextId) -> Result { - let mut versioning_guard = write_lock!(self, versioning).await; - self.check_context_with_meta_node(context_id).await?; - let _timer = start_measure_real_process_timer!(self); - let versioning = versioning_guard.deref_mut(); + let versioning = self.versioning.read().await; + let mut context_info = self.context_info.write().await; + self.check_context_with_meta_node(context_id, &context_info) + .await?; + let _timer = start_measure_real_process_timer!(self, "pin_version"); let mut pinned_versions = create_trx_wrapper!( self.meta_store_ref(), BTreeMapTransactionWrapper, - BTreeMapTransaction::new(&mut versioning.pinned_versions,) + BTreeMapTransaction::new(&mut context_info.pinned_versions) ); let mut context_pinned_version = pinned_versions.new_entry_txn_or_default( context_id, @@ -662,12 +654,13 @@ impl HummockManager { { context_pinned_version.min_pinned_id = version_id; commit_multi_var!(self.meta_store_ref(), context_pinned_version)?; - trigger_pin_unpin_version_state(&self.metrics, &versioning.pinned_versions); + trigger_pin_unpin_version_state(&self.metrics, &context_info.pinned_versions); } #[cfg(test)] { - drop(versioning_guard); + drop(context_info); + drop(versioning); self.check_state_consistency().await; } @@ -677,20 +670,19 @@ impl HummockManager { /// Unpin all pins which belongs to `context_id` and has an id which is older than /// `unpin_before`. All versions >= `unpin_before` will be treated as if they are all pinned by /// this `context_id` so they will not be vacuumed. - #[named] pub async fn unpin_version_before( &self, context_id: HummockContextId, unpin_before: HummockVersionId, ) -> Result<()> { - let mut versioning_guard = write_lock!(self, versioning).await; - self.check_context_with_meta_node(context_id).await?; - let _timer = start_measure_real_process_timer!(self); - let versioning = versioning_guard.deref_mut(); + let mut context_info = self.context_info.write().await; + self.check_context_with_meta_node(context_id, &context_info) + .await?; + let _timer = start_measure_real_process_timer!(self, "unpin_version_before"); let mut pinned_versions = create_trx_wrapper!( self.meta_store_ref(), BTreeMapTransactionWrapper, - BTreeMapTransaction::new(&mut versioning.pinned_versions,) + BTreeMapTransaction::new(&mut context_info.pinned_versions,) ); let mut context_pinned_version = pinned_versions.new_entry_txn_or_default( context_id, @@ -707,26 +699,26 @@ impl HummockManager { ); context_pinned_version.min_pinned_id = unpin_before; commit_multi_var!(self.meta_store_ref(), context_pinned_version)?; - trigger_pin_unpin_version_state(&self.metrics, &versioning.pinned_versions); + trigger_pin_unpin_version_state(&self.metrics, &context_info.pinned_versions); #[cfg(test)] { - drop(versioning_guard); + drop(context_info); self.check_state_consistency().await; } Ok(()) } - #[named] pub async fn pin_specific_snapshot( &self, context_id: HummockContextId, epoch: HummockEpoch, ) -> Result { let snapshot = self.latest_snapshot.load(); - let mut guard = write_lock!(self, versioning).await; - self.check_context_with_meta_node(context_id).await?; + let mut guard = self.context_info.write().await; + self.check_context_with_meta_node(context_id, &guard) + .await?; let mut pinned_snapshots = create_trx_wrapper!( self.meta_store_ref(), BTreeMapTransactionWrapper, @@ -748,12 +740,12 @@ impl HummockManager { } /// Make sure `max_committed_epoch` is pinned and return it. - #[named] pub async fn pin_snapshot(&self, context_id: HummockContextId) -> Result { let snapshot = self.latest_snapshot.load(); - let mut guard = write_lock!(self, versioning).await; - self.check_context_with_meta_node(context_id).await?; - let _timer = start_measure_real_process_timer!(self); + let mut guard = self.context_info.write().await; + self.check_context_with_meta_node(context_id, &guard) + .await?; + let _timer = start_measure_real_process_timer!(self, "pin_snapshot"); let mut pinned_snapshots = create_trx_wrapper!( self.meta_store_ref(), BTreeMapTransactionWrapper, @@ -779,25 +771,25 @@ impl HummockManager { HummockSnapshot::clone(&snapshot) } - #[named] pub async fn unpin_snapshot(&self, context_id: HummockContextId) -> Result<()> { - let mut versioning_guard = write_lock!(self, versioning).await; - self.check_context_with_meta_node(context_id).await?; - let _timer = start_measure_real_process_timer!(self); + let mut context_info = self.context_info.write().await; + self.check_context_with_meta_node(context_id, &context_info) + .await?; + let _timer = start_measure_real_process_timer!(self, "unpin_snapshot"); let mut pinned_snapshots = create_trx_wrapper!( self.meta_store_ref(), BTreeMapTransactionWrapper, - BTreeMapTransaction::new(&mut versioning_guard.pinned_snapshots,) + BTreeMapTransaction::new(&mut context_info.pinned_snapshots,) ); let release_snapshot = pinned_snapshots.remove(context_id); if release_snapshot.is_some() { commit_multi_var!(self.meta_store_ref(), pinned_snapshots)?; - trigger_pin_unpin_snapshot_state(&self.metrics, &versioning_guard.pinned_snapshots); + trigger_pin_unpin_snapshot_state(&self.metrics, &context_info.pinned_snapshots); } #[cfg(test)] { - drop(versioning_guard); + drop(context_info); self.check_state_consistency().await; } @@ -805,18 +797,19 @@ impl HummockManager { } /// Unpin all snapshots smaller than specified epoch for current context. - #[named] pub async fn unpin_snapshot_before( &self, context_id: HummockContextId, hummock_snapshot: HummockSnapshot, ) -> Result<()> { - let mut versioning_guard = write_lock!(self, versioning).await; - self.check_context_with_meta_node(context_id).await?; - let _timer = start_measure_real_process_timer!(self); + let versioning = self.versioning.read().await; + let mut context_info = self.context_info.write().await; + self.check_context_with_meta_node(context_id, &context_info) + .await?; + let _timer = start_measure_real_process_timer!(self, "unpin_snapshot_before"); // Use the max_committed_epoch in storage as the snapshot ts so only committed changes are // visible in the snapshot. - let max_committed_epoch = versioning_guard.current_version.max_committed_epoch; + let max_committed_epoch = versioning.current_version.max_committed_epoch; // Ensure the unpin will not clean the latest one. let snapshot_committed_epoch = hummock_snapshot.committed_epoch; #[cfg(not(test))] @@ -828,7 +821,7 @@ impl HummockManager { let mut pinned_snapshots = create_trx_wrapper!( self.meta_store_ref(), BTreeMapTransactionWrapper, - BTreeMapTransaction::new(&mut versioning_guard.pinned_snapshots,) + BTreeMapTransaction::new(&mut context_info.pinned_snapshots,) ); let mut context_pinned_snapshot = pinned_snapshots.new_entry_txn_or_default( context_id, @@ -845,19 +838,19 @@ impl HummockManager { { context_pinned_snapshot.minimal_pinned_snapshot = last_read_epoch; commit_multi_var!(self.meta_store_ref(), context_pinned_snapshot)?; - trigger_pin_unpin_snapshot_state(&self.metrics, &versioning_guard.pinned_snapshots); + trigger_pin_unpin_snapshot_state(&self.metrics, &context_info.pinned_snapshots); } #[cfg(test)] { - drop(versioning_guard); + drop(context_info); + drop(versioning); self.check_state_consistency().await; } Ok(()) } - #[named] pub async fn get_compact_tasks_impl( &self, compaction_groups: Vec, @@ -874,17 +867,20 @@ impl HummockManager { .await .map_err(|err| Error::MetaStore(err.into()))?; - let mut compaction_guard = write_lock!(self, compaction).await; + let mut compaction_guard = self.compaction.write().await; let compaction = compaction_guard.deref_mut(); - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); - let _timer = start_measure_real_process_timer!(self); + let _timer = start_measure_real_process_timer!(self, "get_compact_tasks_impl"); let mut current_version = versioning.current_version.clone(); let start_time = Instant::now(); let max_committed_epoch = current_version.max_committed_epoch; - let watermark = versioning + let watermark = self + .context_info + .read() + .await .pinned_snapshots .values() .map(|v| v.minimal_pinned_snapshot) @@ -1407,9 +1403,8 @@ impl HummockManager { /// /// Return Ok(false) indicates either the task is not found, /// or the task is not owned by `context_id` when `context_id` is not None. - #[named] pub async fn report_compact_tasks(&self, report_tasks: Vec) -> Result> { - let mut guard = write_lock!(self, compaction).await; + let mut guard = self.compaction.write().await; let deterministic_mode = self.env.opts.compaction_deterministic_test; let compaction = guard.deref_mut(); let start_time = Instant::now(); @@ -1426,9 +1421,9 @@ impl HummockManager { BTreeMapTransaction::new(&mut compaction.compact_task_assignment,) ); // The compaction task is finished. - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); - let _timer = start_measure_real_process_timer!(self); + let _timer = start_measure_real_process_timer!(self, "report_compact_tasks"); let mut current_version = versioning.current_version.clone(); // purge stale compact_status @@ -1625,7 +1620,6 @@ impl HummockManager { } /// Caller should ensure `epoch` > `max_committed_epoch` - #[named] pub async fn commit_epoch( &self, epoch: HummockEpoch, @@ -1638,8 +1632,8 @@ impl HummockManager { new_table_fragment_info, change_log_delta, } = commit_info; - let mut versioning_guard = write_lock!(self, versioning).await; - let _timer = start_measure_real_process_timer!(self); + let mut versioning_guard = self.versioning.write().await; + let _timer = start_measure_real_process_timer!(self, "commit_epoch"); // Prevent commit new epochs if this flag is set if versioning_guard.disable_commit_epochs { return Ok(None); @@ -1980,87 +1974,94 @@ impl HummockManager { Ok(SstObjectIdRange::new(start_id, start_id + number as u64)) } - #[named] pub async fn get_min_pinned_version_id(&self) -> HummockVersionId { - read_lock!(self, versioning).await.min_pinned_version_id() + self.context_info.read().await.min_pinned_version_id() } - #[named] #[cfg(test)] pub async fn check_state_consistency(&self) { - let mut compaction_guard = write_lock!(self, compaction).await; - let mut versioning_guard = write_lock!(self, versioning).await; - let objects_to_delete = versioning_guard.objects_to_delete.clone(); + let mut compaction_guard = self.compaction.write().await; + let mut versioning_guard = self.versioning.write().await; + let mut context_info_guard = self.context_info.write().await; + let objects_to_delete = self.delete_object_tracker.current(); // We don't check `checkpoint` because it's allowed to update its in memory state without // persisting to object store. - let get_state = - |compaction_guard: &RwLockWriteGuard<'_, Compaction>, - versioning_guard: &RwLockWriteGuard<'_, Versioning>| { - let compact_statuses_copy = compaction_guard.compaction_statuses.clone(); - let compact_task_assignment_copy = compaction_guard.compact_task_assignment.clone(); - let pinned_versions_copy = versioning_guard.pinned_versions.clone(); - let pinned_snapshots_copy = versioning_guard.pinned_snapshots.clone(); - let hummock_version_deltas_copy = versioning_guard.hummock_version_deltas.clone(); - let version_stats_copy = versioning_guard.version_stats.clone(); - (( - compact_statuses_copy, - compact_task_assignment_copy, - pinned_versions_copy, - pinned_snapshots_copy, - hummock_version_deltas_copy, - version_stats_copy, - ),) - }; - let mem_state = get_state(&compaction_guard, &versioning_guard); + let get_state = |compaction_guard: &mut Compaction, + versioning_guard: &mut Versioning, + context_info_guard: &mut ContextInfo| { + let compact_statuses_copy = compaction_guard.compaction_statuses.clone(); + let compact_task_assignment_copy = compaction_guard.compact_task_assignment.clone(); + let pinned_versions_copy = context_info_guard.pinned_versions.clone(); + let pinned_snapshots_copy = context_info_guard.pinned_snapshots.clone(); + let hummock_version_deltas_copy = versioning_guard.hummock_version_deltas.clone(); + let version_stats_copy = versioning_guard.version_stats.clone(); + (( + compact_statuses_copy, + compact_task_assignment_copy, + pinned_versions_copy, + pinned_snapshots_copy, + hummock_version_deltas_copy, + version_stats_copy, + ),) + }; + let mem_state = get_state( + &mut compaction_guard, + &mut versioning_guard, + &mut context_info_guard, + ); self.load_meta_store_state_impl( - compaction_guard.borrow_mut(), - versioning_guard.borrow_mut(), + &mut compaction_guard, + &mut versioning_guard, + &mut context_info_guard, ) .await .expect("Failed to load state from meta store"); - let loaded_state = get_state(&compaction_guard, &versioning_guard); + let loaded_state = get_state( + &mut compaction_guard, + &mut versioning_guard, + &mut context_info_guard, + ); assert_eq!( mem_state, loaded_state, "hummock in-mem state is inconsistent with meta store state", ); - versioning_guard.objects_to_delete = objects_to_delete; + self.delete_object_tracker.clear(); + self.delete_object_tracker + .add(objects_to_delete.into_iter()); } /// Gets current version without pinning it. /// Should not be called inside [`HummockManager`], because it requests locks internally. /// /// Note: this method can hurt performance because it will clone a large object. - #[named] pub async fn get_current_version(&self) -> HummockVersion { - read_lock!(self, versioning).await.current_version.clone() + self.versioning.read().await.current_version.clone() } - #[named] pub async fn get_current_max_committed_epoch(&self) -> HummockEpoch { - read_lock!(self, versioning) + self.versioning + .read() .await .current_version .max_committed_epoch } - #[named] /// Gets the mapping from table id to compaction group id pub async fn get_table_compaction_group_id_mapping( &self, ) -> HashMap { - get_table_compaction_group_id_mapping(&read_lock!(self, versioning).await.current_version) + get_table_compaction_group_id_mapping(&self.versioning.read().await.current_version) } /// Get version deltas from meta store #[cfg_attr(coverage, coverage(off))] - #[named] pub async fn list_version_deltas( &self, start_id: u64, num_limit: u32, committed_epoch_limit: HummockEpoch, ) -> Result> { - let versioning = read_lock!(self, versioning).await; + let versioning = self.versioning.read().await; let version_deltas = versioning .hummock_version_deltas .range(start_id..) @@ -2127,12 +2128,11 @@ impl HummockManager { /// Replay a version delta to current hummock version. /// Returns the `version_id`, `max_committed_epoch` of the new version and the modified /// compaction groups - #[named] pub async fn replay_version_delta( &self, mut version_delta: HummockVersionDelta, ) -> Result<(HummockVersion, Vec)> { - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; // ensure the version id is ascending after replay version_delta.id = versioning_guard.current_version.id + 1; version_delta.prev_id = version_delta.id - 1; @@ -2145,9 +2145,8 @@ impl HummockManager { Ok((version_new, compaction_group_ids)) } - #[named] pub async fn disable_commit_epoch(&self) -> HummockVersion { - let mut versioning_guard = write_lock!(self, versioning).await; + let mut versioning_guard = self.versioning.write().await; versioning_guard.disable_commit_epochs = true; versioning_guard.current_version.clone() } @@ -2271,18 +2270,16 @@ impl HummockManager { } #[cfg(any(test, feature = "test"))] - #[named] pub async fn compaction_task_from_assignment_for_test( &self, task_id: u64, ) -> Option { - let compaction_guard = read_lock!(self, compaction).await; + let compaction_guard = self.compaction.read().await; let assignment_ref = &compaction_guard.compact_task_assignment; assignment_ref.get(&task_id).cloned() } #[cfg(any(test, feature = "test"))] - #[named] pub async fn report_compact_task_for_test( &self, task_id: u64, @@ -2292,7 +2289,7 @@ impl HummockManager { table_stats_change: Option, ) -> Result<()> { if let Some(task) = compact_task { - let mut guard = write_lock!(self, compaction).await; + let mut guard = self.compaction.write().await; guard.compact_task_assignment.insert( task_id, CompactTaskAssignment { @@ -2357,10 +2354,7 @@ impl HummockManager { .notify_frontend_without_version(Operation::Update, Info::HummockStats(stats.clone())); } - #[named] pub fn hummock_timer_task(hummock_manager: Arc) -> (JoinHandle<()>, Sender<()>) { - use futures::{FutureExt, StreamExt}; - let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); let join_handle = tokio::spawn(async move { const CHECK_PENDING_TASK_PERIOD_SEC: u64 = 300; @@ -2525,7 +2519,7 @@ impl HummockManager { HummockTimerEvent::Report => { let (current_version, id_to_config, version_stats) = { let versioning_guard = - read_lock!(hummock_manager.as_ref(), versioning).await; + hummock_manager.versioning.read().await; let configs = hummock_manager.get_compaction_group_map().await; @@ -2669,12 +2663,11 @@ impl HummockManager { (join_handle, shutdown_tx) } - #[named] pub async fn check_dead_task(&self) { const MAX_COMPACTION_L0_MULTIPLIER: u64 = 32; const MAX_COMPACTION_DURATION_SEC: u64 = 20 * 60; let (groups, configs) = { - let versioning_guard = read_lock!(self, versioning).await; + let versioning_guard = self.versioning.read().await; let g = versioning_guard .current_version .levels @@ -2713,7 +2706,7 @@ impl HummockManager { } let mut pending_tasks: HashMap = HashMap::default(); { - let compaction_guard = read_lock!(self, compaction).await; + let compaction_guard = self.compaction.read().await; for group_id in slowdown_groups.keys() { if let Some(status) = compaction_guard.compaction_statuses.get(group_id) { for (idx, level_handler) in status.level_handlers.iter().enumerate() { @@ -3192,15 +3185,13 @@ impl HummockManager { async fn initial_compaction_group_config_after_load( &self, - versioning_guard: &mut RwLockWriteGuard<'_, Versioning>, + versioning_guard: &Versioning, + compaction_group_manager: &mut CompactionGroupManager, ) -> Result<()> { // 1. Due to version compatibility, we fix some of the configuration of older versions after hummock starts. let current_version = &versioning_guard.current_version; let all_group_ids = get_compaction_group_ids(current_version); - let mut configs = self - .compaction_group_manager - .write() - .await + let mut configs = compaction_group_manager .get_or_insert_compaction_group_configs(&all_group_ids.collect_vec()) .await?; @@ -3240,10 +3231,7 @@ impl HummockManager { tracing::info!("Compaction group {:?} configs rewrite ", rewrite_cg_ids); // update meta store - let result = self - .compaction_group_manager - .write() - .await + let result = compaction_group_manager .update_compaction_config( &rewrite_cg_ids, &[ @@ -3260,10 +3248,13 @@ impl HummockManager { } } - versioning_guard.write_limit = + compaction_group_manager.write_limit = calc_new_write_limits(configs, HashMap::new(), &versioning_guard.current_version); - trigger_write_stop_stats(&self.metrics, &versioning_guard.write_limit); - tracing::debug!("Hummock stopped write: {:#?}", versioning_guard.write_limit); + trigger_write_stop_stats(&self.metrics, &compaction_group_manager.write_limit); + tracing::debug!( + "Hummock stopped write: {:#?}", + compaction_group_manager.write_limit + ); { // 2. Restore the memory data structure according to the memory of the compaction group config. @@ -3276,14 +3267,13 @@ impl HummockManager { Ok(()) } - #[named] pub async fn list_change_log_epochs( &self, table_id: u32, min_epoch: u64, max_count: u32, ) -> Vec { - let versioning = read_lock!(self, versioning).await; + let versioning = self.versioning.read().await; if let Some(table_change_log) = versioning .current_version .table_change_log @@ -3443,6 +3433,8 @@ use tokio::sync::mpsc::error::SendError; use super::compaction::CompactionSelector; use crate::hummock::manager::checkpoint::HummockVersionCheckpoint; +use crate::hummock::manager::context::ContextInfo; +use crate::hummock::manager::gc::DeleteObjectTracker; use crate::hummock::sequence::next_sstable_object_id; #[derive(Debug, Default)] diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 3af055574e18a..160f9518f648e 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -588,7 +588,7 @@ async fn test_hummock_manager_basic() { ); } // objects_to_delete is always empty because no compaction is ever invoked. - assert!(hummock_manager.get_objects_to_delete().await.is_empty()); + assert!(hummock_manager.get_objects_to_delete().is_empty()); assert_eq!( hummock_manager .delete_version_deltas(usize::MAX) @@ -600,7 +600,7 @@ async fn test_hummock_manager_basic() { hummock_manager.create_version_checkpoint(1).await.unwrap(), commit_log_count + register_log_count ); - assert!(hummock_manager.get_objects_to_delete().await.is_empty()); + assert!(hummock_manager.get_objects_to_delete().is_empty()); assert_eq!( hummock_manager .delete_version_deltas(usize::MAX) @@ -1125,7 +1125,7 @@ async fn test_extend_objects_to_delete() { .map(|s| s.get_object_id()) .chain(max_committed_object_id + 1..=max_committed_object_id + orphan_sst_num) .collect_vec(); - assert!(hummock_manager.get_objects_to_delete().await.is_empty()); + assert!(hummock_manager.get_objects_to_delete().is_empty()); assert_eq!( hummock_manager .extend_objects_to_delete_from_scan(&all_object_ids) @@ -1133,7 +1133,7 @@ async fn test_extend_objects_to_delete() { orphan_sst_num as usize ); assert_eq!( - hummock_manager.get_objects_to_delete().await.len(), + hummock_manager.get_objects_to_delete().len(), orphan_sst_num as usize ); @@ -1143,7 +1143,7 @@ async fn test_extend_objects_to_delete() { 6 ); assert_eq!( - hummock_manager.get_objects_to_delete().await.len(), + hummock_manager.get_objects_to_delete().len(), orphan_sst_num as usize ); // since version1 is still pinned, the sst removed in compaction can not be reclaimed. @@ -1153,10 +1153,10 @@ async fn test_extend_objects_to_delete() { .await, orphan_sst_num as usize ); - let objects_to_delete = hummock_manager.get_objects_to_delete().await; + let objects_to_delete = hummock_manager.get_objects_to_delete(); assert_eq!(objects_to_delete.len(), orphan_sst_num as usize); let pinned_version2: HummockVersion = hummock_manager.pin_version(context_id).await.unwrap(); - let objects_to_delete = hummock_manager.get_objects_to_delete().await; + let objects_to_delete = hummock_manager.get_objects_to_delete(); assert_eq!( objects_to_delete.len(), orphan_sst_num as usize, @@ -1167,7 +1167,7 @@ async fn test_extend_objects_to_delete() { .unpin_version_before(context_id, pinned_version2.id) .await .unwrap(); - let objects_to_delete = hummock_manager.get_objects_to_delete().await; + let objects_to_delete = hummock_manager.get_objects_to_delete(); assert_eq!( objects_to_delete.len(), orphan_sst_num as usize, @@ -1182,7 +1182,7 @@ async fn test_extend_objects_to_delete() { .await, orphan_sst_num as usize ); - let objects_to_delete = hummock_manager.get_objects_to_delete().await; + let objects_to_delete = hummock_manager.get_objects_to_delete(); assert_eq!(objects_to_delete.len(), orphan_sst_num as usize); let new_epoch = pinned_version2.max_committed_epoch.next_epoch(); hummock_manager @@ -1206,7 +1206,7 @@ async fn test_extend_objects_to_delete() { .await, orphan_sst_num as usize + 3 ); - let objects_to_delete = hummock_manager.get_objects_to_delete().await; + let objects_to_delete = hummock_manager.get_objects_to_delete(); assert_eq!(objects_to_delete.len(), orphan_sst_num as usize + 3); } diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index 7b99c77fd881e..464404b8f25c0 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -15,7 +15,6 @@ use std::cmp; use std::collections::{BTreeMap, HashMap, HashSet}; -use function_name::named; use itertools::Itertools; use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ @@ -38,8 +37,10 @@ use risingwave_pb::meta::subscribe_response::{Info, Operation}; use super::check_cg_write_limit; use crate::hummock::error::Result; use crate::hummock::manager::checkpoint::HummockVersionCheckpoint; +use crate::hummock::manager::context::ContextInfo; +use crate::hummock::manager::gc::DeleteObjectTracker; use crate::hummock::manager::worker::{HummockManagerEvent, HummockManagerEventSender}; -use crate::hummock::manager::{commit_multi_var, create_trx_wrapper, read_lock, write_lock}; +use crate::hummock::manager::{commit_multi_var, create_trx_wrapper}; use crate::hummock::metrics_utils::{ trigger_safepoint_stat, trigger_write_stop_stats, LocalTableMetrics, }; @@ -71,30 +72,21 @@ impl Drop for HummockVersionSafePoint { #[derive(Default)] pub struct Versioning { // Volatile states below - /// Avoide commit epoch epochs + /// Avoid commit epoch epochs /// Don't persist compaction version delta to meta store pub disable_commit_epochs: bool, /// Latest hummock version pub current_version: HummockVersion, - /// Objects that waits to be deleted from object store. It comes from either compaction, or - /// full GC (listing object store). - pub objects_to_delete: HashSet, - /// `version_safe_points` is similar to `pinned_versions` expect for being a transient state. - pub version_safe_points: Vec, - /// Tables that write limit is trigger for. - pub write_limit: HashMap, + pub local_metrics: HashMap, // Persistent states below pub hummock_version_deltas: BTreeMap, - pub pinned_versions: BTreeMap, - pub pinned_snapshots: BTreeMap, /// Stats for latest hummock version. pub version_stats: HummockVersionStats, pub checkpoint: HummockVersionCheckpoint, - pub local_metrics: HashMap, } -impl Versioning { +impl ContextInfo { pub fn min_pinned_version_id(&self) -> HummockVersionId { let mut min_pinned_version_id = HummockVersionId::MAX; for id in self @@ -107,24 +99,30 @@ impl Versioning { } min_pinned_version_id } +} +impl Versioning { /// Marks all objects <= `min_pinned_version_id` for deletion. - pub(super) fn mark_objects_for_deletion(&mut self) { - let min_pinned_version_id = self.min_pinned_version_id(); - self.objects_to_delete.extend( + pub(super) fn mark_objects_for_deletion( + &self, + context_info: &ContextInfo, + delete_object_tracker: &DeleteObjectTracker, + ) { + let min_pinned_version_id = context_info.min_pinned_version_id(); + delete_object_tracker.add( self.checkpoint .stale_objects .iter() .filter(|(version_id, _)| **version_id <= min_pinned_version_id) - .flat_map(|(_, stale_objects)| stale_objects.id.clone()), + .flat_map(|(_, stale_objects)| stale_objects.id.iter().cloned()), ); } } impl HummockManager { - #[named] pub async fn list_pinned_version(&self) -> Vec { - read_lock!(self, versioning) + self.context_info + .read() .await .pinned_versions .values() @@ -132,9 +130,9 @@ impl HummockManager { .collect_vec() } - #[named] pub async fn list_pinned_snapshot(&self) -> Vec { - read_lock!(self, versioning) + self.context_info + .read() .await .pinned_snapshots .values() @@ -159,16 +157,15 @@ impl HummockManager { Ok(workers) } - #[named] pub async fn get_version_stats(&self) -> HummockVersionStats { - read_lock!(self, versioning).await.version_stats.clone() + self.versioning.read().await.version_stats.clone() } - #[named] pub async fn register_safe_point(&self) -> HummockVersionSafePoint { - let mut wl = write_lock!(self, versioning).await; + let versioning = self.versioning.read().await; + let mut wl = self.context_info.write().await; let safe_point = HummockVersionSafePoint { - id: wl.current_version.id, + id: versioning.current_version.id, event_sender: self.event_sender.clone(), }; wl.version_safe_points.push(safe_point.id); @@ -176,9 +173,8 @@ impl HummockManager { safe_point } - #[named] pub async fn unregister_safe_point(&self, safe_point: HummockVersionId) { - let mut wl = write_lock!(self, versioning).await; + let mut wl = self.context_info.write().await; let version_safe_points = &mut wl.version_safe_points; if let Some(pos) = version_safe_points.iter().position(|sp| *sp == safe_point) { version_safe_points.remove(pos); @@ -189,41 +185,40 @@ impl HummockManager { /// Updates write limits for `target_groups` and sends notification. /// Returns true if `write_limit` has been modified. /// The implementation acquires `versioning` lock and `compaction_group_manager` lock. - #[named] pub(super) async fn try_update_write_limits( &self, target_group_ids: &[CompactionGroupId], ) -> bool { - let mut guard = write_lock!(self, versioning).await; - let config_mgr = self.compaction_group_manager.read().await; + let versioning = self.versioning.read().await; + let mut cg_manager = self.compaction_group_manager.write().await; let target_group_configs = target_group_ids .iter() .filter_map(|id| { - config_mgr + cg_manager .try_get_compaction_group_config(*id) .map(|config| (*id, config)) }) .collect(); let mut new_write_limits = calc_new_write_limits( target_group_configs, - guard.write_limit.clone(), - &guard.current_version, + cg_manager.write_limit.clone(), + &versioning.current_version, ); let all_group_ids: HashSet<_> = - HashSet::from_iter(get_compaction_group_ids(&guard.current_version)); + HashSet::from_iter(get_compaction_group_ids(&versioning.current_version)); new_write_limits.retain(|group_id, _| all_group_ids.contains(group_id)); - if new_write_limits == guard.write_limit { + if new_write_limits == cg_manager.write_limit { return false; } tracing::debug!("Hummock stopped write is updated: {:#?}", new_write_limits); trigger_write_stop_stats(&self.metrics, &new_write_limits); - guard.write_limit = new_write_limits; + cg_manager.write_limit = new_write_limits; self.env .notification_manager() .notify_hummock_without_version( Operation::Add, Info::HummockWriteLimits(risingwave_pb::hummock::WriteLimits { - write_limits: guard.write_limit.clone(), + write_limits: cg_manager.write_limit.clone(), }), ); true @@ -231,22 +226,19 @@ impl HummockManager { /// Gets write limits. /// The implementation acquires `versioning` lock. - #[named] pub async fn write_limits(&self) -> HashMap { - let guard = read_lock!(self, versioning).await; + let guard = self.compaction_group_manager.read().await; guard.write_limit.clone() } - #[named] pub async fn list_branched_objects(&self) -> BTreeMap { - let guard = read_lock!(self, versioning).await; + let guard = self.versioning.read().await; guard.current_version.build_branched_sst_info() } - #[named] pub async fn rebuild_table_stats(&self) -> Result<()> { use crate::model::ValTransaction; - let mut versioning = write_lock!(self, versioning).await; + let mut versioning = self.versioning.write().await; let new_stats = rebuild_table_stats(&versioning.current_version); let mut version_stats = create_trx_wrapper!( self.meta_store_ref(), @@ -370,29 +362,30 @@ mod tests { }; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; + use crate::hummock::manager::context::ContextInfo; use crate::hummock::manager::versioning::{ - calc_new_write_limits, estimate_table_stats, rebuild_table_stats, Versioning, + calc_new_write_limits, estimate_table_stats, rebuild_table_stats, }; use crate::hummock::model::CompactionGroup; #[test] fn test_min_pinned_version_id() { - let mut versioning = Versioning::default(); - assert_eq!(versioning.min_pinned_version_id(), HummockVersionId::MAX); - versioning.pinned_versions.insert( + let mut context_info = ContextInfo::default(); + assert_eq!(context_info.min_pinned_version_id(), HummockVersionId::MAX); + context_info.pinned_versions.insert( 1, HummockPinnedVersion { context_id: 1, min_pinned_id: 10, }, ); - assert_eq!(versioning.min_pinned_version_id(), 10); - versioning.version_safe_points.push(5); - assert_eq!(versioning.min_pinned_version_id(), 5); - versioning.version_safe_points.clear(); - assert_eq!(versioning.min_pinned_version_id(), 10); - versioning.pinned_versions.clear(); - assert_eq!(versioning.min_pinned_version_id(), HummockVersionId::MAX); + assert_eq!(context_info.min_pinned_version_id(), 10); + context_info.version_safe_points.push(5); + assert_eq!(context_info.min_pinned_version_id(), 5); + context_info.version_safe_points.clear(); + assert_eq!(context_info.min_pinned_version_id(), 10); + context_info.pinned_versions.clear(); + assert_eq!(context_info.min_pinned_version_id(), HummockVersionId::MAX); } #[test] diff --git a/src/meta/src/hummock/vacuum.rs b/src/meta/src/hummock/vacuum.rs index 9e45fa5a2bcc9..6cde13507836b 100644 --- a/src/meta/src/hummock/vacuum.rs +++ b/src/meta/src/hummock/vacuum.rs @@ -95,7 +95,7 @@ impl VacuumManager { pending_object_ids } else { // 2. If no pending SST objects, then fetch new ones. - let mut objects_to_delete = self.hummock_manager.get_objects_to_delete().await; + let mut objects_to_delete = self.hummock_manager.get_objects_to_delete(); self.filter_out_pinned_ssts(&mut objects_to_delete).await?; if objects_to_delete.is_empty() { return Ok(vec![]); @@ -237,13 +237,13 @@ mod tests { assert_eq!(vacuum.vacuum_metadata().await.unwrap(), 6); assert_eq!(vacuum.vacuum_metadata().await.unwrap(), 0); - assert!(hummock_manager.get_objects_to_delete().await.is_empty()); + assert!(hummock_manager.get_objects_to_delete().is_empty()); hummock_manager .unpin_version_before(context_id, HummockVersionId::MAX) .await .unwrap(); hummock_manager.create_version_checkpoint(0).await.unwrap(); - assert!(!hummock_manager.get_objects_to_delete().await.is_empty()); + assert!(!hummock_manager.get_objects_to_delete().is_empty()); // No SST deletion is scheduled because no available worker. assert_eq!(vacuum.vacuum_object().await.unwrap().len(), 0); let _receiver = compactor_manager.add_compactor(context_id); diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index 2df847516c4a8..28520720e98fe 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -449,7 +449,7 @@ impl MetaMetrics { let hummock_manager_lock_time = register_histogram_vec_with_registry!( "hummock_manager_lock_time", "latency for hummock manager to acquire the rwlock", - &["method", "lock_name", "lock_type"], + &["lock_name", "lock_type"], registry ) .unwrap(); From 8c37e4ebaeac039c90f18c0bbe93fa9f8d7f30c8 Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Thu, 16 May 2024 18:23:25 +0800 Subject: [PATCH 05/12] fix(compactor): fix unneccessary compactor meta grpc stream re-creation (#16783) --- src/storage/src/hummock/compactor/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index be9a3ffad0385..6b17250938a65 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -510,7 +510,7 @@ pub fn start_compactor( &request_sender, ); - continue 'start_stream; + continue 'consume_stream; } running_task_parallelism From 8a98b856f4e9440d8ee6be0fffdda770001ffbc6 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Thu, 16 May 2024 18:31:15 +0800 Subject: [PATCH 06/12] feat(storage): only update related read version on flush finish (#16725) --- src/common/src/config.rs | 1 + .../src/hummock_read_version_tests.rs | 12 +- .../event_handler/hummock_event_handler.rs | 352 +++------ src/storage/src/hummock/event_handler/mod.rs | 1 + .../src/hummock/event_handler/uploader.rs | 698 +++--------------- .../shared_buffer/shared_buffer_batch.rs | 3 +- src/storage/src/hummock/store/version.rs | 204 ++--- src/storage/src/opts.rs | 3 - 8 files changed, 273 insertions(+), 1001 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 41a4e1415b301..fd457c7faf1ca 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -620,6 +620,7 @@ pub struct StorageConfig { /// The threshold for the number of immutable memtables to merge to a new imm. #[serde(default = "default::storage::imm_merge_threshold")] + #[deprecated] pub imm_merge_threshold: usize, /// Whether to enable write conflict detection diff --git a/src/storage/hummock_test/src/hummock_read_version_tests.rs b/src/storage/hummock_test/src/hummock_read_version_tests.rs index 993ad88d0bf33..2329970b485f5 100644 --- a/src/storage/hummock_test/src/hummock_read_version_tests.rs +++ b/src/storage/hummock_test/src/hummock_read_version_tests.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::Bound; use std::sync::Arc; @@ -26,6 +27,7 @@ use risingwave_hummock_sdk::key::{key_with_epoch, map_table_key_range}; use risingwave_hummock_sdk::LocalSstableInfo; use risingwave_meta::hummock::test_utils::setup_compute_env; use risingwave_pb::hummock::{KeyRange, SstableInfo}; +use risingwave_storage::hummock::event_handler::TEST_LOCAL_INSTANCE_ID; use risingwave_storage::hummock::iterator::test_utils::{ iterator_test_table_key_of, iterator_test_user_key_of, }; @@ -48,7 +50,12 @@ async fn test_read_version_basic() { let mut epoch = test_epoch(1); let table_id = 0; let vnodes = Arc::new(Bitmap::ones(VirtualNode::COUNT)); - let mut read_version = HummockReadVersion::new(TableId::from(table_id), pinned_version, vnodes); + let mut read_version = HummockReadVersion::new( + TableId::from(table_id), + TEST_LOCAL_INSTANCE_ID, + pinned_version, + vnodes, + ); { // single imm @@ -178,7 +185,7 @@ async fn test_read_version_basic() { ], vec![], epoch_id_vec_for_clear, - batch_id_vec_for_clear, + HashMap::from_iter([(TEST_LOCAL_INSTANCE_ID, batch_id_vec_for_clear)]), 1, )); @@ -267,6 +274,7 @@ async fn test_read_filter_basic() { let vnodes = Arc::new(Bitmap::ones(VirtualNode::COUNT)); let read_version = Arc::new(RwLock::new(HummockReadVersion::new( TableId::from(table_id), + TEST_LOCAL_INSTANCE_ID, pinned_version, vnodes.clone(), ))); 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 18c34f4c25440..c04ad40ae7ed8 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -12,12 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap}; -use std::ops::DerefMut; +use std::collections::{BTreeMap, HashMap, HashSet, VecDeque}; use std::pin::pin; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::Relaxed; use std::sync::{Arc, LazyLock}; +use std::time::Duration; use arc_swap::ArcSwap; use await_tree::InstrumentAwait; @@ -43,8 +43,8 @@ use crate::hummock::compactor::{await_tree_key, compact, CompactorContext}; use crate::hummock::conflict_detector::ConflictDetector; use crate::hummock::event_handler::refiller::{CacheRefillerEvent, SpawnRefillTask}; use crate::hummock::event_handler::uploader::{ - default_spawn_merging_task, HummockUploader, SpawnMergingTask, SpawnUploadTask, SyncedData, - UploadTaskInfo, UploadTaskOutput, UploadTaskPayload, UploaderEvent, + HummockUploader, SpawnUploadTask, SyncedData, UploadTaskInfo, UploadTaskOutput, + UploadTaskPayload, UploaderEvent, }; use crate::hummock::event_handler::{ HummockEvent, HummockReadVersionRef, HummockVersionUpdate, ReadOnlyReadVersionMapping, @@ -258,7 +258,10 @@ impl HummockEventHandler { let future = async move { let _timer = upload_task_latency.start_timer(); let mut output = flush_imms( - payload, + payload + .values() + .flat_map(|imms| imms.iter().cloned()) + .collect(), task_info, upload_compactor_context.clone(), filter_key_extractor_manager.clone(), @@ -281,10 +284,6 @@ impl HummockEventHandler { } }) }), - default_spawn_merging_task( - compactor_context.compaction_executor.clone(), - compactor_context.await_tree_reg.clone(), - ), CacheRefiller::default_spawn_refill_task(), ) } @@ -297,7 +296,6 @@ impl HummockEventHandler { state_store_metrics: Arc, storage_opts: &StorageOpts, spawn_upload_task: SpawnUploadTask, - spawn_merging_task: SpawnMergingTask, spawn_refill_task: SpawnRefillTask, ) -> Self { let (hummock_event_tx, hummock_event_rx) = @@ -331,7 +329,6 @@ impl HummockEventHandler { state_store_metrics, pinned_version.clone(), spawn_upload_task, - spawn_merging_task, buffer_tracker, storage_opts, ); @@ -388,20 +385,29 @@ impl HummockEventHandler { newly_uploaded_sstables: Vec, ) { debug!("epoch has been synced: {}.", epoch); + let newly_uploaded_sstables = newly_uploaded_sstables + .into_iter() + .map(Arc::new) + .collect_vec(); if !newly_uploaded_sstables.is_empty() { - newly_uploaded_sstables - .into_iter() - // Take rev because newer data come first in `newly_uploaded_sstables` but we apply - // older data first - .rev() - .for_each(|staging_sstable_info| { - let staging_sstable_info_ref = Arc::new(staging_sstable_info); - self.for_each_read_version(|read_version| { - read_version.update(VersionUpdate::Staging(StagingData::Sst( - staging_sstable_info_ref.clone(), - ))) + let related_instance_ids: HashSet<_> = newly_uploaded_sstables + .iter() + .flat_map(|sst| sst.imm_ids().keys().cloned()) + .collect(); + self.for_each_read_version(related_instance_ids, |instance_id, read_version| { + newly_uploaded_sstables + .iter() + // Take rev because newer data come first in `newly_uploaded_sstables` but we apply + // older data first + .rev() + .for_each(|staging_sstable_info| { + if staging_sstable_info.imm_ids().contains_key(&instance_id) { + read_version.update(VersionUpdate::Staging(StagingData::Sst( + staging_sstable_info.clone(), + ))); + } }); - }); + }); } let result = self .uploader @@ -434,21 +440,74 @@ impl HummockEventHandler { /// This function will be performed under the protection of the `read_version_mapping` read /// lock, and add write lock on each `read_version` operation - fn for_each_read_version(&self, mut f: impl FnMut(&mut HummockReadVersion)) { - self.local_read_version_mapping - .values() - .for_each(|read_version: &HummockReadVersionRef| f(read_version.write().deref_mut())); + fn for_each_read_version( + &self, + instances: impl IntoIterator, + mut f: impl FnMut(LocalInstanceId, &mut HummockReadVersion), + ) { + let instances = { + #[cfg(debug_assertion)] + { + // check duplication on debug_mode + use std::collections::HashSet; + let mut id_set = HashSet::new(); + for instance in instances { + assert!(id_set.insert(instance)); + } + id_set + } + #[cfg(not(debug_assertion))] + { + instances + } + }; + let mut pending = VecDeque::new(); + let mut total_count = 0; + for instance_id in instances { + let Some(read_version) = self.local_read_version_mapping.get(&instance_id) else { + continue; + }; + total_count += 1; + if let Some(mut write_guard) = read_version.try_write() { + f(instance_id, &mut write_guard); + } else { + pending.push_back(instance_id); + } + } + if !pending.is_empty() { + warn!( + pending_count = pending.len(), + total_count, "cannot acquire lock for all read version" + ); + } + + const TRY_LOCK_TIMEOUT: Duration = Duration::from_millis(1); + + while let Some(instance_id) = pending.pop_front() { + let read_version = self + .local_read_version_mapping + .get(&instance_id) + .expect("have checked exist before"); + if let Some(mut write_guard) = read_version.try_write_for(TRY_LOCK_TIMEOUT) { + f(instance_id, &mut write_guard); + } else { + warn!(instance_id, "failed to get lock again for instance"); + pending.push_back(instance_id); + } + } } fn handle_data_spilled(&mut self, staging_sstable_info: StagingSstableInfo) { - // todo: do some prune for version update let staging_sstable_info = Arc::new(staging_sstable_info); - self.for_each_read_version(|read_version| { - trace!("data_spilled. SST size {}", staging_sstable_info.imm_size()); - read_version.update(VersionUpdate::Staging(StagingData::Sst( - staging_sstable_info.clone(), - ))) - }) + trace!("data_spilled. SST size {}", staging_sstable_info.imm_size()); + self.for_each_read_version( + staging_sstable_info.imm_ids().keys().cloned(), + |_, read_version| { + read_version.update(VersionUpdate::Staging(StagingData::Sst( + staging_sstable_info.clone(), + ))) + }, + ) } fn handle_await_sync_epoch( @@ -679,9 +738,13 @@ impl HummockEventHandler { .store(Arc::new(new_pinned_version.clone())); { - self.for_each_read_version(|read_version| { - read_version.update(VersionUpdate::CommittedSnapshot(new_pinned_version.clone())) - }); + self.for_each_read_version( + self.local_read_version_mapping.keys().cloned(), + |_, read_version| { + read_version + .update(VersionUpdate::CommittedSnapshot(new_pinned_version.clone())) + }, + ); } let prev_max_committed_epoch = pinned_version.max_committed_epoch(); @@ -769,26 +832,6 @@ impl HummockEventHandler { let _timer = self.metrics.event_handler_on_spilled_latency.start_timer(); self.handle_data_spilled(staging_sstable_info); } - - UploaderEvent::ImmMerged(merge_output) => { - // update read version for corresponding table shards - if let Some(read_version) = self - .local_read_version_mapping - .get(&merge_output.instance_id) - { - read_version - .write() - .update(VersionUpdate::Staging(StagingData::MergedImmMem( - merge_output.merged_imm, - merge_output.imm_ids, - ))); - } else { - warn!( - "handle ImmMerged: table instance not found. table {:?}, instance {}", - &merge_output.table_id, &merge_output.instance_id - ) - } - } } } @@ -830,9 +873,6 @@ impl HummockEventHandler { if is_checkpoint { self.uploader.start_sync_epoch(epoch); - } else { - // start merging task on non-checkpoint epochs sealed - self.uploader.start_merge_imms(epoch); } } @@ -868,17 +908,17 @@ impl HummockEventHandler { vnodes, } => { let pinned_version = self.pinned_version.load(); + let instance_id = self.generate_instance_id(); let basic_read_version = Arc::new(RwLock::new( HummockReadVersion::new_with_replication_option( table_id, + instance_id, (**pinned_version).clone(), is_replicated, vnodes, ), )); - let instance_id = self.generate_instance_id(); - debug!( "new read version registered: table_id: {}, instance_id: {}", table_id, instance_id @@ -1011,209 +1051,22 @@ fn to_sync_result(result: &HummockResult) -> HummockResult(); - spawn_upload_task_tx.send(tx).unwrap(); - spawn(async move { - // wait for main thread to notify returning error - rx.await.unwrap(); - Err(HummockError::other("".to_string())) - }) - }), - Arc::new(move |_, _, imms, _| { - let (tx, rx) = oneshot::channel::<()>(); - let (finish_tx, finish_rx) = oneshot::channel::<()>(); - spawn_merging_task_tx.send((tx, finish_rx)).unwrap(); - spawn(async move { - rx.await.unwrap(); - finish_tx.send(()).unwrap(); - imms[0].clone() - }) - }), - CacheRefiller::default_spawn_refill_task(), - ); - - let tx = event_handler.event_sender(); - - let _join_handle = spawn(event_handler.start_hummock_event_handler_worker()); - - let (read_version_tx, read_version_rx) = oneshot::channel(); - - tx.send(HummockEvent::RegisterReadVersion { - table_id, - new_read_version_sender: read_version_tx, - is_replicated: false, - vnodes: Arc::new(Bitmap::ones(VirtualNode::COUNT)), - }) - .unwrap(); - let (read_version, guard) = read_version_rx.await.unwrap(); - let instance_id = guard.instance_id; - - let build_batch = |epoch, spill_offset| { - SharedBufferBatch::build_shared_buffer_batch( - epoch, - spill_offset, - vec![(TableKey(Bytes::from("key")), SharedBufferValue::Delete)], - None, - 10, - table_id, - instance_id, - None, - ) - }; - - let epoch1 = epoch0.next_epoch(); - let imm1 = build_batch(epoch1, 0); - read_version - .write() - .update(VersionUpdate::Staging(StagingData::ImmMem(imm1.clone()))); - tx.send(HummockEvent::ImmToUploader(imm1.clone())).unwrap(); - tx.send(HummockEvent::SealEpoch { - epoch: epoch1, - is_checkpoint: true, - }) - .unwrap(); - let (sync_tx, mut sync_rx) = oneshot::channel(); - tx.send(HummockEvent::AwaitSyncEpoch { - new_sync_epoch: epoch1, - sync_result_sender: sync_tx, - }) - .unwrap(); - - let upload_finish_tx = spawn_upload_task_rx.recv().await.unwrap(); - assert!(poll_fn(|cx| Poll::Ready(sync_rx.poll_unpin(cx))) - .await - .is_pending()); - - let epoch2 = epoch1.next_epoch(); - let mut imm_ids = Vec::new(); - for i in 0..10 { - let imm = build_batch(epoch2, i); - imm_ids.push(imm.batch_id()); - read_version - .write() - .update(VersionUpdate::Staging(StagingData::ImmMem(imm.clone()))); - tx.send(HummockEvent::ImmToUploader(imm)).unwrap(); - } - - for (staging_imm, imm_id) in read_version - .read() - .staging() - .imm - .iter() - .zip_eq_debug(imm_ids.iter().copied().rev().chain(once(imm1.batch_id()))) - { - assert_eq!(staging_imm.batch_id(), imm_id); - } - - // should start merging task - tx.send(HummockEvent::SealEpoch { - epoch: epoch2, - is_checkpoint: false, - }) - .unwrap(); - - println!("before wait spawn merging task"); - - let (merging_start_tx, merging_finish_rx) = spawn_merging_task_rx.recv().await.unwrap(); - merging_start_tx.send(()).unwrap(); - - println!("after wait spawn merging task"); - - // yield to possibly poll the merging task, though it shouldn't poll it because there is unfinished syncing task - yield_now().await; - - for (staging_imm, imm_id) in read_version - .read() - .staging() - .imm - .iter() - .zip_eq_debug(imm_ids.iter().copied().rev().chain(once(imm1.batch_id()))) - { - assert_eq!(staging_imm.batch_id(), imm_id); - } - - upload_finish_tx.send(()).unwrap(); - assert!(sync_rx.await.unwrap().is_err()); - - merging_finish_rx.await.unwrap(); - - // yield to poll the merging task, and then it should have finished. - for _ in 0..10 { - yield_now().await; - } - - assert_eq!( - read_version - .read() - .staging() - .imm - .iter() - .map(|imm| imm.batch_id()) - .collect_vec(), - vec![*imm_ids.last().unwrap(), imm1.batch_id()] - ); - } - #[tokio::test] async fn test_clear_shared_buffer() { let epoch0 = 233; @@ -1243,7 +1096,6 @@ mod tests { Arc::new(HummockStateStoreMetrics::unused()), &default_opts_for_test(), Arc::new(|_, _| unreachable!("should not spawn upload task")), - Arc::new(|_, _, _, _| unreachable!("should not spawn merging task")), Arc::new(move |_, _, old_version, new_version| { let (tx, rx) = oneshot::channel(); refill_task_tx_clone diff --git a/src/storage/src/hummock/event_handler/mod.rs b/src/storage/src/hummock/event_handler/mod.rs index 9e98fac4ee3b1..efbae0ac21544 100644 --- a/src/storage/src/hummock/event_handler/mod.rs +++ b/src/storage/src/hummock/event_handler/mod.rs @@ -170,6 +170,7 @@ impl std::fmt::Debug for HummockEvent { } pub type LocalInstanceId = u64; +pub const TEST_LOCAL_INSTANCE_ID: LocalInstanceId = 233; pub type HummockReadVersionRef = Arc>; pub type ReadVersionMappingType = HashMap>; pub type ReadOnlyReadVersionMapping = ReadOnlyRwLockRef; diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index e4b5e6806c28b..c80bb0a9577cc 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -13,26 +13,23 @@ // limitations under the License. use std::collections::hash_map::Entry; -use std::collections::{BTreeMap, BTreeSet, HashMap, VecDeque}; +use std::collections::{BTreeMap, HashMap, VecDeque}; use std::fmt::{Debug, Display, Formatter}; use std::future::{poll_fn, Future}; -use std::mem::swap; +use std::mem::take; use std::pin::Pin; -use std::sync::atomic::AtomicUsize; -use std::sync::atomic::Ordering::Relaxed; -use std::sync::{Arc, LazyLock}; +use std::sync::Arc; use std::task::{ready, Context, Poll}; use futures::future::{try_join_all, TryJoinAll}; use futures::FutureExt; use itertools::Itertools; -use more_asserts::{assert_ge, assert_gt, assert_le}; +use more_asserts::{assert_ge, assert_gt}; use prometheus::core::{AtomicU64, GenericGauge}; use prometheus::{HistogramTimer, IntGauge}; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; -use risingwave_hummock_sdk::key::EPOCH_LEN; use risingwave_hummock_sdk::table_watermark::{ TableWatermarks, VnodeWatermark, WatermarkDirection, }; @@ -41,19 +38,16 @@ use thiserror_ext::AsReport; use tokio::task::JoinHandle; use tracing::{debug, error, info}; -use crate::hummock::compactor::{ - await_tree_key, merge_imms_in_memory, CompactionAwaitTreeRegRef, CompactionExecutor, -}; use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::LocalInstanceId; use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::store::version::StagingSstableInfo; -use crate::hummock::utils::MemoryTracker; use crate::hummock::{HummockError, HummockResult, ImmutableMemtable}; use crate::mem_table::ImmId; use crate::monitor::HummockStateStoreMetrics; use crate::opts::StorageOpts; +pub type UploadTaskInput = HashMap>; pub type UploadTaskPayload = Vec; #[derive(Debug)] @@ -63,64 +57,17 @@ pub struct UploadTaskOutput { pub wait_poll_timer: Option, } pub type SpawnUploadTask = Arc< - dyn Fn(UploadTaskPayload, UploadTaskInfo) -> JoinHandle> - + Send - + Sync - + 'static, ->; - -pub type SpawnMergingTask = Arc< - dyn Fn( - TableId, - LocalInstanceId, - Vec, - Option, - ) -> JoinHandle + dyn Fn(UploadTaskInput, UploadTaskInfo) -> JoinHandle> + Send + Sync + 'static, >; -pub(crate) fn default_spawn_merging_task( - compaction_executor: Arc, - await_tree_reg: Option, -) -> SpawnMergingTask { - Arc::new(move |table_id, instance_id, imms, tracker| { - compaction_executor.spawn({ - static NEXT_MERGING_TASK_ID: LazyLock = - LazyLock::new(|| AtomicUsize::new(0)); - let tree_root = await_tree_reg.as_ref().map(|reg| { - let merging_task_id = NEXT_MERGING_TASK_ID.fetch_add(1, Relaxed); - reg.register( - await_tree_key::MergingTask { - id: merging_task_id, - }, - format!( - "Merging Imm {:?} {:?} {:?}", - table_id, - instance_id, - imms.iter() - .flat_map(|imm| imm.epochs().iter()) - .copied() - .collect::>() - ), - ) - }); - let future = merge_imms_in_memory(table_id, instance_id, imms, tracker); - if let Some(root) = tree_root { - root.instrument(future).left_future() - } else { - future.right_future() - } - }) - }) -} - #[derive(Clone)] pub struct UploadTaskInfo { pub task_size: usize, pub epochs: Vec, - pub imm_ids: Vec, + pub imm_ids: HashMap>, pub compaction_group_index: Arc>, } @@ -147,7 +94,8 @@ impl Debug for UploadTaskInfo { /// A wrapper for a uploading task that compacts and uploads the imm payload. Task context are /// stored so that when the task fails, it can be re-tried. struct UploadingTask { - payload: UploadTaskPayload, + // newer data at the front + payload: UploadTaskInput, join_handle: JoinHandle>, task_info: UploadTaskInfo, spawn_upload_task: SpawnUploadTask, @@ -155,75 +103,6 @@ struct UploadingTask { task_count_guard: IntGauge, } -pub struct MergeImmTaskOutput { - /// Input imm ids of the merging task. Larger imm ids at the front. - pub imm_ids: Vec, - pub table_id: TableId, - pub instance_id: LocalInstanceId, - pub merged_imm: ImmutableMemtable, -} - -// A future that merges multiple immutable memtables to a single immutable memtable. -struct MergingImmTask { - table_id: TableId, - instance_id: LocalInstanceId, - input_imms: Vec, - join_handle: JoinHandle, -} - -impl Debug for MergingImmTask { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("MergingImmTask") - .field("table_id", &self.table_id) - .field("instance_id", &self.instance_id) - .field("input_imms", &self.input_imms) - .finish() - } -} - -impl MergingImmTask { - fn new( - table_id: TableId, - instance_id: LocalInstanceId, - imms: Vec, - memory_tracker: Option, - context: &UploaderContext, - ) -> Self { - assert!(imms.iter().rev().map(|imm| imm.batch_id()).is_sorted()); - let input_imms = imms.clone(); - let join_handle = (context.spawn_merging_task)(table_id, instance_id, imms, memory_tracker); - - MergingImmTask { - table_id, - instance_id, - input_imms, - join_handle, - } - } - - /// Poll the result of the merge task - fn poll_result(&mut self, cx: &mut Context<'_>) -> Poll { - Poll::Ready(match ready!(self.join_handle.poll_unpin(cx)) { - Ok(task_result) => task_result, - Err(err) => { - panic!( - "failed to join merging task: {:?} {:?}", - err.as_report(), - self - ); - } - }) - } -} - -impl Future for MergingImmTask { - type Output = ImmutableMemtable; - - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - self.poll_result(cx) - } -} - impl Drop for UploadingTask { fn drop(&mut self) { self.task_size_guard.sub(self.task_info.task_size as u64); @@ -244,19 +123,30 @@ impl UploadingTask { // INFO logs will be enabled for task with size exceeding 50MB. const LOG_THRESHOLD_FOR_UPLOAD_TASK_SIZE: usize = 50 * (1 << 20); - fn new(payload: UploadTaskPayload, context: &UploaderContext) -> Self { + fn new(payload: UploadTaskInput, context: &UploaderContext) -> Self { assert!(!payload.is_empty()); let mut epochs = payload .iter() - .flat_map(|imm| imm.epochs().clone()) + .flat_map(|(_, imms)| imms.iter().flat_map(|imm| imm.epochs().iter().cloned())) .sorted() .dedup() .collect_vec(); // reverse to make newer epochs comes first epochs.reverse(); - let imm_ids = payload.iter().map(|imm| imm.batch_id()).collect_vec(); - let task_size = payload.iter().map(|imm| imm.size()).sum(); + let imm_ids = payload + .iter() + .map(|(instance_id, imms)| { + ( + *instance_id, + imms.iter().map(|imm| imm.batch_id()).collect_vec(), + ) + }) + .collect(); + let task_size = payload + .values() + .map(|imms| imms.iter().map(|imm| imm.size()).sum::()) + .sum(); let task_info = UploadTaskInfo { task_size, epochs, @@ -393,7 +283,7 @@ impl SpilledData { #[derive(Default, Debug)] struct UnsealedEpochData { // newer data at the front - imms: VecDeque, + imms: HashMap>, spilled_data: SpilledData, table_watermarks: HashMap, BitmapBuilder)>, @@ -401,7 +291,10 @@ struct UnsealedEpochData { impl UnsealedEpochData { fn flush(&mut self, context: &UploaderContext) { - let imms = self.imms.drain(..).collect_vec(); + let imms: HashMap<_, _> = take(&mut self.imms) + .into_iter() + .map(|(id, imms)| (id, imms.into_iter().collect_vec())) + .collect(); if !imms.is_empty() { let task = UploadingTask::new(imms, context); context.stats.spill_task_counts_from_unsealed.inc(); @@ -463,16 +356,9 @@ struct SealedData { // newer epochs come first epochs: VecDeque, - // store the output of merge task that will be feed into `flush_imms` procedure - merged_imms: VecDeque, - // Sealed imms grouped by table shard. // newer data (larger imm id) at the front - imms_by_table_shard: HashMap<(TableId, LocalInstanceId), VecDeque>, - - // Merging tasks generated from sealed imms - // it should be safe to directly drop these tasks - merging_tasks: VecDeque, + imms_by_table_shard: HashMap>, spilled_data: SpilledData, @@ -482,9 +368,6 @@ struct SealedData { impl SealedData { fn clear(&mut self) { self.spilled_data.clear(); - self.merging_tasks - .drain(..) - .for_each(|task| task.join_handle.abort()); *self = Self::default(); } @@ -525,15 +408,14 @@ impl SealedData { } // rearrange sealed imms by table shard and in epoch descending order - for imm in unseal_epoch_data.imms.into_iter().rev() { - let queue = self - .imms_by_table_shard - .entry((imm.table_id, imm.instance_id)) - .or_default(); - if let Some(front) = queue.front() { - assert_gt!(imm.batch_id(), front.batch_id()); + for (instance_id, imms) in unseal_epoch_data.imms { + let queue = self.imms_by_table_shard.entry(instance_id).or_default(); + for imm in imms.into_iter().rev() { + if let Some(front) = queue.front() { + assert_gt!(imm.batch_id(), front.batch_id()); + } + queue.push_front(imm); } - queue.push_front(imm); } self.epochs.push_front(epoch); @@ -563,51 +445,12 @@ impl SealedData { } } - fn add_merged_imm(&mut self, merged_imm: &ImmutableMemtable) { - // add merged_imm to merged_imms - self.merged_imms.push_front(merged_imm.clone()); - } - - fn drop_merging_tasks(&mut self) { - // pop from newest merging task to restore candidate imms back - while let Some(task) = self.merging_tasks.pop_front() { - // cancel the task - task.join_handle.abort(); - self.imms_by_table_shard - .get_mut(&(task.table_id, task.instance_id)) - .unwrap() - .extend(task.input_imms.into_iter()); - } - } - // Flush can be triggered by either a sync_epoch or a spill (`may_flush`) request. fn flush(&mut self, context: &UploaderContext, is_spilled: bool) { - // drop unfinished merging tasks - self.drop_merging_tasks(); - - // group imms by epoch and order by epoch - let mut imms_by_epoch: BTreeMap> = BTreeMap::new(); - self.imms_by_table_shard.drain().for_each(|(_, imms)| { - for imm in imms { - debug_assert_eq!(imm.max_epoch(), imm.min_epoch()); - imms_by_epoch.entry(imm.max_epoch()).or_default().push(imm); - } - }); - - // When ImmMerged, we have removed those imms that have been merged from the - // `self.imms`, thus we need to feed merged_imms into the `flush` procedure - // to complete a checkpoint. - let merged_imms = self.merged_imms.drain(..); - - // newer epoch comes first, the order is matter for cleaning the staging imms - // when adding a staging sst to the `StagingVersion` - let payload = imms_by_epoch + let payload: HashMap<_, _> = take(&mut self.imms_by_table_shard) .into_iter() - .rev() - // in `imms`, newer data comes first - .flat_map(|(_epoch, imms)| imms) - .chain(merged_imms) - .collect_vec(); + .map(|(id, imms)| (id, imms.into_iter().collect())) + .collect(); if !payload.is_empty() { let task = UploadingTask::new(payload, context); @@ -623,30 +466,9 @@ impl SealedData { } } - fn poll_success_merge_imm(&mut self, cx: &mut Context<'_>) -> Poll> { - // only poll the oldest merge task if there is any - if let Some(task) = self.merging_tasks.back_mut() { - let merged_imm = ready!(task.poll_unpin(cx)); - - // pop the finished task - let task = self.merging_tasks.pop_back().expect("must exist"); - - Poll::Ready(Some(MergeImmTaskOutput { - imm_ids: task.input_imms.iter().map(|imm| imm.batch_id()).collect(), - table_id: task.table_id, - instance_id: task.instance_id, - merged_imm, - })) - } else { - Poll::Ready(None) - } - } - /// Clear self and return the current sealed data fn drain(&mut self) -> SealedData { - let mut ret = SealedData::default(); - swap(&mut ret, self); - ret + take(self) } #[cfg(test)] @@ -656,21 +478,6 @@ impl SealedData { .map(|imms| imms.len()) .sum() } - - #[cfg(test)] - fn imms_by_epoch(&self) -> BTreeMap> { - let mut imms_by_epoch: BTreeMap> = BTreeMap::new(); - self.imms_by_table_shard.iter().for_each(|(_, imms)| { - for imm in imms { - debug_assert!(imm.max_epoch() == imm.min_epoch()); - imms_by_epoch - .entry(imm.max_epoch()) - .or_default() - .push(imm.clone()); - } - }); - imms_by_epoch - } } struct SyncingData { @@ -703,12 +510,7 @@ struct UploaderContext { pinned_version: PinnedVersion, /// When called, it will spawn a task to flush the imm into sst and return the join handle. spawn_upload_task: SpawnUploadTask, - spawn_merging_task: SpawnMergingTask, buffer_tracker: BufferTracker, - /// The number of immutable memtables that will be merged into a new imm. - /// When the number of imms of a table shard exceeds this threshold, uploader will generate - /// merging tasks to merge them. - imm_merge_threshold: usize, stats: Arc, } @@ -717,17 +519,14 @@ impl UploaderContext { fn new( pinned_version: PinnedVersion, spawn_upload_task: SpawnUploadTask, - spawn_merging_task: SpawnMergingTask, buffer_tracker: BufferTracker, - config: &StorageOpts, + _config: &StorageOpts, stats: Arc, ) -> Self { UploaderContext { pinned_version, spawn_upload_task, - spawn_merging_task, buffer_tracker, - imm_merge_threshold: config.imm_merge_threshold, stats, } } @@ -778,7 +577,6 @@ impl HummockUploader { state_store_metrics: Arc, pinned_version: PinnedVersion, spawn_upload_task: SpawnUploadTask, - spawn_merging_task: SpawnMergingTask, buffer_tracker: BufferTracker, config: &StorageOpts, ) -> Self { @@ -794,7 +592,6 @@ impl HummockUploader { context: UploaderContext::new( pinned_version, spawn_upload_task, - spawn_merging_task, buffer_tracker, config, state_store_metrics, @@ -802,11 +599,6 @@ impl HummockUploader { } } - #[cfg(test)] - pub(crate) fn imm_merge_threshold(&self) -> usize { - self.context.imm_merge_threshold - } - pub(crate) fn buffer_tracker(&self) -> &BufferTracker { &self.context.buffer_tracker } @@ -841,7 +633,11 @@ impl HummockUploader { self.max_sealed_epoch ); let unsealed_data = self.unsealed_data.entry(epoch).or_default(); - unsealed_data.imms.push_front(imm); + unsealed_data + .imms + .entry(imm.instance_id) + .or_default() + .push_front(imm); } pub(crate) fn add_table_watermarks( @@ -897,60 +693,6 @@ impl HummockUploader { self.sealed_data.seal_new_epoch(epoch, unsealed_data); } - pub(crate) fn start_merge_imms(&mut self, sealed_epoch: HummockEpoch) { - // skip merging if merge threshold is 1 - if self.context.imm_merge_threshold <= 1 { - return; - } - - let memory_limiter = self.context.buffer_tracker.get_memory_limiter(); - // scan imms of each table shard to generate merging task - // when the number of imms exceeds the merge threshold - for ((table_id, shard_id), imms) in self - .sealed_data - .imms_by_table_shard - .iter_mut() - .filter(|(_, imms)| imms.len() >= self.context.imm_merge_threshold) - { - let imms_to_merge = imms.drain(..).collect_vec(); - let mut value_count = 0; - let mut imm_size = 0; - imms_to_merge.iter().for_each(|imm| { - // ensure imms are sealed - assert_le!(imm.max_epoch(), sealed_epoch); - value_count += imm.value_count(); - imm_size += imm.size(); - }); - - // acquire memory before generate merge task - // if acquire memory failed, the task will not be generated - let memory_sz = (imm_size + value_count * EPOCH_LEN) as u64; - if let Some(tracker) = memory_limiter.try_require_memory(memory_sz) { - self.sealed_data - .merging_tasks - .push_front(MergingImmTask::new( - *table_id, - *shard_id, - imms_to_merge, - Some(tracker), - &self.context, - )); - } else { - tracing::warn!( - "fail to acqiure memory {} B, skip merging imms for table {}, shard {}", - memory_sz, - table_id, - shard_id - ); - imms.extend(imms_to_merge); - } - } - } - - pub(crate) fn update_sealed_data(&mut self, merged_imm: &ImmutableMemtable) { - self.sealed_data.add_merged_imm(merged_imm); - } - pub(crate) fn start_sync_epoch(&mut self, epoch: HummockEpoch) { debug!("start sync epoch: {}", epoch); assert!( @@ -1161,40 +903,12 @@ impl HummockUploader { } Poll::Ready(None) } - - fn poll_sealed_merge_imm_task( - &mut self, - cx: &mut Context<'_>, - ) -> Poll> { - let poll_ret = self.sealed_data.poll_success_merge_imm(cx); - if let Poll::Ready(Some(output)) = &poll_ret { - let table_id_label = output.table_id.to_string(); - - // monitor finished task - self.context - .stats - .merge_imm_task_counts - .with_label_values(&[table_id_label.as_str()]) - .inc(); - // monitor merge imm memory size - // we should also add up the size of EPOCH stored in each entry - self.context - .stats - .merge_imm_batch_memory_sz - .with_label_values(&[table_id_label.as_str()]) - .inc_by( - (output.merged_imm.size() + output.merged_imm.value_count() * EPOCH_LEN) as _, - ); - } - poll_ret - } } pub(crate) enum UploaderEvent { // staging sstable info of newer data comes first SyncFinish(HummockEpoch, Vec), DataSpilled(StagingSstableInfo), - ImmMerged(MergeImmTaskOutput), } impl HummockUploader { @@ -1212,11 +926,6 @@ impl HummockUploader { return Poll::Ready(UploaderEvent::DataSpilled(sstable_info)); } - if let Some(merge_output) = ready!(self.poll_sealed_merge_imm_task(cx)) { - // add the merged imm into sealed data - self.update_sealed_data(&merge_output.merged_imm); - return Poll::Ready(UploaderEvent::ImmMerged(merge_output)); - } Poll::Pending }) } @@ -1231,7 +940,6 @@ mod tests { use std::sync::atomic::Ordering::SeqCst; use std::sync::Arc; use std::task::Poll; - use std::time::Duration; use bytes::Bytes; use futures::future::BoxFuture; @@ -1249,19 +957,15 @@ mod tests { use tokio::sync::oneshot; use tokio::task::yield_now; - use crate::hummock::compactor::CompactionExecutor; use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::uploader::{ - default_spawn_merging_task, HummockUploader, MergingImmTask, UploadTaskInfo, - UploadTaskOutput, UploadTaskPayload, UploaderContext, UploaderEvent, UploadingTask, - }; - use crate::hummock::event_handler::LocalInstanceId; - use crate::hummock::iterator::test_utils::{ - iterator_test_table_key_of, transform_shared_buffer, + HummockUploader, UploadTaskInfo, UploadTaskInput, UploadTaskOutput, UploaderContext, + UploaderEvent, UploadingTask, }; + use crate::hummock::event_handler::{LocalInstanceId, TEST_LOCAL_INSTANCE_ID}; use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::shared_buffer::shared_buffer_batch::{ - SharedBufferBatch, SharedBufferValue, + SharedBufferBatch, SharedBufferBatchId, SharedBufferValue, }; use crate::hummock::{HummockError, HummockResult, MemoryLimiter}; use crate::mem_table::{ImmId, ImmutableMemtable}; @@ -1274,7 +978,7 @@ mod tests { pub trait UploadOutputFuture = Future> + Send + 'static; pub trait UploadFn = - Fn(UploadTaskPayload, UploadTaskInfo) -> Fut + Send + Sync + 'static; + Fn(UploadTaskInput, UploadTaskInfo) -> Fut + Send + Sync + 'static; fn test_hummock_version(epoch: HummockEpoch) -> HummockVersion { HummockVersion { @@ -1315,7 +1019,7 @@ mod tests { None, size, TEST_TABLE_ID, - LocalInstanceId::default(), + TEST_LOCAL_INSTANCE_ID, tracker, ) } @@ -1350,11 +1054,9 @@ mod tests { F: UploadFn, { let config = StorageOpts::default(); - let compaction_executor = Arc::new(CompactionExecutor::new(None)); UploaderContext::new( initial_pinned_version(), Arc::new(move |payload, task_info| spawn(upload_fn(payload, task_info))), - default_spawn_merging_task(compaction_executor, None), BufferTracker::for_test(), &config, Arc::new(HummockStateStoreMetrics::unused()), @@ -1367,15 +1069,12 @@ mod tests { F: UploadFn, { let config = StorageOpts { - imm_merge_threshold: 4, ..Default::default() }; - let compaction_executor = Arc::new(CompactionExecutor::new(None)); HummockUploader::new( Arc::new(HummockStateStoreMetrics::unused()), initial_pinned_version(), Arc::new(move |payload, task_info| spawn(upload_fn(payload, task_info))), - default_spawn_merging_task(compaction_executor, None), BufferTracker::for_test(), &config, ) @@ -1391,7 +1090,7 @@ mod tests { #[allow(clippy::unused_async)] async fn dummy_success_upload_future( - _: UploadTaskPayload, + _: UploadTaskInput, _: UploadTaskInfo, ) -> HummockResult { Ok(dummy_success_upload_output()) @@ -1399,22 +1098,42 @@ mod tests { #[allow(clippy::unused_async)] async fn dummy_fail_upload_future( - _: UploadTaskPayload, + _: UploadTaskInput, _: UploadTaskInfo, ) -> HummockResult { Err(HummockError::other("failed")) } + impl UploadingTask { + fn from_vec(imms: Vec, context: &UploaderContext) -> Self { + let mut input: HashMap<_, Vec<_>> = HashMap::new(); + for imm in imms { + input.entry(imm.instance_id).or_default().push(imm); + } + Self::new(input, context) + } + } + + fn get_imm_ids<'a>( + imms: impl IntoIterator, + ) -> HashMap> { + let mut ret: HashMap<_, Vec<_>> = HashMap::new(); + for imm in imms { + ret.entry(imm.instance_id).or_default().push(imm.batch_id()) + } + ret + } + #[tokio::test] pub async fn test_uploading_task_future() { let uploader_context = test_uploader_context(dummy_success_upload_future); let imm = gen_imm(INITIAL_EPOCH).await; let imm_size = imm.size(); - let imm_id = imm.batch_id(); - let task = UploadingTask::new(vec![imm], &uploader_context); + let imm_ids = get_imm_ids(vec![&imm]); + let task = UploadingTask::from_vec(vec![imm], &uploader_context); assert_eq!(imm_size, task.task_info.task_size); - assert_eq!(vec![imm_id], task.task_info.imm_ids); + assert_eq!(imm_ids, task.task_info.imm_ids); assert_eq!(vec![INITIAL_EPOCH], task.task_info.epochs); let output = task.await.unwrap(); assert_eq!( @@ -1422,18 +1141,20 @@ mod tests { &dummy_success_upload_output().new_value_ssts ); assert_eq!(imm_size, output.imm_size()); - assert_eq!(&vec![imm_id], output.imm_ids()); + assert_eq!(&imm_ids, output.imm_ids()); assert_eq!(&vec![INITIAL_EPOCH], output.epochs()); let uploader_context = test_uploader_context(dummy_fail_upload_future); - let task = UploadingTask::new(vec![gen_imm(INITIAL_EPOCH).await], &uploader_context); + let imm = gen_imm(INITIAL_EPOCH).await; + let task = UploadingTask::from_vec(vec![imm], &uploader_context); let _ = task.await.unwrap_err(); } #[tokio::test] pub async fn test_uploading_task_poll_result() { let uploader_context = test_uploader_context(dummy_success_upload_future); - let mut task = UploadingTask::new(vec![gen_imm(INITIAL_EPOCH).await], &uploader_context); + let mut task = + UploadingTask::from_vec(vec![gen_imm(INITIAL_EPOCH).await], &uploader_context); let output = poll_fn(|cx| task.poll_result(cx)).await.unwrap(); assert_eq!( output.sstable_infos(), @@ -1441,7 +1162,8 @@ mod tests { ); let uploader_context = test_uploader_context(dummy_fail_upload_future); - let mut task = UploadingTask::new(vec![gen_imm(INITIAL_EPOCH).await], &uploader_context); + let mut task = + UploadingTask::from_vec(vec![gen_imm(INITIAL_EPOCH).await], &uploader_context); let _ = poll_fn(|cx| task.poll_result(cx)).await.unwrap_err(); } @@ -1463,7 +1185,8 @@ mod tests { ret } }); - let mut task = UploadingTask::new(vec![gen_imm(INITIAL_EPOCH).await], &uploader_context); + let mut task = + UploadingTask::from_vec(vec![gen_imm(INITIAL_EPOCH).await], &uploader_context); let output = poll_fn(|cx| task.poll_ok_with_retry(cx)).await; assert_eq!(fail_num + 1, run_count_clone.load(SeqCst)); assert_eq!( @@ -1514,7 +1237,7 @@ mod tests { assert_eq!(1, ssts.len()); let staging_sst = ssts.first().unwrap(); assert_eq!(&vec![epoch1], staging_sst.epochs()); - assert_eq!(&vec![imm.batch_id()], staging_sst.imm_ids()); + assert_eq!(&get_imm_ids([&imm]), staging_sst.imm_ids()); assert_eq!( &dummy_success_upload_output().new_value_ssts, staging_sst.sstable_infos() @@ -1528,7 +1251,7 @@ mod tests { assert_eq!(1, ssts.len()); let staging_sst = ssts.first().unwrap(); assert_eq!(&vec![epoch1], staging_sst.epochs()); - assert_eq!(&vec![imm.batch_id()], staging_sst.imm_ids()); + assert_eq!(&get_imm_ids([&imm]), staging_sst.imm_ids()); assert_eq!( &dummy_success_upload_output().new_value_ssts, staging_sst.sstable_infos() @@ -1543,107 +1266,6 @@ mod tests { assert_eq!(epoch1, uploader.max_committed_epoch()); } - #[tokio::test] - async fn test_uploader_merge_imms_without_flush() { - let mut uploader = test_uploader(dummy_success_upload_future); - let mut all_imms = VecDeque::new(); - // assume a chckpoint consists of 11 epochs - let ckpt_intervals = 11; - let imm_merge_threshold: usize = uploader.imm_merge_threshold(); - - // For each epoch, we gen imm for 2 shards and add them to uploader and seal the epoch - // afterward. check uploader's state after each epoch has been sealed - // When we get IMM_MERGE_THRESHOLD epochs, there should be merging task started for sealed - // data. Then we await the merging task and check the uploader's state again. - let mut merged_imms = VecDeque::new(); - - let mut epoch = INITIAL_EPOCH; - for i in 1..=ckpt_intervals { - epoch.inc_epoch(); - let mut imm1 = gen_imm(epoch).await; - let mut imm2 = gen_imm(epoch).await; - - imm1.instance_id = 1 as LocalInstanceId; - imm2.instance_id = 2 as LocalInstanceId; - - uploader.add_imm(imm1.clone()); - uploader.add_imm(imm2.clone()); - - // newer imm comes in front - all_imms.push_front(imm1); - all_imms.push_front(imm2); - - uploader.seal_epoch(epoch); - - assert_eq!(epoch, uploader.max_sealed_epoch); - // check sealed data has two imms - let imms_by_epoch = uploader.sealed_data.imms_by_epoch(); - if let Some((e, imms)) = imms_by_epoch.last_key_value() - && *e == epoch - { - assert_eq!(2, imms.len()); - } - - let epoch_cnt = i; - - if epoch_cnt < imm_merge_threshold { - assert!(uploader.sealed_data.merging_tasks.is_empty()); - assert!(uploader.sealed_data.spilled_data.is_empty()); - assert_eq!(epoch_cnt, uploader.sealed_data.epochs.len()); - } else { - assert_eq!(epoch_cnt, uploader.sealed_data.epochs.len()); - - let unmerged_imm_cnt: usize = epoch_cnt - imm_merge_threshold * merged_imms.len(); - - if unmerged_imm_cnt < imm_merge_threshold { - continue; - } - - let imms_by_shard = &mut uploader.sealed_data.imms_by_table_shard; - // check shard 1 - if let Some(imms) = imms_by_shard.get(&(TEST_TABLE_ID, 1 as LocalInstanceId)) { - assert_eq!(imm_merge_threshold, imms.len()); - } - - // check shard 2 - if let Some(imms) = imms_by_shard.get(&(TEST_TABLE_ID, 2 as LocalInstanceId)) { - assert_eq!(imm_merge_threshold, imms.len()); - } - - // we have enough sealed imms, start merging task - println!("start merging task for epoch {}", epoch); - uploader.start_merge_imms(epoch); - assert!(!uploader.sealed_data.merging_tasks.is_empty()); - assert!(uploader.sealed_data.spilled_data.is_empty()); - - // check after generate merging task - if let Some(imms) = uploader - .sealed_data - .imms_by_table_shard - .get(&(TEST_TABLE_ID, 1 as LocalInstanceId)) - { - assert_eq!(0, imms.len()); - } - if let Some(imms) = uploader - .sealed_data - .imms_by_table_shard - .get(&(TEST_TABLE_ID, 2 as LocalInstanceId)) - { - assert_eq!(0, imms.len()); - } - - // poll the merging task and check the result - match uploader.next_event().await { - UploaderEvent::ImmMerged(output) => { - println!("merging task success for epoch {}", epoch); - merged_imms.push_front(output.merged_imm); - } - _ => unreachable!(), - }; - } - } - } - #[tokio::test] async fn test_uploader_empty_epoch() { let mut uploader = test_uploader(dummy_success_upload_future); @@ -1675,90 +1297,6 @@ mod tests { assert_eq!(epoch1, uploader.max_committed_epoch()); } - #[tokio::test] - async fn test_drop_success_merging_task() { - let table_id = TableId { table_id: 1004 }; - let shared_buffer_items1: Vec<(Vec, SharedBufferValue)> = vec![ - ( - iterator_test_table_key_of(1), - SharedBufferValue::Insert(Bytes::from("value1")), - ), - ( - iterator_test_table_key_of(2), - SharedBufferValue::Insert(Bytes::from("value2")), - ), - ( - iterator_test_table_key_of(3), - SharedBufferValue::Insert(Bytes::from("value3")), - ), - ]; - let epoch = test_epoch(1); - let imm1 = SharedBufferBatch::for_test( - transform_shared_buffer(shared_buffer_items1.clone()), - epoch, - table_id, - ); - let shared_buffer_items2: Vec<(Vec, SharedBufferValue)> = vec![ - ( - iterator_test_table_key_of(1), - SharedBufferValue::Insert(Bytes::from("value12")), - ), - ( - iterator_test_table_key_of(2), - SharedBufferValue::Insert(Bytes::from("value22")), - ), - ( - iterator_test_table_key_of(3), - SharedBufferValue::Insert(Bytes::from("value32")), - ), - ]; - let epoch = test_epoch(2); - let imm2 = SharedBufferBatch::for_test( - transform_shared_buffer(shared_buffer_items2.clone()), - epoch, - table_id, - ); - - let shared_buffer_items3: Vec<(Vec, SharedBufferValue)> = vec![ - ( - iterator_test_table_key_of(1), - SharedBufferValue::Insert(Bytes::from("value13")), - ), - ( - iterator_test_table_key_of(2), - SharedBufferValue::Insert(Bytes::from("value23")), - ), - ( - iterator_test_table_key_of(3), - SharedBufferValue::Insert(Bytes::from("value33")), - ), - ]; - let epoch = test_epoch(3); - let imm3 = SharedBufferBatch::for_test( - transform_shared_buffer(shared_buffer_items3.clone()), - epoch, - table_id, - ); - - // newer data comes first - let imms = vec![imm3, imm2, imm1]; - let context = test_uploader_context(dummy_success_upload_future); - let mut task = MergingImmTask::new(table_id, 0, imms, None, &context); - let sleep = tokio::time::sleep(Duration::from_millis(500)); - tokio::select! { - _ = sleep => { - println!("sleep timeout") - } - imm = &mut task => { - println!("merging task success"); - assert_eq!(table_id, imm.table_id); - assert_eq!(9, imm.value_count()); - } - } - task.join_handle.abort(); - println!("merging task abort success"); - } - #[tokio::test] async fn test_uploader_poll_empty() { let mut uploader = test_uploader(dummy_success_upload_future); @@ -1816,7 +1354,6 @@ mod tests { assert_eq!(epoch6, epoch); } UploaderEvent::DataSpilled(_) => unreachable!(), - UploaderEvent::ImmMerged(_) => unreachable!(), } uploader.update_pinned_version(version5); assert_eq!(epoch6, uploader.max_synced_epoch); @@ -1827,7 +1364,7 @@ mod tests { fn prepare_uploader_order_test() -> ( BufferTracker, HummockUploader, - impl Fn(Vec) -> (BoxFuture<'static, ()>, oneshot::Sender<()>), + impl Fn(HashMap>) -> (BoxFuture<'static, ()>, oneshot::Sender<()>), ) { // flush threshold is 0. Flush anyway let buffer_tracker = @@ -1840,7 +1377,7 @@ mod tests { let new_task_notifier = { let task_notifier_holder = task_notifier_holder.clone(); - move |imm_ids: Vec| { + move |imm_ids: HashMap>| { let (start_tx, start_rx) = oneshot::channel(); let (finish_tx, finish_rx) = oneshot::channel(); task_notifier_holder @@ -1856,12 +1393,11 @@ mod tests { }; let config = StorageOpts::default(); - let compaction_executor = Arc::new(CompactionExecutor::new(None)); let uploader = HummockUploader::new( Arc::new(HummockStateStoreMetrics::unused()), initial_pinned_version(), Arc::new({ - move |_: UploadTaskPayload, task_info: UploadTaskInfo| { + move |_, task_info: UploadTaskInfo| { let task_notifier_holder = task_notifier_holder.clone(); let (start_tx, finish_rx) = task_notifier_holder.lock().pop_back().unwrap(); let start_epoch = *task_info.epochs.last().unwrap(); @@ -1879,7 +1415,6 @@ mod tests { }) } }), - default_spawn_merging_task(compaction_executor, None), buffer_tracker.clone(), &config, ); @@ -1915,9 +1450,8 @@ mod tests { uploader.add_imm(imm1_2.clone()); // imm1 will be spilled first - let (await_start1, finish_tx1) = - new_task_notifier(vec![imm1_2.batch_id(), imm1_1.batch_id()]); - let (await_start2, finish_tx2) = new_task_notifier(vec![imm2.batch_id()]); + let (await_start1, finish_tx1) = new_task_notifier(get_imm_ids([&imm1_2, &imm1_1])); + let (await_start2, finish_tx2) = new_task_notifier(get_imm_ids([&imm2])); uploader.may_flush(); await_start1.await; await_start2.await; @@ -1929,14 +1463,14 @@ mod tests { finish_tx1.send(()).unwrap(); if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { - assert_eq!(&vec![imm1_2.batch_id(), imm1_1.batch_id()], sst.imm_ids()); + assert_eq!(&get_imm_ids([&imm1_2, &imm1_1]), sst.imm_ids()); assert_eq!(&vec![epoch1], sst.epochs()); } else { unreachable!("") } if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { - assert_eq!(&vec![imm2.batch_id()], sst.imm_ids()); + assert_eq!(&get_imm_ids([&imm2]), sst.imm_ids()); assert_eq!(&vec![epoch2], sst.epochs()); } else { unreachable!("") @@ -1944,12 +1478,12 @@ mod tests { let imm1_3 = gen_imm_with_limiter(epoch1, memory_limiter).await; uploader.add_imm(imm1_3.clone()); - let (await_start1_3, finish_tx1_3) = new_task_notifier(vec![imm1_3.batch_id()]); + let (await_start1_3, finish_tx1_3) = new_task_notifier(get_imm_ids([&imm1_3])); uploader.may_flush(); await_start1_3.await; let imm1_4 = gen_imm_with_limiter(epoch1, memory_limiter).await; uploader.add_imm(imm1_4.clone()); - let (await_start1_4, finish_tx1_4) = new_task_notifier(vec![imm1_4.batch_id()]); + let (await_start1_4, finish_tx1_4) = new_task_notifier(get_imm_ids([&imm1_4])); uploader.seal_epoch(epoch1); uploader.start_sync_epoch(epoch1); await_start1_4.await; @@ -1964,12 +1498,12 @@ mod tests { let epoch3 = epoch2.next_epoch(); let imm3_1 = gen_imm_with_limiter(epoch3, memory_limiter).await; uploader.add_imm(imm3_1.clone()); - let (await_start3_1, finish_tx3_1) = new_task_notifier(vec![imm3_1.batch_id()]); + let (await_start3_1, finish_tx3_1) = new_task_notifier(get_imm_ids([&imm3_1])); uploader.may_flush(); await_start3_1.await; let imm3_2 = gen_imm_with_limiter(epoch3, memory_limiter).await; uploader.add_imm(imm3_2.clone()); - let (await_start3_2, finish_tx3_2) = new_task_notifier(vec![imm3_2.batch_id()]); + let (await_start3_2, finish_tx3_2) = new_task_notifier(get_imm_ids([&imm3_2])); uploader.may_flush(); await_start3_2.await; let imm3_3 = gen_imm_with_limiter(epoch3, memory_limiter).await; @@ -2000,8 +1534,8 @@ mod tests { if let UploaderEvent::SyncFinish(epoch, newly_upload_sst) = uploader.next_event().await { assert_eq!(epoch1, epoch); assert_eq!(2, newly_upload_sst.len()); - assert_eq!(&vec![imm1_4.batch_id()], newly_upload_sst[0].imm_ids()); - assert_eq!(&vec![imm1_3.batch_id()], newly_upload_sst[1].imm_ids()); + assert_eq!(&get_imm_ids([&imm1_4]), newly_upload_sst[0].imm_ids()); + assert_eq!(&get_imm_ids([&imm1_3]), newly_upload_sst[1].imm_ids()); } else { unreachable!("should be sync finish"); } @@ -2013,12 +1547,9 @@ mod tests { .unwrap() .staging_ssts; assert_eq!(3, synced_data1.len()); - assert_eq!(&vec![imm1_4.batch_id()], synced_data1[0].imm_ids()); - assert_eq!(&vec![imm1_3.batch_id()], synced_data1[1].imm_ids()); - assert_eq!( - &vec![imm1_2.batch_id(), imm1_1.batch_id()], - synced_data1[2].imm_ids() - ); + assert_eq!(&get_imm_ids([&imm1_4]), synced_data1[0].imm_ids()); + assert_eq!(&get_imm_ids([&imm1_3]), synced_data1[1].imm_ids()); + assert_eq!(&get_imm_ids([&imm1_2, &imm1_1]), synced_data1[2].imm_ids()); // current uploader state: // unsealed: epoch3: imm: imm3_3, uploading: [imm3_2], [imm3_1] @@ -2042,7 +1573,7 @@ mod tests { .unwrap() .staging_ssts; assert_eq!(1, synced_data2.len()); - assert_eq!(&vec![imm2.batch_id()], synced_data2[0].imm_ids()); + assert_eq!(&get_imm_ids([&imm2]), synced_data2[0].imm_ids()); // current uploader state: // unsealed: epoch3: imm: imm3_3, uploading: [imm3_2], [imm3_1] @@ -2054,7 +1585,7 @@ mod tests { uploader.seal_epoch(epoch3); if let UploaderEvent::DataSpilled(sst) = uploader.next_event().await { - assert_eq!(&vec![imm3_1.batch_id()], sst.imm_ids()); + assert_eq!(&get_imm_ids([&imm3_1]), sst.imm_ids()); } else { unreachable!("should be data spilled"); } @@ -2068,7 +1599,7 @@ mod tests { uploader.seal_epoch(epoch4); let (await_start4_with_3_3, finish_tx4_with_3_3) = - new_task_notifier(vec![imm4.batch_id(), imm3_3.batch_id()]); + new_task_notifier(get_imm_ids([&imm4, &imm3_3])); uploader.start_sync_epoch(epoch4); await_start4_with_3_3.await; @@ -2088,10 +1619,10 @@ mod tests { assert_eq!(epoch4, epoch); assert_eq!(2, newly_upload_sst.len()); assert_eq!( - &vec![imm4.batch_id(), imm3_3.batch_id()], + &get_imm_ids([&imm4, &imm3_3]), newly_upload_sst[0].imm_ids() ); - assert_eq!(&vec![imm3_2.batch_id()], newly_upload_sst[1].imm_ids()); + assert_eq!(&get_imm_ids([&imm3_2]), newly_upload_sst[1].imm_ids()); } else { unreachable!("should be sync finish"); } @@ -2104,12 +1635,9 @@ mod tests { .staging_ssts; assert_eq!(3, synced_data4.len()); assert_eq!(&vec![epoch4, epoch3], synced_data4[0].epochs()); - assert_eq!( - &vec![imm4.batch_id(), imm3_3.batch_id()], - synced_data4[0].imm_ids() - ); - assert_eq!(&vec![imm3_2.batch_id()], synced_data4[1].imm_ids()); - assert_eq!(&vec![imm3_1.batch_id()], synced_data4[2].imm_ids()); + assert_eq!(&get_imm_ids([&imm4, &imm3_3]), synced_data4[0].imm_ids()); + assert_eq!(&get_imm_ids([&imm3_2]), synced_data4[1].imm_ids()); + assert_eq!(&get_imm_ids([&imm3_1]), synced_data4[2].imm_ids()); // current uploader state: // unsealed: empty diff --git a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs index ba9d3311b2e0b..39f8a23cacc4b 100644 --- a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs +++ b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs @@ -578,10 +578,11 @@ impl SharedBufferBatch { ) -> Self { let inner = SharedBufferBatchInner::new(epoch, spill_offset, sorted_items, None, size, None); + use crate::hummock::event_handler::TEST_LOCAL_INSTANCE_ID; SharedBufferBatch { inner: Arc::new(inner), table_id, - instance_id: LocalInstanceId::default(), + instance_id: TEST_LOCAL_INSTANCE_ID, } } } diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index f772fbbe79d93..81bff5c5a30da 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -14,7 +14,7 @@ use std::cmp::Ordering; use std::collections::vec_deque::VecDeque; -use std::collections::HashSet; +use std::collections::HashMap; use std::iter::once; use std::ops::Bound::Included; use std::sync::Arc; @@ -39,17 +39,19 @@ use risingwave_hummock_sdk::version::HummockVersionDelta; use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch, LocalSstableInfo}; use risingwave_pb::hummock::{EpochNewChangeLog, LevelType, SstableInfo}; use sync_point::sync_point; +use tracing::warn; use super::StagingDataIterator; use crate::error::StorageResult; +use crate::hummock::event_handler::LocalInstanceId; use crate::hummock::iterator::change_log::ChangeLogIterator; use crate::hummock::iterator::{ConcatIterator, HummockIteratorUnion, MergeIterator, UserIterator}; use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::sstable::SstableIteratorReadOptions; use crate::hummock::sstable_store::SstableStoreRef; use crate::hummock::utils::{ - check_subset_preserve_order, filter_single_sst, prune_nonoverlapping_ssts, - prune_overlapping_ssts, range_overlap, search_sst_idx, + filter_single_sst, prune_nonoverlapping_ssts, prune_overlapping_ssts, range_overlap, + search_sst_idx, }; use crate::hummock::{ get_from_batch, get_from_sstable_info, hit_sstable_bloom_filter, HummockError, HummockResult, @@ -77,7 +79,8 @@ pub struct StagingSstableInfo { /// Epochs whose data are included in the Sstable. The newer epoch comes first. /// The field must not be empty. epochs: Vec, - imm_ids: Vec, + // newer data at the front + imm_ids: HashMap>, imm_size: usize, } @@ -86,7 +89,7 @@ impl StagingSstableInfo { sstable_infos: Vec, old_value_sstable_infos: Vec, epochs: Vec, - imm_ids: Vec, + imm_ids: HashMap>, imm_size: usize, ) -> Self { // the epochs are sorted from higher epoch to lower epoch @@ -116,7 +119,7 @@ impl StagingSstableInfo { &self.epochs } - pub fn imm_ids(&self) -> &Vec { + pub fn imm_ids(&self) -> &HashMap> { &self.imm_ids } } @@ -124,7 +127,6 @@ impl StagingSstableInfo { #[derive(Clone)] pub enum StagingData { ImmMem(ImmutableMemtable), - MergedImmMem(ImmutableMemtable, Vec), Sst(Arc), } @@ -208,6 +210,7 @@ impl StagingVersion { /// A container of information required for reading from hummock. pub struct HummockReadVersion { table_id: TableId, + instance_id: LocalInstanceId, /// Local version for staging data. staging: StagingVersion, @@ -231,6 +234,7 @@ pub struct HummockReadVersion { impl HummockReadVersion { pub fn new_with_replication_option( table_id: TableId, + instance_id: LocalInstanceId, committed_version: CommittedVersion, is_replicated: bool, vnodes: Arc, @@ -241,6 +245,7 @@ impl HummockReadVersion { assert!(committed_version.is_valid()); Self { table_id, + instance_id, table_watermarks: committed_version .version() .table_watermarks @@ -265,10 +270,11 @@ impl HummockReadVersion { pub fn new( table_id: TableId, + instance_id: LocalInstanceId, committed_version: CommittedVersion, vnodes: Arc, ) -> Self { - Self::new_with_replication_option(table_id, committed_version, false, vnodes) + Self::new_with_replication_option(table_id, instance_id, committed_version, false, vnodes) } pub fn table_id(&self) -> TableId { @@ -297,77 +303,41 @@ impl HummockReadVersion { self.staging.imm.push_front(imm) } - StagingData::MergedImmMem(merged_imm, imm_ids) => { - self.add_merged_imm(merged_imm, imm_ids); - } StagingData::Sst(staging_sst_ref) => { - // The following properties must be ensured: - // 1) self.staging.imm is sorted by imm id descendingly - // 2) staging_sst.imm_ids preserves the imm id partial - // ordering of the participating read version imms. Example: - // If staging_sst contains two read versions r1: [i1, i3] and r2: [i2, i4], - // then [i2, i1, i3, i4] is valid while [i3, i1, i2, i4] is invalid. - // 3) The intersection between staging_sst.imm_ids and self.staging.imm - // are always the suffix of self.staging.imm - - // Check 1) - debug_assert!(self - .staging - .imm - .iter() - .rev() - .is_sorted_by_key(|imm| imm.batch_id())); - - // Calculate intersection - let staging_imm_ids_from_imms: HashSet = - self.staging.imm.iter().map(|imm| imm.batch_id()).collect(); - - // intersected batch_id order from oldest to newest - let intersect_imm_ids = staging_sst_ref - .imm_ids - .iter() - .rev() - .copied() - .filter(|id| staging_imm_ids_from_imms.contains(id)) - .collect_vec(); - - if !intersect_imm_ids.is_empty() { - // Check 2) - debug_assert!(check_subset_preserve_order( - intersect_imm_ids.iter().copied(), - self.staging.imm.iter().map(|imm| imm.batch_id()).rev(), - )); - - // Check 3) and replace imms with a staging sst - for imm_id in &intersect_imm_ids { - if let Some(imm) = self.staging.imm.back() { - if *imm_id == imm.batch_id() { - self.staging.imm.pop_back(); - } - } else { - let local_imm_ids = self - .staging - .imm - .iter() - .map(|imm| imm.batch_id()) - .collect_vec(); - - unreachable!( - "should not reach here staging_sst.size {}, + let Some(imms) = staging_sst_ref.imm_ids.get(&self.instance_id) else { + warn!( + instance_id = self.instance_id, + "no related imm in sst input" + ); + return; + }; + + // old data comes first + for imm_id in imms.iter().rev() { + let valid = match self.staging.imm.pop_back() { + None => false, + Some(prev_imm_id) => prev_imm_id.batch_id() == *imm_id, + }; + assert!( + valid, + "should be valid staging_sst.size {}, staging_sst.imm_ids {:?}, staging_sst.epochs {:?}, local_imm_ids {:?}, - intersect_imm_ids {:?}", - staging_sst_ref.imm_size, - staging_sst_ref.imm_ids, - staging_sst_ref.epochs, - local_imm_ids, - intersect_imm_ids, - ); - } - } - self.staging.sst.push_front(staging_sst_ref); + instance_id {}", + staging_sst_ref.imm_size, + staging_sst_ref.imm_ids, + staging_sst_ref.epochs, + self.staging + .imm + .iter() + .map(|imm| imm.batch_id()) + .collect_vec(), + self.instance_id, + ); } + + self.staging.sst.push_front(staging_sst_ref); } }, @@ -445,92 +415,6 @@ impl HummockReadVersion { } } - /// `imm_ids` is the list of imm ids that are merged into this batch - /// This field is immutable. Larger imm id at the front. - pub fn add_merged_imm(&mut self, merged_imm: ImmutableMemtable, imm_ids: Vec) { - assert!(imm_ids.iter().rev().is_sorted()); - let min_imm_id = *imm_ids.last().expect("non-empty"); - - let back = self.staging.imm.back().expect("should not be empty"); - - // pop and save imms that are written earlier than the oldest imm if there is any - let earlier_imms = if back.batch_id() < min_imm_id { - let mut earlier_imms = VecDeque::with_capacity(self.staging.imm.len()); - loop { - let batch_id = self - .staging - .imm - .back() - .expect("should not be empty") - .batch_id(); - match batch_id.cmp(&min_imm_id) { - Ordering::Less => { - let imm = self.staging.imm.pop_back().unwrap(); - earlier_imms.push_front(imm); - } - Ordering::Equal => { - break; - } - Ordering::Greater => { - let remaining_staging_imm_ids = self - .staging - .imm - .iter() - .map(|imm| imm.batch_id()) - .collect_vec(); - let earlier_imm_ids = - earlier_imms.iter().map(|imm| imm.batch_id()).collect_vec(); - - unreachable!( - "must have break in equal: {:?} {:?} {:?}", - remaining_staging_imm_ids, earlier_imm_ids, imm_ids - ) - } - } - } - Some(earlier_imms) - } else { - assert_eq!( - back.batch_id(), - min_imm_id, - "{:?} {:?}", - { - self.staging - .imm - .iter() - .map(|imm| imm.batch_id()) - .collect_vec() - }, - imm_ids - ); - None - }; - - // iter from smaller imm and take the older imm at the back. - for imm_id in imm_ids.iter().rev() { - let imm = self.staging.imm.pop_back().expect("should exist"); - assert_eq!( - imm.batch_id(), - *imm_id, - "{:?} {:?} {}", - { - self.staging - .imm - .iter() - .map(|imm| imm.batch_id()) - .collect_vec() - }, - imm_ids, - imm_id, - ); - } - - self.staging.imm.push_back(merged_imm); - if let Some(earlier_imms) = earlier_imms { - self.staging.imm.extend(earlier_imms); - } - } - pub fn is_replicated(&self) -> bool { self.is_replicated } diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index 8feaa10cbfe88..85d8d5c772a06 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -39,8 +39,6 @@ pub struct StorageOpts { /// The shared buffer will start flushing data to object when the ratio of memory usage to the /// shared buffer capacity exceed such ratio. pub shared_buffer_flush_ratio: f32, - /// The threshold for the number of immutable memtables to merge to a new imm. - pub imm_merge_threshold: usize, /// Remote directory for storing data and metadata objects. pub data_directory: String, /// Whether to enable write conflict detection @@ -160,7 +158,6 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt .share_buffer_compaction_worker_threads_number, shared_buffer_capacity_mb: s.shared_buffer_capacity_mb, shared_buffer_flush_ratio: c.storage.shared_buffer_flush_ratio, - imm_merge_threshold: c.storage.imm_merge_threshold, data_directory: p.data_directory().to_string(), write_conflict_detection_enabled: c.storage.write_conflict_detection_enabled, block_cache_capacity_mb: s.block_cache_capacity_mb, From a65ab9d70a8f9de0529eb38a7183f92163fe7ebe Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 16 May 2024 20:41:13 +0800 Subject: [PATCH 07/12] chore(storage): bump `object_store_read_attempt_timeout_ms` from `8s` to `16s` (#16719) --- ci/workflows/pull-request.yml | 2 +- src/common/src/config.rs | 6 +++--- src/config/example.toml | 20 ++++++++++---------- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 475b981a18b07..4c7308289ae73 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -97,7 +97,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 21 + timeout_in_minutes: 23 retry: *auto-retry - label: "end-to-end test (parallel)" diff --git a/src/common/src/config.rs b/src/common/src/config.rs index fd457c7faf1ca..f4258481dfd17 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -1751,7 +1751,7 @@ pub mod default { pub mod object_store_config { const DEFAULT_REQ_BACKOFF_INTERVAL_MS: u64 = 1000; // 1s const DEFAULT_REQ_BACKOFF_MAX_DELAY_MS: u64 = 10 * 1000; // 10s - const DEFAULT_REQ_MAX_RETRY_ATTEMPTS: usize = 3; + const DEFAULT_REQ_MAX_RETRY_ATTEMPTS: usize = 5; pub fn object_store_set_atomic_write_dir() -> bool { false @@ -1779,7 +1779,7 @@ pub mod default { // init + upload_part + finish pub fn object_store_streaming_upload_attempt_timeout_ms() -> u64 { - 5 * 1000 // 5s + 10 * 1000 // 10s } pub fn object_store_streaming_upload_retry_attempts() -> usize { @@ -1788,7 +1788,7 @@ pub mod default { // tips: depend on block_size pub fn object_store_read_attempt_timeout_ms() -> u64 { - 8 * 1000 // 8s + 16 * 1000 // 16s } pub fn object_store_read_retry_attempts() -> usize { diff --git a/src/config/example.toml b/src/config/example.toml index abdb1e77353fe..69606831791d1 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -192,21 +192,21 @@ req_backoff_interval_ms = 1000 req_backoff_max_delay_ms = 10000 req_backoff_factor = 2 upload_attempt_timeout_ms = 8000 -upload_retry_attempts = 3 -streaming_upload_attempt_timeout_ms = 5000 -streaming_upload_retry_attempts = 3 -read_attempt_timeout_ms = 8000 -read_retry_attempts = 3 +upload_retry_attempts = 5 +streaming_upload_attempt_timeout_ms = 10000 +streaming_upload_retry_attempts = 5 +read_attempt_timeout_ms = 16000 +read_retry_attempts = 5 streaming_read_attempt_timeout_ms = 3000 -streaming_read_retry_attempts = 3 +streaming_read_retry_attempts = 5 metadata_attempt_timeout_ms = 60000 -metadata_retry_attempts = 3 +metadata_retry_attempts = 5 delete_attempt_timeout_ms = 5000 -delete_retry_attempts = 3 +delete_retry_attempts = 5 delete_objects_attempt_timeout_ms = 5000 -delete_objects_retry_attempts = 3 +delete_objects_retry_attempts = 5 list_attempt_timeout_ms = 600000 -list_retry_attempts = 3 +list_retry_attempts = 5 [storage.object_store.s3] object_store_keepalive_ms = 600000 From ee8b3123caf9bdda5711336d1a59b5e5395cbe8e Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 16 May 2024 22:26:14 +0800 Subject: [PATCH 08/12] chore(ci): bump object store timeout (#16782) --- src/config/ci.toml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/config/ci.toml b/src/config/ci.toml index 02e1509546f27..43e9836bd304f 100644 --- a/src/config/ci.toml +++ b/src/config/ci.toml @@ -16,6 +16,12 @@ stream_exchange_concurrent_barriers = 10 [storage] imm_merge_threshold = 2 +[storage.object_store.retry] +streaming_upload_attempt_timeout_ms = 10000 +upload_retry_attempts = 5 +read_attempt_timeout_ms = 16000 +read_retry_attempts = 6 + [system] barrier_interval_ms = 250 checkpoint_frequency = 5 From 5ef4b2826613f8c7d69ff052136733fa5dcb05db Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 16 May 2024 23:05:00 +0800 Subject: [PATCH 09/12] chore(storage): revert bump `object_store_read_attempt_timeout_ms` from `8s` to `16s`" (#16789) --- ci/workflows/pull-request.yml | 2 +- src/common/src/config.rs | 6 +++--- src/config/example.toml | 20 ++++++++++---------- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 4c7308289ae73..475b981a18b07 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -97,7 +97,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 23 + timeout_in_minutes: 21 retry: *auto-retry - label: "end-to-end test (parallel)" diff --git a/src/common/src/config.rs b/src/common/src/config.rs index f4258481dfd17..fd457c7faf1ca 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -1751,7 +1751,7 @@ pub mod default { pub mod object_store_config { const DEFAULT_REQ_BACKOFF_INTERVAL_MS: u64 = 1000; // 1s const DEFAULT_REQ_BACKOFF_MAX_DELAY_MS: u64 = 10 * 1000; // 10s - const DEFAULT_REQ_MAX_RETRY_ATTEMPTS: usize = 5; + const DEFAULT_REQ_MAX_RETRY_ATTEMPTS: usize = 3; pub fn object_store_set_atomic_write_dir() -> bool { false @@ -1779,7 +1779,7 @@ pub mod default { // init + upload_part + finish pub fn object_store_streaming_upload_attempt_timeout_ms() -> u64 { - 10 * 1000 // 10s + 5 * 1000 // 5s } pub fn object_store_streaming_upload_retry_attempts() -> usize { @@ -1788,7 +1788,7 @@ pub mod default { // tips: depend on block_size pub fn object_store_read_attempt_timeout_ms() -> u64 { - 16 * 1000 // 16s + 8 * 1000 // 8s } pub fn object_store_read_retry_attempts() -> usize { diff --git a/src/config/example.toml b/src/config/example.toml index 69606831791d1..abdb1e77353fe 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -192,21 +192,21 @@ req_backoff_interval_ms = 1000 req_backoff_max_delay_ms = 10000 req_backoff_factor = 2 upload_attempt_timeout_ms = 8000 -upload_retry_attempts = 5 -streaming_upload_attempt_timeout_ms = 10000 -streaming_upload_retry_attempts = 5 -read_attempt_timeout_ms = 16000 -read_retry_attempts = 5 +upload_retry_attempts = 3 +streaming_upload_attempt_timeout_ms = 5000 +streaming_upload_retry_attempts = 3 +read_attempt_timeout_ms = 8000 +read_retry_attempts = 3 streaming_read_attempt_timeout_ms = 3000 -streaming_read_retry_attempts = 5 +streaming_read_retry_attempts = 3 metadata_attempt_timeout_ms = 60000 -metadata_retry_attempts = 5 +metadata_retry_attempts = 3 delete_attempt_timeout_ms = 5000 -delete_retry_attempts = 5 +delete_retry_attempts = 3 delete_objects_attempt_timeout_ms = 5000 -delete_objects_retry_attempts = 5 +delete_objects_retry_attempts = 3 list_attempt_timeout_ms = 600000 -list_retry_attempts = 5 +list_retry_attempts = 3 [storage.object_store.s3] object_store_keepalive_ms = 600000 From 30f1ebe4ee4d54f26b41d7e9fb351e3ea6826312 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 17 May 2024 11:32:42 +0800 Subject: [PATCH 10/12] fix(postgres-cdc): fix validation of PK with uppercase (#16793) --- e2e_test/source/cdc/cdc.load.slt | 2 +- e2e_test/source/cdc/postgres_cdc.sql | 4 ++-- .../risingwave/connector/source/common/PostgresValidator.java | 3 +-- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/e2e_test/source/cdc/cdc.load.slt b/e2e_test/source/cdc/cdc.load.slt index 6de27bfe8e5da..2c372cbd3ffdd 100644 --- a/e2e_test/source/cdc/cdc.load.slt +++ b/e2e_test/source/cdc/cdc.load.slt @@ -161,7 +161,7 @@ create table shipments_2 ( # Test user-provided publication statement ok create table t1_rw ( - v1 int primary key, + "V1" int primary key, v3 varchar ) with ( connector = 'postgres-cdc', diff --git a/e2e_test/source/cdc/postgres_cdc.sql b/e2e_test/source/cdc/postgres_cdc.sql index 6a60644ad6980..b936fb7876ade 100644 --- a/e2e_test/source/cdc/postgres_cdc.sql +++ b/e2e_test/source/cdc/postgres_cdc.sql @@ -31,8 +31,8 @@ INSERT INTO person VALUES (1001, 'peter white', 'myckhsp@xpmpe.com', '1781 2313 INSERT INTO person VALUES (1002, 'sarah spencer', 'wipvdbm@dkaap.com', '3453 4987 9481 6270', 'los angeles'); create schema abs; -create table abs.t1 (v1 int primary key, v2 double precision, v3 varchar, v4 numeric); -create publication my_publicaton for table abs.t1 (v1, v3); +create table abs.t1 ("V1" int primary key, v2 double precision, v3 varchar, v4 numeric); +create publication my_publicaton for table abs.t1 ("V1", v3); insert into abs.t1 values (1, 1.1, 'aaa', '5431.1234'); diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/PostgresValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/PostgresValidator.java index d91e5f885d609..084843672cb6f 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/PostgresValidator.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/PostgresValidator.java @@ -180,8 +180,7 @@ private void validateTableSchema() throws SQLException { var pkFields = new HashSet(); while (res.next()) { var name = res.getString(1); - // RisingWave always use lower case for column name - pkFields.add(name.toLowerCase()); + pkFields.add(name); } if (!ValidatorUtils.isPrimaryKeyMatch(tableSchema, pkFields)) { From 31cc7c04c9e644a88b6f183429dbea9c2db0a27d Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 17 May 2024 14:45:52 +0800 Subject: [PATCH 11/12] feat(stream): add debug logs for join key with high amplification (#16799) --- ci/workflows/pull-request.yml | 2 +- src/stream/src/executor/hash_join.rs | 13 +++++++++++++ src/stream/src/executor/join/hash_join.rs | 4 ++++ 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 475b981a18b07..4c7308289ae73 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -97,7 +97,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 21 + timeout_in_minutes: 23 retry: *auto-retry - label: "end-to-end test (parallel)" diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 60e2e532eb1fa..427b036fe7fe1 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -829,6 +829,19 @@ impl HashJoinExecutor 10000 { + let join_key_data_types = side_update.ht.join_key_data_types(); + let key = key.deserialize(join_key_data_types)?; + tracing::debug!(target: "hash_join_amplification", + matched_rows_len = rows.len(), + update_table_id = side_update.ht.table_id(), + match_table_id = side_match.ht.table_id(), + join_key = ?key, + actor_id = ctx.id, + fragment_id = ctx.fragment_id, + "large rows matched for join key" + ); + } } else { join_matched_join_keys.observe(0.0) } diff --git a/src/stream/src/executor/join/hash_join.rs b/src/stream/src/executor/join/hash_join.rs index 8805ea3cc3120..0c6a12b1e68f5 100644 --- a/src/stream/src/executor/join/hash_join.rs +++ b/src/stream/src/executor/join/hash_join.rs @@ -626,6 +626,10 @@ impl JoinHashMap { pub fn table_id(&self) -> u32 { self.state.table.table_id() } + + pub fn join_key_data_types(&self) -> &[DataType] { + &self.join_key_data_types + } } use risingwave_common_estimate_size::KvSize; From 347e186d1bef729709fddfd7c8e84af92ce3cc5d Mon Sep 17 00:00:00 2001 From: August Date: Fri, 17 May 2024 15:14:58 +0800 Subject: [PATCH 12/12] fix: missing specified meta backend for backwards compat test config (#16797) Co-authored-by: Noel Kwan --- backwards-compat-tests/scripts/run_local.sh | 28 ++++++++++++++++++--- ci/scripts/backwards-compat-test.sh | 16 +++++++++++- 2 files changed, 39 insertions(+), 5 deletions(-) diff --git a/backwards-compat-tests/scripts/run_local.sh b/backwards-compat-tests/scripts/run_local.sh index 20e820ef3e092..e2604e6a114e0 100755 --- a/backwards-compat-tests/scripts/run_local.sh +++ b/backwards-compat-tests/scripts/run_local.sh @@ -13,8 +13,11 @@ trap on_exit EXIT source backwards-compat-tests/scripts/utils.sh configure_rw() { -echo "--- Setting up cluster config" -cat < risedev-profiles.user.yml + VERSION="$1" + + echo "--- Setting up cluster config" + if version_le "$VERSION" "1.9.0"; then + cat < risedev-profiles.user.yml full-without-monitoring: steps: - use: minio @@ -28,6 +31,23 @@ full-without-monitoring: address: message_queue port: 29092 EOF + else + cat < risedev-profiles.user.yml + full-without-monitoring: + steps: + - use: minio + - use: etcd + - use: meta-node + meta-backend: etcd + - use: compute-node + - use: frontend + - use: compactor + - use: kafka + user-managed: true + address: message_queue + port: 29092 +EOF + fi cat < risedev-components.user.env RISEDEV_CONFIGURED=false @@ -58,11 +78,11 @@ main() { set -euo pipefail get_rw_versions setup_old_cluster - configure_rw + configure_rw "$OLD_VERSION" seed_old_cluster "$OLD_VERSION" setup_new_cluster - configure_rw + configure_rw "99.99.99" validate_new_cluster "$NEW_VERSION" } diff --git a/ci/scripts/backwards-compat-test.sh b/ci/scripts/backwards-compat-test.sh index 3ffb3fb8284a8..b701054fdabba 100755 --- a/ci/scripts/backwards-compat-test.sh +++ b/ci/scripts/backwards-compat-test.sh @@ -41,7 +41,8 @@ VERSION="$1" ENABLE_BUILD="$2" echo "--- Setting up cluster config" -cat < risedev-profiles.user.yml + if version_le "$VERSION" "1.9.0"; then + cat < risedev-profiles.user.yml full-without-monitoring: steps: - use: minio @@ -51,6 +52,19 @@ full-without-monitoring: - use: frontend - use: compactor EOF + else + cat < risedev-profiles.user.yml +full-without-monitoring: + steps: + - use: minio + - use: etcd + - use: meta-node + meta-backend: etcd + - use: compute-node + - use: frontend + - use: compactor +EOF + fi cat < risedev-components.user.env RISEDEV_CONFIGURED=true