mirror of
https://github.com/neondatabase/neon.git
synced 2025-12-22 21:59:59 +00:00
Support tenant manifests in the scrubber (#9942)
Support tenant manifests in the storage scrubber: * list the manifests, order them by generation * delete all manifests except for the two most recent generations * for the latest manifest: try parsing it. I've tested this patch by running the against a staging bucket and it successfully deleted stuff (and avoided deleting the latest two generations). In follow-up work, we might want to also check some invariants of the manifest, as mentioned in #8088. Part of #9386 Part of #8088 --------- Co-authored-by: Christian Schwarz <christian@neon.tech>
This commit is contained in:
@@ -4,17 +4,21 @@ use itertools::Itertools;
|
||||
use pageserver::tenant::checks::check_valid_layermap;
|
||||
use pageserver::tenant::layer_map::LayerMap;
|
||||
use pageserver::tenant::remote_timeline_client::index::LayerFileMetadata;
|
||||
use pageserver::tenant::remote_timeline_client::manifest::TenantManifest;
|
||||
use pageserver_api::shard::ShardIndex;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use tracing::{info, warn};
|
||||
use utils::generation::Generation;
|
||||
use utils::id::TimelineId;
|
||||
use utils::shard::TenantShardId;
|
||||
|
||||
use crate::cloud_admin_api::BranchData;
|
||||
use crate::metadata_stream::stream_listing;
|
||||
use crate::{download_object_with_retries, RootTarget, TenantShardTimelineId};
|
||||
use futures_util::StreamExt;
|
||||
use pageserver::tenant::remote_timeline_client::{parse_remote_index_path, remote_layer_path};
|
||||
use pageserver::tenant::remote_timeline_client::{
|
||||
parse_remote_index_path, parse_remote_tenant_manifest_path, remote_layer_path,
|
||||
};
|
||||
use pageserver::tenant::storage_layer::LayerName;
|
||||
use pageserver::tenant::IndexPart;
|
||||
use remote_storage::{GenericRemoteStorage, ListingObject, RemotePath};
|
||||
@@ -527,3 +531,132 @@ async fn list_timeline_blobs_impl(
|
||||
unknown_keys,
|
||||
}))
|
||||
}
|
||||
|
||||
pub(crate) struct RemoteTenantManifestInfo {
|
||||
pub(crate) latest_generation: Option<Generation>,
|
||||
pub(crate) manifests: Vec<(Generation, ListingObject)>,
|
||||
}
|
||||
|
||||
pub(crate) enum ListTenantManifestResult {
|
||||
WithErrors {
|
||||
errors: Vec<(String, String)>,
|
||||
#[allow(dead_code)]
|
||||
unknown_keys: Vec<ListingObject>,
|
||||
},
|
||||
NoErrors(RemoteTenantManifestInfo),
|
||||
}
|
||||
|
||||
/// Lists the tenant manifests in remote storage and parses the latest one, returning a [`ListTenantManifestResult`] object.
|
||||
pub(crate) async fn list_tenant_manifests(
|
||||
remote_client: &GenericRemoteStorage,
|
||||
tenant_id: TenantShardId,
|
||||
root_target: &RootTarget,
|
||||
) -> anyhow::Result<ListTenantManifestResult> {
|
||||
let mut errors = Vec::new();
|
||||
let mut unknown_keys = Vec::new();
|
||||
|
||||
let mut tenant_root_target = root_target.tenant_root(&tenant_id);
|
||||
let original_prefix = tenant_root_target.prefix_in_bucket.clone();
|
||||
const TENANT_MANIFEST_STEM: &str = "tenant-manifest";
|
||||
tenant_root_target.prefix_in_bucket += TENANT_MANIFEST_STEM;
|
||||
tenant_root_target.delimiter = String::new();
|
||||
|
||||
let mut manifests: Vec<(Generation, ListingObject)> = Vec::new();
|
||||
|
||||
let prefix_str = &original_prefix
|
||||
.strip_prefix("/")
|
||||
.unwrap_or(&original_prefix);
|
||||
|
||||
let mut stream = std::pin::pin!(stream_listing(remote_client, &tenant_root_target));
|
||||
'outer: while let Some(obj) = stream.next().await {
|
||||
let (key, Some(obj)) = obj? else {
|
||||
panic!("ListingObject not specified");
|
||||
};
|
||||
|
||||
'err: {
|
||||
// TODO a let chain would be nicer here.
|
||||
let Some(name) = key.object_name() else {
|
||||
break 'err;
|
||||
};
|
||||
if !name.starts_with(TENANT_MANIFEST_STEM) {
|
||||
break 'err;
|
||||
}
|
||||
let Some(generation) = parse_remote_tenant_manifest_path(key.clone()) else {
|
||||
break 'err;
|
||||
};
|
||||
tracing::debug!("tenant manifest {key}");
|
||||
manifests.push((generation, obj));
|
||||
continue 'outer;
|
||||
}
|
||||
tracing::info!("Listed an unknown key: {key}");
|
||||
unknown_keys.push(obj);
|
||||
}
|
||||
|
||||
if manifests.is_empty() {
|
||||
tracing::debug!("No manifest for timeline.");
|
||||
|
||||
return Ok(ListTenantManifestResult::WithErrors {
|
||||
errors,
|
||||
unknown_keys,
|
||||
});
|
||||
}
|
||||
if !unknown_keys.is_empty() {
|
||||
errors.push(((*prefix_str).to_owned(), "unknown keys listed".to_string()));
|
||||
|
||||
return Ok(ListTenantManifestResult::WithErrors {
|
||||
errors,
|
||||
unknown_keys,
|
||||
});
|
||||
}
|
||||
|
||||
// Find the manifest with the highest generation
|
||||
let (latest_generation, latest_listing_object) = manifests
|
||||
.iter()
|
||||
.max_by_key(|i| i.0)
|
||||
.map(|(g, obj)| (*g, obj.clone()))
|
||||
.unwrap();
|
||||
|
||||
let manifest_bytes =
|
||||
match download_object_with_retries(remote_client, &latest_listing_object.key).await {
|
||||
Ok(bytes) => bytes,
|
||||
Err(e) => {
|
||||
// It is possible that the tenant gets deleted in-between we list the objects
|
||||
// and we download the manifest file.
|
||||
errors.push((
|
||||
latest_listing_object.key.get_path().as_str().to_owned(),
|
||||
format!("failed to download tenant-manifest.json: {e}"),
|
||||
));
|
||||
return Ok(ListTenantManifestResult::WithErrors {
|
||||
errors,
|
||||
unknown_keys,
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
match TenantManifest::from_json_bytes(&manifest_bytes) {
|
||||
Ok(_manifest) => {
|
||||
return Ok(ListTenantManifestResult::NoErrors(
|
||||
RemoteTenantManifestInfo {
|
||||
latest_generation: Some(latest_generation),
|
||||
manifests,
|
||||
},
|
||||
));
|
||||
}
|
||||
Err(parse_error) => errors.push((
|
||||
latest_listing_object.key.get_path().as_str().to_owned(),
|
||||
format!("tenant-manifest.json body parsing error: {parse_error}"),
|
||||
)),
|
||||
}
|
||||
|
||||
if errors.is_empty() {
|
||||
errors.push((
|
||||
(*prefix_str).to_owned(),
|
||||
"Unexpected: no errors did not lead to a successfully parsed blob return".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
Ok(ListTenantManifestResult::WithErrors {
|
||||
errors,
|
||||
unknown_keys,
|
||||
})
|
||||
}
|
||||
|
||||
@@ -2,12 +2,16 @@ use std::collections::{BTreeMap, BTreeSet, HashMap};
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
|
||||
use crate::checks::{list_timeline_blobs, BlobDataParseResult};
|
||||
use crate::checks::{
|
||||
list_tenant_manifests, list_timeline_blobs, BlobDataParseResult, ListTenantManifestResult,
|
||||
};
|
||||
use crate::metadata_stream::{stream_tenant_timelines, stream_tenants};
|
||||
use crate::{init_remote, BucketConfig, NodeKind, RootTarget, TenantShardTimelineId, MAX_RETRIES};
|
||||
use futures_util::{StreamExt, TryStreamExt};
|
||||
use pageserver::tenant::remote_timeline_client::index::LayerFileMetadata;
|
||||
use pageserver::tenant::remote_timeline_client::{parse_remote_index_path, remote_layer_path};
|
||||
use pageserver::tenant::remote_timeline_client::{
|
||||
parse_remote_index_path, parse_remote_tenant_manifest_path, remote_layer_path,
|
||||
};
|
||||
use pageserver::tenant::storage_layer::LayerName;
|
||||
use pageserver::tenant::IndexPart;
|
||||
use pageserver_api::controller_api::TenantDescribeResponse;
|
||||
@@ -25,6 +29,7 @@ use utils::id::{TenantId, TenantTimelineId};
|
||||
#[derive(Serialize, Default)]
|
||||
pub struct GcSummary {
|
||||
indices_deleted: usize,
|
||||
tenant_manifests_deleted: usize,
|
||||
remote_storage_errors: usize,
|
||||
controller_api_errors: usize,
|
||||
ancestor_layers_deleted: usize,
|
||||
@@ -34,12 +39,14 @@ impl GcSummary {
|
||||
fn merge(&mut self, other: Self) {
|
||||
let Self {
|
||||
indices_deleted,
|
||||
tenant_manifests_deleted,
|
||||
remote_storage_errors,
|
||||
ancestor_layers_deleted,
|
||||
controller_api_errors,
|
||||
} = other;
|
||||
|
||||
self.indices_deleted += indices_deleted;
|
||||
self.tenant_manifests_deleted += tenant_manifests_deleted;
|
||||
self.remote_storage_errors += remote_storage_errors;
|
||||
self.ancestor_layers_deleted += ancestor_layers_deleted;
|
||||
self.controller_api_errors += controller_api_errors;
|
||||
@@ -352,6 +359,69 @@ async fn maybe_delete_index(
|
||||
}
|
||||
}
|
||||
|
||||
async fn maybe_delete_tenant_manifest(
|
||||
remote_client: &GenericRemoteStorage,
|
||||
min_age: &Duration,
|
||||
latest_gen: Generation,
|
||||
obj: &ListingObject,
|
||||
mode: GcMode,
|
||||
summary: &mut GcSummary,
|
||||
) {
|
||||
// Validation: we will only delete things that parse cleanly
|
||||
let basename = obj.key.get_path().file_name().unwrap();
|
||||
let Some(candidate_generation) =
|
||||
parse_remote_tenant_manifest_path(RemotePath::from_string(basename).unwrap())
|
||||
else {
|
||||
// A strange key: we will not delete this because we don't understand it.
|
||||
tracing::warn!("Bad index key");
|
||||
return;
|
||||
};
|
||||
|
||||
// Validation: we will only delete manifests more than one generation old, and in fact we
|
||||
// should never be called with such recent generations.
|
||||
if candidate_generation >= latest_gen {
|
||||
tracing::warn!("Deletion candidate is >= latest generation, this is a bug!");
|
||||
return;
|
||||
} else if candidate_generation.next() == latest_gen {
|
||||
tracing::warn!("Deletion candidate is >= latest generation - 1, this is a bug!");
|
||||
return;
|
||||
}
|
||||
|
||||
if !is_old_enough(min_age, obj, summary) {
|
||||
return;
|
||||
}
|
||||
|
||||
if matches!(mode, GcMode::DryRun) {
|
||||
tracing::info!("Dry run: would delete this key");
|
||||
return;
|
||||
}
|
||||
|
||||
// All validations passed: erase the object
|
||||
let cancel = CancellationToken::new();
|
||||
match backoff::retry(
|
||||
|| remote_client.delete(&obj.key, &cancel),
|
||||
|_| false,
|
||||
3,
|
||||
MAX_RETRIES as u32,
|
||||
"maybe_delete_tenant_manifest",
|
||||
&cancel,
|
||||
)
|
||||
.await
|
||||
{
|
||||
None => {
|
||||
unreachable!("Using a dummy cancellation token");
|
||||
}
|
||||
Some(Ok(_)) => {
|
||||
tracing::info!("Successfully deleted tenant manifest");
|
||||
summary.tenant_manifests_deleted += 1;
|
||||
}
|
||||
Some(Err(e)) => {
|
||||
tracing::warn!("Failed to delete tenant manifest: {e}");
|
||||
summary.remote_storage_errors += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
async fn gc_ancestor(
|
||||
remote_client: &GenericRemoteStorage,
|
||||
@@ -451,13 +521,100 @@ async fn gc_ancestor(
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn gc_tenant_manifests(
|
||||
remote_client: &GenericRemoteStorage,
|
||||
min_age: Duration,
|
||||
target: &RootTarget,
|
||||
mode: GcMode,
|
||||
tenant_shard_id: TenantShardId,
|
||||
) -> anyhow::Result<GcSummary> {
|
||||
let mut gc_summary = GcSummary::default();
|
||||
match list_tenant_manifests(remote_client, tenant_shard_id, target).await? {
|
||||
ListTenantManifestResult::WithErrors {
|
||||
errors,
|
||||
unknown_keys: _,
|
||||
} => {
|
||||
for (_key, error) in errors {
|
||||
tracing::warn!(%tenant_shard_id, "list_tenant_manifests: {error}");
|
||||
}
|
||||
}
|
||||
ListTenantManifestResult::NoErrors(mut manifest_info) => {
|
||||
let Some(latest_gen) = manifest_info.latest_generation else {
|
||||
return Ok(gc_summary);
|
||||
};
|
||||
manifest_info
|
||||
.manifests
|
||||
.sort_by_key(|(generation, _obj)| *generation);
|
||||
// skip the two latest generations (they don't neccessarily have to be 1 apart from each other)
|
||||
let candidates = manifest_info.manifests.iter().rev().skip(2);
|
||||
for (_generation, key) in candidates {
|
||||
maybe_delete_tenant_manifest(
|
||||
remote_client,
|
||||
&min_age,
|
||||
latest_gen,
|
||||
key,
|
||||
mode,
|
||||
&mut gc_summary,
|
||||
)
|
||||
.instrument(
|
||||
info_span!("maybe_delete_tenant_manifest", %tenant_shard_id, ?latest_gen, %key.key),
|
||||
)
|
||||
.await;
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(gc_summary)
|
||||
}
|
||||
|
||||
async fn gc_timeline(
|
||||
remote_client: &GenericRemoteStorage,
|
||||
min_age: &Duration,
|
||||
target: &RootTarget,
|
||||
mode: GcMode,
|
||||
ttid: TenantShardTimelineId,
|
||||
accumulator: &Arc<std::sync::Mutex<TenantRefAccumulator>>,
|
||||
) -> anyhow::Result<GcSummary> {
|
||||
let mut summary = GcSummary::default();
|
||||
let data = list_timeline_blobs(remote_client, ttid, target).await?;
|
||||
|
||||
let (index_part, latest_gen, candidates) = match &data.blob_data {
|
||||
BlobDataParseResult::Parsed {
|
||||
index_part,
|
||||
index_part_generation,
|
||||
s3_layers: _s3_layers,
|
||||
} => (index_part, *index_part_generation, data.unused_index_keys),
|
||||
BlobDataParseResult::Relic => {
|
||||
// Post-deletion tenant location: don't try and GC it.
|
||||
return Ok(summary);
|
||||
}
|
||||
BlobDataParseResult::Incorrect {
|
||||
errors,
|
||||
s3_layers: _,
|
||||
} => {
|
||||
// Our primary purpose isn't to report on bad data, but log this rather than skipping silently
|
||||
tracing::warn!("Skipping timeline {ttid}, bad metadata: {errors:?}");
|
||||
return Ok(summary);
|
||||
}
|
||||
};
|
||||
|
||||
accumulator.lock().unwrap().update(ttid, index_part);
|
||||
|
||||
for key in candidates {
|
||||
maybe_delete_index(remote_client, min_age, latest_gen, &key, mode, &mut summary)
|
||||
.instrument(info_span!("maybe_delete_index", %ttid, ?latest_gen, %key.key))
|
||||
.await;
|
||||
}
|
||||
|
||||
Ok(summary)
|
||||
}
|
||||
|
||||
/// Physical garbage collection: removing unused S3 objects.
|
||||
///
|
||||
/// This is distinct from the garbage collection done inside the pageserver, which operates at a higher level
|
||||
/// (keys, layers). This type of garbage collection is about removing:
|
||||
/// - Objects that were uploaded but never referenced in the remote index (e.g. because of a shutdown between
|
||||
/// uploading a layer and uploading an index)
|
||||
/// - Index objects from historic generations
|
||||
/// - Index objects and tenant manifests from historic generations
|
||||
///
|
||||
/// This type of GC is not necessary for correctness: rather it serves to reduce wasted storage capacity, and
|
||||
/// make sure that object listings don't get slowed down by large numbers of garbage objects.
|
||||
@@ -470,6 +627,7 @@ pub async fn pageserver_physical_gc(
|
||||
) -> anyhow::Result<GcSummary> {
|
||||
let (remote_client, target) = init_remote(bucket_config.clone(), NodeKind::Pageserver).await?;
|
||||
|
||||
let remote_client = Arc::new(remote_client);
|
||||
let tenants = if tenant_shard_ids.is_empty() {
|
||||
futures::future::Either::Left(stream_tenants(&remote_client, &target))
|
||||
} else {
|
||||
@@ -484,59 +642,59 @@ pub async fn pageserver_physical_gc(
|
||||
let accumulator = Arc::new(std::sync::Mutex::new(TenantRefAccumulator::default()));
|
||||
|
||||
// Generate a stream of TenantTimelineId
|
||||
let timelines = tenants.map_ok(|t| stream_tenant_timelines(&remote_client, &target, t));
|
||||
let timelines = timelines.try_buffered(CONCURRENCY);
|
||||
let timelines = timelines.try_flatten();
|
||||
|
||||
// Generate a stream of S3TimelineBlobData
|
||||
async fn gc_timeline(
|
||||
remote_client: &GenericRemoteStorage,
|
||||
min_age: &Duration,
|
||||
target: &RootTarget,
|
||||
mode: GcMode,
|
||||
ttid: TenantShardTimelineId,
|
||||
accumulator: &Arc<std::sync::Mutex<TenantRefAccumulator>>,
|
||||
) -> anyhow::Result<GcSummary> {
|
||||
let mut summary = GcSummary::default();
|
||||
let data = list_timeline_blobs(remote_client, ttid, target).await?;
|
||||
|
||||
let (index_part, latest_gen, candidates) = match &data.blob_data {
|
||||
BlobDataParseResult::Parsed {
|
||||
index_part,
|
||||
index_part_generation,
|
||||
s3_layers: _s3_layers,
|
||||
} => (index_part, *index_part_generation, data.unused_index_keys),
|
||||
BlobDataParseResult::Relic => {
|
||||
// Post-deletion tenant location: don't try and GC it.
|
||||
return Ok(summary);
|
||||
}
|
||||
BlobDataParseResult::Incorrect {
|
||||
errors,
|
||||
s3_layers: _,
|
||||
} => {
|
||||
// Our primary purpose isn't to report on bad data, but log this rather than skipping silently
|
||||
tracing::warn!("Skipping timeline {ttid}, bad metadata: {errors:?}");
|
||||
return Ok(summary);
|
||||
}
|
||||
};
|
||||
|
||||
accumulator.lock().unwrap().update(ttid, index_part);
|
||||
|
||||
for key in candidates {
|
||||
maybe_delete_index(remote_client, min_age, latest_gen, &key, mode, &mut summary)
|
||||
.instrument(info_span!("maybe_delete_index", %ttid, ?latest_gen, %key.key))
|
||||
.await;
|
||||
}
|
||||
|
||||
Ok(summary)
|
||||
enum GcSummaryOrContent<T> {
|
||||
Content(T),
|
||||
GcSummary(GcSummary),
|
||||
}
|
||||
let timelines = tenants.map_ok(|tenant_shard_id| {
|
||||
let target_ref = ⌖
|
||||
let remote_client_ref = &remote_client;
|
||||
async move {
|
||||
let summaries_from_manifests = match gc_tenant_manifests(
|
||||
remote_client_ref,
|
||||
min_age,
|
||||
target_ref,
|
||||
mode,
|
||||
tenant_shard_id,
|
||||
)
|
||||
.await
|
||||
{
|
||||
Ok(gc_summary) => vec![Ok(GcSummaryOrContent::<TenantShardTimelineId>::GcSummary(
|
||||
gc_summary,
|
||||
))],
|
||||
Err(e) => {
|
||||
tracing::warn!(%tenant_shard_id, "Error in gc_tenant_manifests: {e}");
|
||||
Vec::new()
|
||||
}
|
||||
};
|
||||
stream_tenant_timelines(remote_client_ref, target_ref, tenant_shard_id)
|
||||
.await
|
||||
.map(|stream| {
|
||||
stream
|
||||
.map_ok(GcSummaryOrContent::Content)
|
||||
.chain(futures::stream::iter(summaries_from_manifests.into_iter()))
|
||||
})
|
||||
}
|
||||
});
|
||||
let timelines = std::pin::pin!(timelines.try_buffered(CONCURRENCY));
|
||||
let timelines = timelines.try_flatten();
|
||||
|
||||
let mut summary = GcSummary::default();
|
||||
|
||||
// Drain futures for per-shard GC, populating accumulator as a side effect
|
||||
{
|
||||
let timelines = timelines.map_ok(|ttid| {
|
||||
gc_timeline(&remote_client, &min_age, &target, mode, ttid, &accumulator)
|
||||
let timelines = timelines.map_ok(|summary_or_ttid| match summary_or_ttid {
|
||||
GcSummaryOrContent::Content(ttid) => futures::future::Either::Left(gc_timeline(
|
||||
&remote_client,
|
||||
&min_age,
|
||||
&target,
|
||||
mode,
|
||||
ttid,
|
||||
&accumulator,
|
||||
)),
|
||||
GcSummaryOrContent::GcSummary(gc_summary) => {
|
||||
futures::future::Either::Right(futures::future::ok(gc_summary))
|
||||
}
|
||||
});
|
||||
let mut timelines = std::pin::pin!(timelines.try_buffered(CONCURRENCY));
|
||||
|
||||
|
||||
Reference in New Issue
Block a user