Skip to content

Commit

Permalink
pageserver: improved synthetic size & find_gc_cutoff error handling (#…
Browse files Browse the repository at this point in the history
…8051)

## Problem

This PR refactors some error handling to avoid log spam on
tenant/timeline shutdown.

- "ignoring failure to find gc cutoffs: timeline shutting down." logs
(#8012)
- "synthetic_size_worker: failed to calculate synthetic size for tenant
...: Failed to refresh gc_info before gathering inputs: tenant shutting
down", for example here:
https://neon-github-public-dev.s3.amazonaws.com/reports/pr-8049/9502988669/index.html#suites/3fc871d9ee8127d8501d607e03205abb/1a074a66548bbcea

Closes: #8012

## Summary of changes

- Refactor: Add a PageReconstructError variant to GcError: this is the
only kind of error that find_gc_cutoffs can emit.
- Functional change: only ignore shutdown PageReconstructError variant:
for other variants, treat it as a real error
- Refactor: add a structured CalculateSyntheticSizeError type and use it
instead of anyhow::Error in synthetic size calculations
- Functional change: while iterating through timelines gathering logical
sizes, only drop out if the whole tenant is cancelled: individual
timeline cancellations indicate deletion in progress and we can just
ignore those.
  • Loading branch information
jcsp authored Jun 14, 2024
1 parent 6843fd8 commit eb0ca9b
Show file tree
Hide file tree
Showing 7 changed files with 115 additions and 71 deletions.
26 changes: 9 additions & 17 deletions pageserver/src/consumption_metrics.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,9 @@
//! and push them to a HTTP endpoint.
use crate::context::{DownloadBehavior, RequestContext};
use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME};
use crate::tenant::size::CalculateSyntheticSizeError;
use crate::tenant::tasks::BackgroundLoopKind;
use crate::tenant::{
mgr::TenantManager, LogicalSizeCalculationCause, PageReconstructError, Tenant,
};
use crate::tenant::{mgr::TenantManager, LogicalSizeCalculationCause, Tenant};
use camino::Utf8PathBuf;
use consumption_metrics::EventType;
use pageserver_api::models::TenantState;
Expand Down Expand Up @@ -350,19 +349,12 @@ async fn calculate_and_log(tenant: &Tenant, cancel: &CancellationToken, ctx: &Re
// Same for the loop that fetches computed metrics.
// By using the same limiter, we centralize metrics collection for "start" and "finished" counters,
// which turns out is really handy to understand the system.
let Err(e) = tenant.calculate_synthetic_size(CAUSE, cancel, ctx).await else {
return;
};

// this error can be returned if timeline is shutting down, but it does not
// mean the synthetic size worker should terminate.
let shutting_down = matches!(
e.downcast_ref::<PageReconstructError>(),
Some(PageReconstructError::Cancelled)
);

if !shutting_down {
let tenant_shard_id = tenant.tenant_shard_id();
error!("failed to calculate synthetic size for tenant {tenant_shard_id}: {e:#}");
match tenant.calculate_synthetic_size(CAUSE, cancel, ctx).await {
Ok(_) => {}
Err(CalculateSyntheticSizeError::Cancelled) => {}
Err(e) => {
let tenant_shard_id = tenant.tenant_shard_id();
error!("failed to calculate synthetic size for tenant {tenant_shard_id}: {e:#}");
}
}
}
9 changes: 5 additions & 4 deletions pageserver/src/http/routes.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1135,17 +1135,18 @@ async fn tenant_size_handler(
&ctx,
)
.await
.map_err(ApiError::InternalServerError)?;
.map_err(|e| match e {
crate::tenant::size::CalculateSyntheticSizeError::Cancelled => ApiError::ShuttingDown,
other => ApiError::InternalServerError(anyhow::anyhow!(other)),
})?;

let mut sizes = None;
let accepts_html = headers
.get(header::ACCEPT)
.map(|v| v == "text/html")
.unwrap_or_default();
if !inputs_only.unwrap_or(false) {
let storage_model = inputs
.calculate_model()
.map_err(ApiError::InternalServerError)?;
let storage_model = inputs.calculate_model();
let size = storage_model.calculate();

// If request header expects html, return html
Expand Down
37 changes: 21 additions & 16 deletions pageserver/src/tenant.rs
Original file line number Diff line number Diff line change
Expand Up @@ -509,11 +509,24 @@ pub(crate) enum GcError {
#[error(transparent)]
Remote(anyhow::Error),

// An error reading while calculating GC cutoffs
#[error(transparent)]
GcCutoffs(PageReconstructError),

// If GC was invoked for a particular timeline, this error means it didn't exist
#[error("timeline not found")]
TimelineNotFound,
}

impl From<PageReconstructError> for GcError {
fn from(value: PageReconstructError) -> Self {
match value {
PageReconstructError::Cancelled => Self::TimelineCancelled,
other => Self::GcCutoffs(other),
}
}
}

impl Tenant {
/// Yet another helper for timeline initialization.
///
Expand Down Expand Up @@ -2921,17 +2934,9 @@ impl Tenant {
.checked_sub(horizon)
.unwrap_or(Lsn(0));

let res = timeline.find_gc_cutoffs(cutoff, pitr, cancel, ctx).await;

match res {
Ok(cutoffs) => {
let old = gc_cutoffs.insert(timeline.timeline_id, cutoffs);
assert!(old.is_none());
}
Err(e) => {
tracing::warn!(timeline_id = %timeline.timeline_id, "ignoring failure to find gc cutoffs: {e:#}");
}
}
let cutoffs = timeline.find_gc_cutoffs(cutoff, pitr, cancel, ctx).await?;
let old = gc_cutoffs.insert(timeline.timeline_id, cutoffs);
assert!(old.is_none());
}

if !self.is_active() || self.cancel.is_cancelled() {
Expand Down Expand Up @@ -3553,7 +3558,7 @@ impl Tenant {
cause: LogicalSizeCalculationCause,
cancel: &CancellationToken,
ctx: &RequestContext,
) -> anyhow::Result<size::ModelInputs> {
) -> Result<size::ModelInputs, size::CalculateSyntheticSizeError> {
let logical_sizes_at_once = self
.conf
.concurrent_tenant_size_logical_size_queries
Expand All @@ -3568,8 +3573,8 @@ impl Tenant {
// See more for on the issue #2748 condenced out of the initial PR review.
let mut shared_cache = tokio::select! {
locked = self.cached_logical_sizes.lock() => locked,
_ = cancel.cancelled() => anyhow::bail!("cancelled"),
_ = self.cancel.cancelled() => anyhow::bail!("tenant is shutting down"),
_ = cancel.cancelled() => return Err(size::CalculateSyntheticSizeError::Cancelled),
_ = self.cancel.cancelled() => return Err(size::CalculateSyntheticSizeError::Cancelled),
};

size::gather_inputs(
Expand All @@ -3593,10 +3598,10 @@ impl Tenant {
cause: LogicalSizeCalculationCause,
cancel: &CancellationToken,
ctx: &RequestContext,
) -> anyhow::Result<u64> {
) -> Result<u64, size::CalculateSyntheticSizeError> {
let inputs = self.gather_size_inputs(None, cause, cancel, ctx).await?;

let size = inputs.calculate()?;
let size = inputs.calculate();

self.set_cached_synthetic_size(size);

Expand Down
106 changes: 79 additions & 27 deletions pageserver/src/tenant/size.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,15 +3,14 @@ use std::collections::hash_map::Entry;
use std::collections::{HashMap, HashSet};
use std::sync::Arc;

use anyhow::{bail, Context};
use tokio::sync::oneshot::error::RecvError;
use tokio::sync::Semaphore;
use tokio_util::sync::CancellationToken;

use crate::context::RequestContext;
use crate::pgdatadir_mapping::CalculateLogicalSizeError;

use super::{LogicalSizeCalculationCause, Tenant};
use super::{GcError, LogicalSizeCalculationCause, Tenant};
use crate::tenant::Timeline;
use utils::id::TimelineId;
use utils::lsn::Lsn;
Expand Down Expand Up @@ -43,6 +42,44 @@ pub struct SegmentMeta {
pub kind: LsnKind,
}

#[derive(thiserror::Error, Debug)]
pub(crate) enum CalculateSyntheticSizeError {
/// Something went wrong internally to the calculation of logical size at a particular branch point
#[error("Failed to calculated logical size on timeline {timeline_id} at {lsn}: {error}")]
LogicalSize {
timeline_id: TimelineId,
lsn: Lsn,
error: CalculateLogicalSizeError,
},

/// Something went wrong internally when calculating GC parameters at start of size calculation
#[error(transparent)]
GcInfo(GcError),

/// Totally unexpected errors, like panics joining a task
#[error(transparent)]
Fatal(anyhow::Error),

/// The LSN we are trying to calculate a size at no longer exists at the point we query it
#[error("Could not find size at {lsn} in timeline {timeline_id}")]
LsnNotFound { timeline_id: TimelineId, lsn: Lsn },

/// Tenant shut down while calculating size
#[error("Cancelled")]
Cancelled,
}

impl From<GcError> for CalculateSyntheticSizeError {
fn from(value: GcError) -> Self {
match value {
GcError::TenantCancelled | GcError::TimelineCancelled => {
CalculateSyntheticSizeError::Cancelled
}
other => CalculateSyntheticSizeError::GcInfo(other),
}
}
}

impl SegmentMeta {
fn size_needed(&self) -> bool {
match self.kind {
Expand Down Expand Up @@ -116,12 +153,9 @@ pub(super) async fn gather_inputs(
cause: LogicalSizeCalculationCause,
cancel: &CancellationToken,
ctx: &RequestContext,
) -> anyhow::Result<ModelInputs> {
) -> Result<ModelInputs, CalculateSyntheticSizeError> {
// refresh is needed to update gc related pitr_cutoff and horizon_cutoff
tenant
.refresh_gc_info(cancel, ctx)
.await
.context("Failed to refresh gc_info before gathering inputs")?;
tenant.refresh_gc_info(cancel, ctx).await?;

// Collect information about all the timelines
let mut timelines = tenant.list_timelines();
Expand Down Expand Up @@ -327,6 +361,12 @@ pub(super) async fn gather_inputs(
)
.await?;

if tenant.cancel.is_cancelled() {
// If we're shutting down, return an error rather than a sparse result that might include some
// timelines from before we started shutting down
return Err(CalculateSyntheticSizeError::Cancelled);
}

Ok(ModelInputs {
segments,
timeline_inputs,
Expand All @@ -345,7 +385,7 @@ async fn fill_logical_sizes(
logical_size_cache: &mut HashMap<(TimelineId, Lsn), u64>,
cause: LogicalSizeCalculationCause,
ctx: &RequestContext,
) -> anyhow::Result<()> {
) -> Result<(), CalculateSyntheticSizeError> {
let timeline_hash: HashMap<TimelineId, Arc<Timeline>> = HashMap::from_iter(
timelines
.iter()
Expand Down Expand Up @@ -387,7 +427,7 @@ async fn fill_logical_sizes(
}

// Perform the size lookups
let mut have_any_error = false;
let mut have_any_error = None;
while let Some(res) = joinset.join_next().await {
// each of these come with Result<anyhow::Result<_>, JoinError>
// because of spawn + spawn_blocking
Expand All @@ -398,21 +438,36 @@ async fn fill_logical_sizes(
Err(join_error) => {
// cannot really do anything, as this panic is likely a bug
error!("task that calls spawn_ondemand_logical_size_calculation panicked: {join_error:#}");
have_any_error = true;

have_any_error = Some(CalculateSyntheticSizeError::Fatal(
anyhow::anyhow!(join_error)
.context("task that calls spawn_ondemand_logical_size_calculation"),
));
}
Ok(Err(recv_result_error)) => {
// cannot really do anything, as this panic is likely a bug
error!("failed to receive logical size query result: {recv_result_error:#}");
have_any_error = true;
have_any_error = Some(CalculateSyntheticSizeError::Fatal(
anyhow::anyhow!(recv_result_error)
.context("Receiving logical size query result"),
));
}
Ok(Ok(TimelineAtLsnSizeResult(timeline, lsn, Err(error)))) => {
if !matches!(error, CalculateLogicalSizeError::Cancelled) {
if matches!(error, CalculateLogicalSizeError::Cancelled) {
// Skip this: it's okay if one timeline among many is shutting down while we
// calculate inputs for the overall tenant.
continue;
} else {
warn!(
timeline_id=%timeline.timeline_id,
"failed to calculate logical size at {lsn}: {error:#}"
);
have_any_error = Some(CalculateSyntheticSizeError::LogicalSize {
timeline_id: timeline.timeline_id,
lsn,
error,
});
}
have_any_error = true;
}
Ok(Ok(TimelineAtLsnSizeResult(timeline, lsn, Ok(size)))) => {
debug!(timeline_id=%timeline.timeline_id, %lsn, size, "size calculated");
Expand All @@ -426,10 +481,10 @@ async fn fill_logical_sizes(
// prune any keys not needed anymore; we record every used key and added key.
logical_size_cache.retain(|key, _| sizes_needed.contains_key(key));

if have_any_error {
if let Some(error) = have_any_error {
// we cannot complete this round, because we are missing data.
// we have however cached all we were able to request calculation on.
anyhow::bail!("failed to calculate some logical_sizes");
return Err(error);
}

// Insert the looked up sizes to the Segments
Expand All @@ -444,32 +499,29 @@ async fn fill_logical_sizes(
if let Some(Some(size)) = sizes_needed.get(&(timeline_id, lsn)) {
seg.segment.size = Some(*size);
} else {
bail!("could not find size at {} in timeline {}", lsn, timeline_id);
return Err(CalculateSyntheticSizeError::LsnNotFound { timeline_id, lsn });
}
}
Ok(())
}

impl ModelInputs {
pub fn calculate_model(&self) -> anyhow::Result<tenant_size_model::StorageModel> {
pub fn calculate_model(&self) -> tenant_size_model::StorageModel {
// Convert SegmentMetas into plain Segments
let storage = StorageModel {
StorageModel {
segments: self
.segments
.iter()
.map(|seg| seg.segment.clone())
.collect(),
};

Ok(storage)
}
}

// calculate total project size
pub fn calculate(&self) -> anyhow::Result<u64> {
let storage = self.calculate_model()?;
pub fn calculate(&self) -> u64 {
let storage = self.calculate_model();
let sizes = storage.calculate();

Ok(sizes.total_size)
sizes.total_size
}
}

Expand Down Expand Up @@ -656,7 +708,7 @@ fn verify_size_for_multiple_branches() {
"#;
let inputs: ModelInputs = serde_json::from_str(doc).unwrap();

assert_eq!(inputs.calculate().unwrap(), 37_851_408);
assert_eq!(inputs.calculate(), 37_851_408);
}

#[test]
Expand Down Expand Up @@ -711,7 +763,7 @@ fn verify_size_for_one_branch() {

let model: ModelInputs = serde_json::from_str(doc).unwrap();

let res = model.calculate_model().unwrap().calculate();
let res = model.calculate_model().calculate();

println!("calculated synthetic size: {}", res.total_size);
println!("result: {:?}", serde_json::to_string(&res.segments));
Expand Down
2 changes: 1 addition & 1 deletion pageserver/src/tenant/timeline.rs
Original file line number Diff line number Diff line change
Expand Up @@ -4823,7 +4823,7 @@ impl Timeline {
pitr: Duration,
cancel: &CancellationToken,
ctx: &RequestContext,
) -> anyhow::Result<GcCutoffs> {
) -> Result<GcCutoffs, PageReconstructError> {
let _timer = self
.metrics
.find_gc_cutoffs_histo
Expand Down
2 changes: 0 additions & 2 deletions test_runner/fixtures/pageserver/allowed_errors.py
Original file line number Diff line number Diff line change
Expand Up @@ -94,8 +94,6 @@ def scan_pageserver_log_for_errors(
".*WARN.*path=/v1/utilization .*request was dropped before completing",
# Can happen during shutdown
".*scheduling deletion on drop failed: queue is in state Stopped.*",
# Can happen during shutdown
".*ignoring failure to find gc cutoffs: timeline shutting down.*",
)


Expand Down
4 changes: 0 additions & 4 deletions test_runner/regress/test_tenant_size.py
Original file line number Diff line number Diff line change
Expand Up @@ -678,10 +678,6 @@ def test_synthetic_size_while_deleting(neon_env_builder: NeonEnvBuilder):
with pytest.raises(PageserverApiException, match=matcher):
completion.result()

# this happens on both cases
env.pageserver.allowed_errors.append(
".*ignoring failure to find gc cutoffs: timeline shutting down.*"
)
# this happens only in the case of deletion (http response logging)
env.pageserver.allowed_errors.append(".*Failed to refresh gc_info before gathering inputs.*")

Expand Down

1 comment on commit eb0ca9b

@github-actions
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

3304 tests run: 3150 passed, 3 failed, 151 skipped (full report)


Failures on Postgres 14

  • test_metric_collection: debug
  • test_pgbench_intensive_init_workload[neon_off-github-actions-selfhosted-1000]: release
  • test_pageserver_max_throughput_getpage_at_latest_lsn[github-actions-selfhosted-10-13-30]: release
# Run all failed tests locally:
scripts/pytest -vv -n $(nproc) -k "test_metric_collection[debug-pg14] or test_pgbench_intensive_init_workload[neon_off-release-pg14-github-actions-selfhosted-1000] or test_pageserver_max_throughput_getpage_at_latest_lsn[release-pg14-github-actions-selfhosted-10-13-30]"
Flaky tests (1)

Postgres 14

  • test_storage_controller_smoke: release

Test coverage report is not available

The comment gets automatically updated with the latest test results
eb0ca9b at 2024-06-14T11:42:21.675Z :recycle:

Please sign in to comment.