fix(layer): metric splitting, span rename (#5902)

Per [feedback], split the Layer metrics, also finally account for lost
and [re-submitted feedback] on `layer_gc` by renaming it to
`layer_delete`, `Layer::garbage_collect_on_drop` renamed to
`Layer::delete_on_drop`. References to "gc" dropped from metric names
and elsewhere.

Also fixes how the cancellations were tracked: there was one rare
counter. Now there is a top level metric for cancelled inits, and the
rare "download failed but failed to communicate" counter is kept.

Fixes: #6027

[feedback]: https://github.com/neondatabase/neon/pull/5809#pullrequestreview-1720043251
[re-submitted feedback]: https://github.com/neondatabase/neon/pull/5108#discussion_r1401867311
This commit is contained in:
Joonas Koivunen
2023-12-07 11:39:40 +02:00
committed by GitHub
parent 10c77cb410
commit 52718bb8ff
5 changed files with 162 additions and 133 deletions

View File

@@ -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()

View File

@@ -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};

View File

@@ -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<Output = ()> + 'static {
@@ -423,8 +423,8 @@ struct LayerInner {
/// Initialization and deinitialization are done while holding a permit.
inner: heavier_once_cell::OnceCell<ResidentOrWantedEvicted>,
/// 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<Status>,
/// 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<LayerInner>, 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<ResidentLayer> 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<EvictionCancelled, IntCounter>,
started_gcs: IntCounter,
completed_gcs: IntCounter,
failed_gcs: IntCounterVec,
started_deletes: IntCounter,
completed_deletes: IntCounter,
failed_deletes: enum_map::EnumMap<DeleteFailed, IntCounter>,
rare_counters: IntCounterVec,
rare_counters: enum_map::EnumMap<RareEvent, IntCounter>,
inits_cancelled: metrics::core::GenericCounter<metrics::core::AtomicU64>,
}
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<LayerImplMetrics> =
#[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<LayerImplMetrics> =
once_cell::sync::Lazy::new(LayerImplMetrics::default);

View File

@@ -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 {

View File

@@ -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)