diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 6e311041ba..b684c4fa90 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -2094,6 +2094,8 @@ pub fn preinitialize_metrics() { // Tenant manager stats Lazy::force(&TENANT_MANAGER); + Lazy::force(&crate::tenant::storage_layer::layer::LAYER_IMPL_METRICS); + // countervecs [&BACKGROUND_LOOP_PERIOD_OVERRUN_COUNT] .into_iter() diff --git a/pageserver/src/tenant/storage_layer.rs b/pageserver/src/tenant/storage_layer.rs index 944e05883f..6e9a4932d8 100644 --- a/pageserver/src/tenant/storage_layer.rs +++ b/pageserver/src/tenant/storage_layer.rs @@ -4,7 +4,7 @@ pub mod delta_layer; mod filename; pub mod image_layer; mod inmemory_layer; -mod layer; +pub(crate) mod layer; mod layer_desc; use crate::context::{AccessStatsBehavior, RequestContext}; diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index e203d9d334..126d4d5563 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -222,8 +222,8 @@ impl Layer { /// /// [gc]: [`RemoteTimelineClient::schedule_gc_update`] /// [compaction]: [`RemoteTimelineClient::schedule_compaction_update`] - pub(crate) fn garbage_collect_on_drop(&self) { - self.0.garbage_collect_on_drop(); + pub(crate) fn delete_on_drop(&self) { + self.0.delete_on_drop(); } /// Return data needed to reconstruct given page at LSN. @@ -331,10 +331,10 @@ impl Layer { Ok(()) } - /// Waits until this layer has been dropped (and if needed, local garbage collection and remote + /// Waits until this layer has been dropped (and if needed, local file deletion and remote /// deletion scheduling has completed). /// - /// Does not start garbage collection, use [`Self::garbage_collect_on_drop`] for that + /// Does not start local deletion, use [`Self::delete_on_drop`] for that /// separatedly. #[cfg(feature = "testing")] pub(crate) fn wait_drop(&self) -> impl std::future::Future + 'static { @@ -423,8 +423,8 @@ struct LayerInner { /// Initialization and deinitialization are done while holding a permit. inner: heavier_once_cell::OnceCell, - /// Do we want to garbage collect this when `LayerInner` is dropped - wanted_garbage_collected: AtomicBool, + /// Do we want to delete locally and remotely this when `LayerInner` is dropped + wanted_deleted: AtomicBool, /// Do we want to evict this layer as soon as possible? After being set to `true`, all accesses /// will try to downgrade [`ResidentOrWantedEvicted`], which will eventually trigger @@ -438,10 +438,6 @@ struct LayerInner { version: AtomicUsize, /// Allow subscribing to when the layer actually gets evicted. - /// - /// If in future we need to implement "wait until layer instances are gone and done", carrying - /// this over to the gc spawn_blocking from LayerInner::drop will do the trick, and adding a - /// method for "wait_gc" which will wait to this being closed. status: tokio::sync::broadcast::Sender, /// Counter for exponential backoff with the download @@ -483,14 +479,14 @@ enum Status { impl Drop for LayerInner { fn drop(&mut self) { - if !*self.wanted_garbage_collected.get_mut() { + if !*self.wanted_deleted.get_mut() { // should we try to evict if the last wish was for eviction? // feels like there's some hazard of overcrowding near shutdown near by, but we don't // run drops during shutdown (yet) return; } - let span = tracing::info_span!(parent: None, "layer_gc", tenant_id = %self.layer_desc().tenant_shard_id.tenant_id, shard_id=%self.layer_desc().tenant_shard_id.shard_slug(), timeline_id = %self.layer_desc().timeline_id); + let span = tracing::info_span!(parent: None, "layer_delete", tenant_id = %self.layer_desc().tenant_shard_id.tenant_id, shard_id=%self.layer_desc().tenant_shard_id.shard_slug(), timeline_id = %self.layer_desc().timeline_id); let path = std::mem::take(&mut self.path); let file_name = self.layer_desc().filename(); @@ -517,8 +513,8 @@ impl Drop for LayerInner { false } Err(e) => { - tracing::error!("failed to remove garbage collected layer: {e}"); - LAYER_IMPL_METRICS.inc_gc_removes_failed(); + tracing::error!("failed to remove wanted deleted layer: {e}"); + LAYER_IMPL_METRICS.inc_delete_removes_failed(); false } }; @@ -540,15 +536,15 @@ impl Drop for LayerInner { } else { tracing::warn!("scheduling deletion on drop failed: {e:#}"); } - LAYER_IMPL_METRICS.inc_gcs_failed(GcFailed::DeleteSchedulingFailed); + LAYER_IMPL_METRICS.inc_deletes_failed(DeleteFailed::DeleteSchedulingFailed); } else { - LAYER_IMPL_METRICS.inc_completed_gcs(); + LAYER_IMPL_METRICS.inc_completed_deletes(); } } } else { // no need to nag that timeline is gone: under normal situation on // task_mgr::remove_tenant_from_memory the timeline is gone before we get dropped. - LAYER_IMPL_METRICS.inc_gcs_failed(GcFailed::TimelineGone); + LAYER_IMPL_METRICS.inc_deletes_failed(DeleteFailed::TimelineGone); } }); } @@ -583,7 +579,7 @@ impl LayerInner { timeline: Arc::downgrade(timeline), have_remote_client: timeline.remote_client.is_some(), access_stats, - wanted_garbage_collected: AtomicBool::new(false), + wanted_deleted: AtomicBool::new(false), wanted_evicted: AtomicBool::new(false), inner, version: AtomicUsize::new(version), @@ -594,16 +590,13 @@ impl LayerInner { } } - fn garbage_collect_on_drop(&self) { - let res = self.wanted_garbage_collected.compare_exchange( - false, - true, - Ordering::Release, - Ordering::Relaxed, - ); + fn delete_on_drop(&self) { + let res = + self.wanted_deleted + .compare_exchange(false, true, Ordering::Release, Ordering::Relaxed); if res.is_ok() { - LAYER_IMPL_METRICS.inc_started_gcs(); + LAYER_IMPL_METRICS.inc_started_deletes(); } } @@ -671,6 +664,10 @@ impl LayerInner { // disable any scheduled but not yet running eviction deletions for this let next_version = 1 + self.version.fetch_add(1, Ordering::Relaxed); + // count cancellations, which currently remain largely unexpected + let init_cancelled = + scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); + // no need to make the evict_and_wait wait for the actual download to complete drop(self.status.send(Status::Downloaded)); @@ -679,6 +676,8 @@ impl LayerInner { .upgrade() .ok_or_else(|| DownloadError::TimelineShutdown)?; + // FIXME: grab a gate + let can_ever_evict = timeline.remote_client.as_ref().is_some(); // check if we really need to be downloaded; could have been already downloaded by a @@ -739,6 +738,8 @@ impl LayerInner { tracing::info!(waiters, "completing the on-demand download for other tasks"); } + scopeguard::ScopeGuard::into_inner(init_cancelled); + Ok((ResidentOrWantedEvicted::Resident(res), permit)) }; @@ -867,14 +868,13 @@ impl LayerInner { match res { (Ok(()), _) => { // our caller is cancellation safe so this is fine; if someone - // else requests the layer, they'll find it already downloaded - // or redownload. + // else requests the layer, they'll find it already downloaded. // - // however, could be that we should consider marking the layer - // for eviction? alas, cannot: because only DownloadedLayer - // will handle that. - tracing::info!("layer file download completed after requester had cancelled"); - LAYER_IMPL_METRICS.inc_download_completed_without_requester(); + // See counter [`LayerImplMetrics::inc_init_needed_no_download`] + // + // FIXME(#6028): however, could be that we should consider marking the + // layer for eviction? alas, cannot: because only DownloadedLayer will + // handle that. }, (Err(e), _) => { // our caller is cancellation safe, but we might be racing with @@ -994,12 +994,15 @@ impl LayerInner { /// `DownloadedLayer` is being dropped, so it calls this method. fn on_downloaded_layer_drop(self: Arc, version: usize) { - let gc = self.wanted_garbage_collected.load(Ordering::Acquire); + let delete = self.wanted_deleted.load(Ordering::Acquire); let evict = self.wanted_evicted.load(Ordering::Acquire); let can_evict = self.have_remote_client; - if gc { - // do nothing now, only in LayerInner::drop + if delete { + // do nothing now, only in LayerInner::drop -- this was originally implemented because + // we could had already scheduled the deletion at the time. + // + // FIXME: this is not true anymore, we can safely evict wanted deleted files. } else if can_evict && evict { let span = tracing::info_span!(parent: None, "layer_evict", tenant_id = %self.desc.tenant_shard_id.tenant_id, shard_id = %self.desc.tenant_shard_id.shard_slug(), timeline_id = %self.desc.timeline_id, layer=%self, %version); @@ -1014,7 +1017,7 @@ impl LayerInner { crate::task_mgr::BACKGROUND_RUNTIME.spawn_blocking(move || { let _g = span.entered(); - // if LayerInner is already dropped here, do nothing because the garbage collection + // if LayerInner is already dropped here, do nothing because the delete on drop // has already ran while we were in queue let Some(this) = this.upgrade() else { LAYER_IMPL_METRICS.inc_eviction_cancelled(EvictionCancelled::LayerGone); @@ -1405,35 +1408,37 @@ impl From for Layer { } } -use metrics::{IntCounter, IntCounterVec}; +use metrics::IntCounter; -struct LayerImplMetrics { +pub(crate) struct LayerImplMetrics { started_evictions: IntCounter, completed_evictions: IntCounter, - cancelled_evictions: IntCounterVec, + cancelled_evictions: enum_map::EnumMap, - started_gcs: IntCounter, - completed_gcs: IntCounter, - failed_gcs: IntCounterVec, + started_deletes: IntCounter, + completed_deletes: IntCounter, + failed_deletes: enum_map::EnumMap, - rare_counters: IntCounterVec, + rare_counters: enum_map::EnumMap, + inits_cancelled: metrics::core::GenericCounter, } impl Default for LayerImplMetrics { fn default() -> Self { - let evictions = metrics::register_int_counter_vec!( - "pageserver_layer_evictions_count", - "Evictions started and completed in the Layer implementation", - &["state"] + use enum_map::Enum; + + // reminder: these will be pageserver_layer_* with "_total" suffix + + let started_evictions = metrics::register_int_counter!( + "pageserver_layer_started_evictions", + "Evictions started in the Layer implementation" + ) + .unwrap(); + let completed_evictions = metrics::register_int_counter!( + "pageserver_layer_completed_evictions", + "Evictions completed in the Layer implementation" ) .unwrap(); - - let started_evictions = evictions - .get_metric_with_label_values(&["started"]) - .unwrap(); - let completed_evictions = evictions - .get_metric_with_label_values(&["completed"]) - .unwrap(); let cancelled_evictions = metrics::register_int_counter_vec!( "pageserver_layer_cancelled_evictions_count", @@ -1442,24 +1447,36 @@ impl Default for LayerImplMetrics { ) .unwrap(); - // reminder: this will be pageserver_layer_gcs_count_total with "_total" suffix - let gcs = metrics::register_int_counter_vec!( - "pageserver_layer_gcs_count", - "Garbage collections started and completed in the Layer implementation", - &["state"] + let cancelled_evictions = enum_map::EnumMap::from_array(std::array::from_fn(|i| { + let reason = EvictionCancelled::from_usize(i); + let s = reason.as_str(); + cancelled_evictions.with_label_values(&[s]) + })); + + let started_deletes = metrics::register_int_counter!( + "pageserver_layer_started_deletes", + "Deletions on drop pending in the Layer implementation" + ) + .unwrap(); + let completed_deletes = metrics::register_int_counter!( + "pageserver_layer_completed_deletes", + "Deletions on drop completed in the Layer implementation" ) .unwrap(); - let started_gcs = gcs.get_metric_with_label_values(&["pending"]).unwrap(); - let completed_gcs = gcs.get_metric_with_label_values(&["completed"]).unwrap(); - - let failed_gcs = metrics::register_int_counter_vec!( - "pageserver_layer_failed_gcs_count", - "Different reasons for garbage collections to have failed", + let failed_deletes = metrics::register_int_counter_vec!( + "pageserver_layer_failed_deletes_count", + "Different reasons for deletions on drop to have failed", &["reason"] ) .unwrap(); + let failed_deletes = enum_map::EnumMap::from_array(std::array::from_fn(|i| { + let reason = DeleteFailed::from_usize(i); + let s = reason.as_str(); + failed_deletes.with_label_values(&[s]) + })); + let rare_counters = metrics::register_int_counter_vec!( "pageserver_layer_assumed_rare_count", "Times unexpected or assumed rare event happened", @@ -1467,16 +1484,29 @@ impl Default for LayerImplMetrics { ) .unwrap(); + let rare_counters = enum_map::EnumMap::from_array(std::array::from_fn(|i| { + let event = RareEvent::from_usize(i); + let s = event.as_str(); + rare_counters.with_label_values(&[s]) + })); + + let inits_cancelled = metrics::register_int_counter!( + "pageserver_layer_inits_cancelled_count", + "Times Layer initialization was cancelled", + ) + .unwrap(); + Self { started_evictions, completed_evictions, cancelled_evictions, - started_gcs, - completed_gcs, - failed_gcs, + started_deletes, + completed_deletes, + failed_deletes, rare_counters, + inits_cancelled, } } } @@ -1489,57 +1519,33 @@ impl LayerImplMetrics { self.completed_evictions.inc(); } fn inc_eviction_cancelled(&self, reason: EvictionCancelled) { - self.cancelled_evictions - .get_metric_with_label_values(&[reason.as_str()]) - .unwrap() - .inc() + self.cancelled_evictions[reason].inc() } - fn inc_started_gcs(&self) { - self.started_gcs.inc(); + fn inc_started_deletes(&self) { + self.started_deletes.inc(); } - fn inc_completed_gcs(&self) { - self.completed_gcs.inc(); + fn inc_completed_deletes(&self) { + self.completed_deletes.inc(); } - fn inc_gcs_failed(&self, reason: GcFailed) { - self.failed_gcs - .get_metric_with_label_values(&[reason.as_str()]) - .unwrap() - .inc(); + fn inc_deletes_failed(&self, reason: DeleteFailed) { + self.failed_deletes[reason].inc(); } - /// Counted separatedly from failed gcs because we will complete the gc attempt regardless of - /// failure to delete local file. - fn inc_gc_removes_failed(&self) { - self.rare_counters - .get_metric_with_label_values(&["gc_remove_failed"]) - .unwrap() - .inc(); + /// Counted separatedly from failed layer deletes because we will complete the layer deletion + /// attempt regardless of failure to delete local file. + fn inc_delete_removes_failed(&self) { + self.rare_counters[RareEvent::RemoveOnDropFailed].inc(); } - /// Expected rare because requires a race with `evict_blocking` and - /// `get_or_maybe_download`. + /// Expected rare because requires a race with `evict_blocking` and `get_or_maybe_download`. fn inc_retried_get_or_maybe_download(&self) { - self.rare_counters - .get_metric_with_label_values(&["retried_gomd"]) - .unwrap() - .inc(); + self.rare_counters[RareEvent::RetriedGetOrMaybeDownload].inc(); } - /// Expected rare because cancellations are unexpected - fn inc_download_completed_without_requester(&self) { - self.rare_counters - .get_metric_with_label_values(&["download_completed_without"]) - .unwrap() - .inc(); - } - - /// Expected rare because cancellations are unexpected + /// Expected rare because cancellations are unexpected, and failures are unexpected fn inc_download_failed_without_requester(&self) { - self.rare_counters - .get_metric_with_label_values(&["download_failed_without"]) - .unwrap() - .inc(); + self.rare_counters[RareEvent::DownloadFailedWithoutRequester].inc(); } /// The Weak in ResidentOrWantedEvicted::WantedEvicted was successfully upgraded. @@ -1547,37 +1553,30 @@ impl LayerImplMetrics { /// If this counter is always zero, we should replace ResidentOrWantedEvicted type with an /// Option. fn inc_raced_wanted_evicted_accesses(&self) { - self.rare_counters - .get_metric_with_label_values(&["raced_wanted_evicted"]) - .unwrap() - .inc(); + self.rare_counters[RareEvent::UpgradedWantedEvicted].inc(); } - /// These are only expected for [`Self::inc_download_completed_without_requester`] amount when + /// These are only expected for [`Self::inc_init_cancelled`] amount when /// running with remote storage. fn inc_init_needed_no_download(&self) { - self.rare_counters - .get_metric_with_label_values(&["init_needed_no_download"]) - .unwrap() - .inc(); + self.rare_counters[RareEvent::InitWithoutDownload].inc(); } /// Expected rare because all layer files should be readable and good fn inc_permanent_loading_failures(&self) { - self.rare_counters - .get_metric_with_label_values(&["permanent_loading_failure"]) - .unwrap() - .inc(); + self.rare_counters[RareEvent::PermanentLoadingFailure].inc(); } fn inc_broadcast_lagged(&self) { - self.rare_counters - .get_metric_with_label_values(&["broadcast_lagged"]) - .unwrap() - .inc(); + self.rare_counters[RareEvent::EvictAndWaitLagged].inc(); + } + + fn inc_init_cancelled(&self) { + self.inits_cancelled.inc() } } +#[derive(enum_map::Enum)] enum EvictionCancelled { LayerGone, TimelineGone, @@ -1606,19 +1605,47 @@ impl EvictionCancelled { } } -enum GcFailed { +#[derive(enum_map::Enum)] +enum DeleteFailed { TimelineGone, DeleteSchedulingFailed, } -impl GcFailed { +impl DeleteFailed { fn as_str(&self) -> &'static str { match self { - GcFailed::TimelineGone => "timeline_gone", - GcFailed::DeleteSchedulingFailed => "delete_scheduling_failed", + DeleteFailed::TimelineGone => "timeline_gone", + DeleteFailed::DeleteSchedulingFailed => "delete_scheduling_failed", } } } -static LAYER_IMPL_METRICS: once_cell::sync::Lazy = +#[derive(enum_map::Enum)] +enum RareEvent { + RemoveOnDropFailed, + RetriedGetOrMaybeDownload, + DownloadFailedWithoutRequester, + UpgradedWantedEvicted, + InitWithoutDownload, + PermanentLoadingFailure, + EvictAndWaitLagged, +} + +impl RareEvent { + fn as_str(&self) -> &'static str { + use RareEvent::*; + + match self { + RemoveOnDropFailed => "remove_on_drop_failed", + RetriedGetOrMaybeDownload => "retried_gomd", + DownloadFailedWithoutRequester => "download_failed_without", + UpgradedWantedEvicted => "raced_wanted_evicted", + InitWithoutDownload => "init_needed_no_download", + PermanentLoadingFailure => "permanent_loading_failure", + EvictAndWaitLagged => "broadcast_lagged", + } + } +} + +pub(crate) static LAYER_IMPL_METRICS: once_cell::sync::Lazy = once_cell::sync::Lazy::new(LayerImplMetrics::default); diff --git a/pageserver/src/tenant/timeline/layer_manager.rs b/pageserver/src/tenant/timeline/layer_manager.rs index dcd82949dd..e38f5be209 100644 --- a/pageserver/src/tenant/timeline/layer_manager.rs +++ b/pageserver/src/tenant/timeline/layer_manager.rs @@ -243,7 +243,7 @@ impl LayerManager { // map index without actually rebuilding the index. updates.remove_historic(desc); mapping.remove(layer); - layer.garbage_collect_on_drop(); + layer.delete_on_drop(); } pub(crate) fn contains(&self, layer: &Layer) -> bool { diff --git a/test_runner/regress/test_remote_storage.py b/test_runner/regress/test_remote_storage.py index 9c2bb2db11..67daf5f901 100644 --- a/test_runner/regress/test_remote_storage.py +++ b/test_runner/regress/test_remote_storage.py @@ -840,7 +840,7 @@ def test_compaction_waits_for_upload( ), "there should be one L1 after L0 => L1 compaction (without #5863 being fixed)" def layer_deletes_completed(): - m = client.get_metric_value("pageserver_layer_gcs_count_total", {"state": "completed"}) + m = client.get_metric_value("pageserver_layer_completed_deletes_total") if m is None: return 0 return int(m)