Timeline offloading persistence (#9444)

Persist timeline offloaded state to S3.

Right now, as of #8907, at each restart of the pageserver, all offloaded
state is lost, so we load the full timeline again. As it starts with an
empty local directory, we might potentially download some files again,
leading to downloads that are ultimately wasteful.

This patch adds support for persisting the offloaded state, allowing us
to never load offloaded timelines in the first place. The persistence
feature is facilitated via a new file in S3 that is tenant-global, which
contains a list of all offloaded timelines. It is updated each time we
offload or unoffload a timeline, and otherwise never touched.

This choice means that tenants where no offloading is happening will not
immediately get a manifest, keeping the change very minimal at the
start.

We leave generation support for future work. It is important to support
generations, as in the worst case, the manifest might be overwritten by
an older generation after a timeline has been unoffloaded (and
unarchived), so the next pageserver process instantiation might wrongly
believe that some timeline is still offloaded even though it should be
active.

Part of #9386, #8088
This commit is contained in:
Arpad Müller
2024-10-22 22:52:30 +02:00
committed by GitHub
parent fcb55a2aa2
commit 6f8fcdf9ea
11 changed files with 637 additions and 80 deletions

View File

@@ -699,6 +699,8 @@ pub struct OffloadedTimelineInfo {
pub ancestor_timeline_id: Option<TimelineId>,
/// Whether to retain the branch lsn at the ancestor or not
pub ancestor_retain_lsn: Option<Lsn>,
/// The time point when the timeline was archived
pub archived_at: chrono::DateTime<chrono::Utc>,
}
/// This represents the output of the "timeline_detail" and "timeline_list" API calls.

View File

@@ -486,6 +486,7 @@ fn build_timeline_offloaded_info(offloaded: &Arc<OffloadedTimeline>) -> Offloade
timeline_id,
ancestor_retain_lsn,
ancestor_timeline_id,
archived_at,
..
} = offloaded.as_ref();
OffloadedTimelineInfo {
@@ -493,6 +494,7 @@ fn build_timeline_offloaded_info(offloaded: &Arc<OffloadedTimeline>) -> Offloade
timeline_id,
ancestor_retain_lsn,
ancestor_timeline_id,
archived_at: archived_at.and_utc(),
}
}

View File

@@ -16,6 +16,7 @@ use anyhow::{bail, Context};
use arc_swap::ArcSwap;
use camino::Utf8Path;
use camino::Utf8PathBuf;
use chrono::NaiveDateTime;
use enumset::EnumSet;
use futures::stream::FuturesUnordered;
use futures::StreamExt;
@@ -31,6 +32,10 @@ use pageserver_api::shard::TenantShardId;
use remote_storage::DownloadError;
use remote_storage::GenericRemoteStorage;
use remote_storage::TimeoutOrCancel;
use remote_timeline_client::manifest::{
OffloadedTimelineManifest, TenantManifest, LATEST_TENANT_MANIFEST_VERSION,
};
use remote_timeline_client::UploadQueueNotReadyError;
use std::collections::BTreeMap;
use std::fmt;
use std::future::Future;
@@ -65,13 +70,14 @@ use self::config::TenantConf;
use self::metadata::TimelineMetadata;
use self::mgr::GetActiveTenantError;
use self::mgr::GetTenantError;
use self::remote_timeline_client::upload::upload_index_part;
use self::remote_timeline_client::upload::{upload_index_part, upload_tenant_manifest};
use self::remote_timeline_client::{RemoteTimelineClient, WaitCompletionError};
use self::timeline::uninit::TimelineCreateGuard;
use self::timeline::uninit::TimelineExclusionError;
use self::timeline::uninit::UninitializedTimeline;
use self::timeline::EvictionTaskTenantState;
use self::timeline::GcCutoffs;
use self::timeline::TimelineDeleteProgress;
use self::timeline::TimelineResources;
use self::timeline::WaitLsnError;
use crate::config::PageServerConf;
@@ -240,6 +246,7 @@ struct TimelinePreload {
}
pub(crate) struct TenantPreload {
tenant_manifest: TenantManifest,
timelines: HashMap<TimelineId, TimelinePreload>,
}
@@ -488,6 +495,12 @@ impl WalRedoManager {
}
}
/// A very lightweight memory representation of an offloaded timeline.
///
/// We need to store the list of offloaded timelines so that we can perform operations on them,
/// like unoffloading them, or (at a later date), decide to perform flattening.
/// This type has a much smaller memory impact than [`Timeline`], and thus we can store many
/// more offloaded timelines than we can manage ones that aren't.
pub struct OffloadedTimeline {
pub tenant_shard_id: TenantShardId,
pub timeline_id: TimelineId,
@@ -495,27 +508,78 @@ pub struct OffloadedTimeline {
/// Whether to retain the branch lsn at the ancestor or not
pub ancestor_retain_lsn: Option<Lsn>,
// TODO: once we persist offloaded state, make this lazily constructed
pub remote_client: Arc<RemoteTimelineClient>,
/// When the timeline was archived.
///
/// Present for future flattening deliberations.
pub archived_at: NaiveDateTime,
/// Lazily constructed remote client for the timeline
///
/// If we offload a timeline, we keep around the remote client
/// for the duration of the process. If we find it through the
/// manifest, we don't construct it up until it's needed (deletion).
pub remote_client: Option<Arc<RemoteTimelineClient>>,
/// Prevent two tasks from deleting the timeline at the same time. If held, the
/// timeline is being deleted. If 'true', the timeline has already been deleted.
pub delete_progress: Arc<tokio::sync::Mutex<DeleteTimelineFlow>>,
pub delete_progress: TimelineDeleteProgress,
}
impl OffloadedTimeline {
fn from_timeline(timeline: &Timeline) -> Self {
/// Obtains an offloaded timeline from a given timeline object.
///
/// Returns `None` if the `archived_at` flag couldn't be obtained, i.e.
/// the timeline is not in a stopped state.
/// Panics if the timeline is not archived.
fn from_timeline(timeline: &Timeline) -> Result<Self, UploadQueueNotReadyError> {
let ancestor_retain_lsn = timeline
.get_ancestor_timeline_id()
.map(|_timeline_id| timeline.get_ancestor_lsn());
Self {
let archived_at = timeline
.remote_client
.archived_at_stopped_queue()?
.expect("must be called on an archived timeline");
Ok(Self {
tenant_shard_id: timeline.tenant_shard_id,
timeline_id: timeline.timeline_id,
ancestor_timeline_id: timeline.get_ancestor_timeline_id(),
ancestor_retain_lsn,
archived_at,
remote_client: timeline.remote_client.clone(),
remote_client: Some(timeline.remote_client.clone()),
delete_progress: timeline.delete_progress.clone(),
})
}
fn from_manifest(tenant_shard_id: TenantShardId, manifest: &OffloadedTimelineManifest) -> Self {
let OffloadedTimelineManifest {
timeline_id,
ancestor_timeline_id,
ancestor_retain_lsn,
archived_at,
} = *manifest;
Self {
tenant_shard_id,
timeline_id,
ancestor_timeline_id,
ancestor_retain_lsn,
archived_at,
remote_client: None,
delete_progress: TimelineDeleteProgress::default(),
}
}
fn manifest(&self) -> OffloadedTimelineManifest {
let Self {
timeline_id,
ancestor_timeline_id,
ancestor_retain_lsn,
archived_at,
..
} = self;
OffloadedTimelineManifest {
timeline_id: *timeline_id,
ancestor_timeline_id: *ancestor_timeline_id,
ancestor_retain_lsn: *ancestor_retain_lsn,
archived_at: *archived_at,
}
}
}
@@ -551,10 +615,19 @@ impl TimelineOrOffloaded {
TimelineOrOffloaded::Offloaded(offloaded) => &offloaded.delete_progress,
}
}
pub fn remote_client(&self) -> &Arc<RemoteTimelineClient> {
pub fn remote_client_maybe_construct(&self, tenant: &Tenant) -> Arc<RemoteTimelineClient> {
match self {
TimelineOrOffloaded::Timeline(timeline) => &timeline.remote_client,
TimelineOrOffloaded::Offloaded(offloaded) => &offloaded.remote_client,
TimelineOrOffloaded::Timeline(timeline) => timeline.remote_client.clone(),
TimelineOrOffloaded::Offloaded(offloaded) => match offloaded.remote_client.clone() {
Some(remote_client) => remote_client,
None => {
let remote_client = tenant.build_timeline_client(
offloaded.timeline_id,
tenant.remote_storage.clone(),
);
Arc::new(remote_client)
}
},
}
}
}
@@ -1131,14 +1204,35 @@ impl Tenant {
cancel.clone(),
)
.await?;
let (offloaded_add, tenant_manifest) =
match remote_timeline_client::do_download_tenant_manifest(
remote_storage,
&self.tenant_shard_id,
&cancel,
)
.await
{
Ok((tenant_manifest, _generation)) => (
format!("{} offloaded", tenant_manifest.offloaded_timelines.len()),
tenant_manifest,
),
Err(DownloadError::NotFound) => {
("no manifest".to_string(), TenantManifest::empty())
}
Err(e) => Err(e)?,
};
info!("found {} timelines", remote_timeline_ids.len(),);
info!(
"found {} timelines, and {offloaded_add}",
remote_timeline_ids.len()
);
for k in other_keys {
warn!("Unexpected non timeline key {k}");
}
Ok(TenantPreload {
tenant_manifest,
timelines: self
.load_timelines_metadata(remote_timeline_ids, remote_storage, cancel)
.await?,
@@ -1163,12 +1257,26 @@ impl Tenant {
anyhow::bail!("local-only deployment is no longer supported, https://github.com/neondatabase/neon/issues/5624");
};
let mut offloaded_timeline_ids = HashSet::new();
let mut offloaded_timelines_list = Vec::new();
for timeline_manifest in preload.tenant_manifest.offloaded_timelines.iter() {
let timeline_id = timeline_manifest.timeline_id;
let offloaded_timeline =
OffloadedTimeline::from_manifest(self.tenant_shard_id, timeline_manifest);
offloaded_timelines_list.push((timeline_id, Arc::new(offloaded_timeline)));
offloaded_timeline_ids.insert(timeline_id);
}
let mut timelines_to_resume_deletions = vec![];
let mut remote_index_and_client = HashMap::new();
let mut timeline_ancestors = HashMap::new();
let mut existent_timelines = HashSet::new();
for (timeline_id, preload) in preload.timelines {
if offloaded_timeline_ids.remove(&timeline_id) {
// The timeline is offloaded, skip loading it.
continue;
}
let index_part = match preload.index_part {
Ok(i) => {
debug!("remote index part exists for timeline {timeline_id}");
@@ -1272,6 +1380,43 @@ impl Tenant {
.context("resume_deletion")
.map_err(LoadLocalTimelineError::ResumeDeletion)?;
}
// Complete deletions for offloaded timeline id's.
offloaded_timelines_list
.retain(|(offloaded_id, _offloaded)| {
// At this point, offloaded_timeline_ids has the list of all offloaded timelines
// without a prefix in S3, so they are inexistent.
// In the end, existence of a timeline is finally determined by the existence of an index-part.json in remote storage.
// If there is a dangling reference in another location, they need to be cleaned up.
let delete = offloaded_timeline_ids.contains(offloaded_id);
if delete {
tracing::info!("Removing offloaded timeline {offloaded_id} from manifest as no remote prefix was found");
}
!delete
});
if !offloaded_timelines_list.is_empty() {
tracing::info!(
"Tenant has {} offloaded timelines",
offloaded_timelines_list.len()
);
}
{
let mut offloaded_timelines_accessor = self.timelines_offloaded.lock().unwrap();
offloaded_timelines_accessor.extend(offloaded_timelines_list.into_iter());
}
if !offloaded_timeline_ids.is_empty() {
let manifest = self.tenant_manifest();
// TODO: generation support
let generation = remote_timeline_client::TENANT_MANIFEST_GENERATION;
upload_tenant_manifest(
&self.remote_storage,
&self.tenant_shard_id,
generation,
&manifest,
&self.cancel,
)
.await
.map_err(TimelineArchivalError::Other)?;
}
// The local filesystem contents are a cache of what's in the remote IndexPart;
// IndexPart is the source of truth.
@@ -1443,20 +1588,28 @@ impl Tenant {
Ok(timeline_preloads)
}
fn load_timeline_metadata(
self: &Arc<Tenant>,
fn build_timeline_client(
&self,
timeline_id: TimelineId,
remote_storage: GenericRemoteStorage,
cancel: CancellationToken,
) -> impl Future<Output = TimelinePreload> {
let client = RemoteTimelineClient::new(
) -> RemoteTimelineClient {
RemoteTimelineClient::new(
remote_storage.clone(),
self.deletion_queue_client.clone(),
self.conf,
self.tenant_shard_id,
timeline_id,
self.generation,
);
)
}
fn load_timeline_metadata(
self: &Arc<Tenant>,
timeline_id: TimelineId,
remote_storage: GenericRemoteStorage,
cancel: CancellationToken,
) -> impl Future<Output = TimelinePreload> {
let client = self.build_timeline_client(timeline_id, remote_storage);
async move {
debug_assert_current_span_has_tenant_and_timeline_id();
debug!("starting index part download");
@@ -1547,7 +1700,7 @@ impl Tenant {
info!("unoffloading timeline");
let cancel = self.cancel.clone();
let timeline_preload = self
.load_timeline_metadata(timeline_id, self.remote_storage.clone(), cancel)
.load_timeline_metadata(timeline_id, self.remote_storage.clone(), cancel.clone())
.await;
let index_part = match timeline_preload.index_part {
@@ -1592,17 +1745,37 @@ impl Tenant {
)
})
.map_err(TimelineArchivalError::Other)?;
let timelines = self.timelines.lock().unwrap();
let Some(timeline) = timelines.get(&timeline_id) else {
warn!("timeline not available directly after attach");
return Err(TimelineArchivalError::Other(anyhow::anyhow!(
"timeline not available directly after attach"
)));
let timeline = {
let timelines = self.timelines.lock().unwrap();
let Some(timeline) = timelines.get(&timeline_id) else {
warn!("timeline not available directly after attach");
// This is not a panic because no locks are held between `load_remote_timeline`
// which puts the timeline into timelines, and our look into the timeline map.
return Err(TimelineArchivalError::Other(anyhow::anyhow!(
"timeline not available directly after attach"
)));
};
let mut offloaded_timelines = self.timelines_offloaded.lock().unwrap();
if offloaded_timelines.remove(&timeline_id).is_none() {
warn!("timeline already removed from offloaded timelines");
}
Arc::clone(timeline)
};
let mut offloaded_timelines = self.timelines_offloaded.lock().unwrap();
if offloaded_timelines.remove(&timeline_id).is_none() {
warn!("timeline already removed from offloaded timelines");
}
// Upload new list of offloaded timelines to S3
let manifest = self.tenant_manifest();
// TODO: generation support
let generation = remote_timeline_client::TENANT_MANIFEST_GENERATION;
upload_tenant_manifest(
&self.remote_storage,
&self.tenant_shard_id,
generation,
&manifest,
&cancel,
)
.await
.map_err(TimelineArchivalError::Other)?;
// Activate the timeline (if it makes sense)
if !(timeline.is_broken() || timeline.is_stopping()) {
@@ -1616,7 +1789,7 @@ impl Tenant {
}
info!("timeline unoffloading complete");
Ok(Arc::clone(timeline))
Ok(timeline)
}
pub(crate) async fn apply_timeline_archival_config(
@@ -2793,6 +2966,26 @@ impl Tenant {
}
}
// TODO: also copy index files of offloaded timelines
let tenant_manifest = self.tenant_manifest();
// TODO: generation support
let generation = remote_timeline_client::TENANT_MANIFEST_GENERATION;
for child_shard in child_shards {
tracing::info!(
"Uploading tenant manifest for child {}",
child_shard.to_index()
);
upload_tenant_manifest(
&self.remote_storage,
child_shard,
generation,
&tenant_manifest,
&self.cancel,
)
.await?;
}
Ok(())
}
@@ -2970,6 +3163,22 @@ impl Tenant {
.unwrap_or(self.conf.default_tenant_conf.lsn_lease_length)
}
pub(crate) fn tenant_manifest(&self) -> TenantManifest {
let timelines_offloaded = self.timelines_offloaded.lock().unwrap();
let mut timeline_manifests = timelines_offloaded
.iter()
.map(|(_timeline_id, offloaded)| offloaded.manifest())
.collect::<Vec<_>>();
// Sort the manifests so that our output is deterministic
timeline_manifests.sort_by_key(|timeline_manifest| timeline_manifest.timeline_id);
TenantManifest {
version: LATEST_TENANT_MANIFEST_VERSION,
offloaded_timelines: timeline_manifests,
}
}
pub fn set_new_tenant_config(&self, new_tenant_conf: TenantConfOpt) {
// Use read-copy-update in order to avoid overwriting the location config
// state if this races with [`Tenant::set_new_location_config`]. Note that
@@ -3962,18 +4171,21 @@ impl Tenant {
Ok(timeline)
}
/// Call this before constructing a timeline, to build its required structures
fn build_timeline_resources(&self, timeline_id: TimelineId) -> TimelineResources {
let remote_client = RemoteTimelineClient::new(
fn build_timeline_remote_client(&self, timeline_id: TimelineId) -> RemoteTimelineClient {
RemoteTimelineClient::new(
self.remote_storage.clone(),
self.deletion_queue_client.clone(),
self.conf,
self.tenant_shard_id,
timeline_id,
self.generation,
);
)
}
/// Call this before constructing a timeline, to build its required structures
fn build_timeline_resources(&self, timeline_id: TimelineId) -> TimelineResources {
TimelineResources {
remote_client,
remote_client: self.build_timeline_remote_client(timeline_id),
timeline_get_throttle: self.timeline_get_throttle.clone(),
l0_flush_global_state: self.l0_flush_global_state.clone(),
}

View File

@@ -180,6 +180,7 @@
pub(crate) mod download;
pub mod index;
pub mod manifest;
pub(crate) mod upload;
use anyhow::Context;
@@ -191,7 +192,6 @@ use pageserver_api::models::TimelineArchivalState;
use pageserver_api::shard::{ShardIndex, TenantShardId};
use scopeguard::ScopeGuard;
use tokio_util::sync::CancellationToken;
pub(crate) use upload::upload_initdb_dir;
use utils::backoff::{
self, exponential_backoff, DEFAULT_BASE_BACKOFF_SECONDS, DEFAULT_MAX_BACKOFF_SECONDS,
};
@@ -245,9 +245,11 @@ use super::upload_queue::{NotInitialized, SetDeletedFlagProgress};
use super::Generation;
pub(crate) use download::{
download_index_part, is_temp_download_file, list_remote_tenant_shards, list_remote_timelines,
do_download_tenant_manifest, download_index_part, is_temp_download_file,
list_remote_tenant_shards, list_remote_timelines,
};
pub(crate) use index::LayerFileMetadata;
pub(crate) use upload::{upload_initdb_dir, upload_tenant_manifest};
// Occasional network issues and such can cause remote operations to fail, and
// that's expected. If a download fails, we log it at info-level, and retry.
@@ -272,6 +274,12 @@ pub(crate) const BUFFER_SIZE: usize = 32 * 1024;
/// which we warn and skip.
const DELETION_QUEUE_FLUSH_TIMEOUT: Duration = Duration::from_secs(10);
/// Hardcode a generation for the tenant manifest for now so that we don't
/// need to deal with generation-less manifests in the future.
///
/// TODO: add proper generation support to all the places that use this.
pub(crate) const TENANT_MANIFEST_GENERATION: Generation = Generation::new(1);
pub enum MaybeDeletedIndexPart {
IndexPart(IndexPart),
Deleted(IndexPart),
@@ -295,6 +303,10 @@ pub enum WaitCompletionError {
UploadQueueShutDownOrStopped,
}
#[derive(Debug, thiserror::Error)]
#[error("Upload queue either in unexpected state or hasn't downloaded manifest yet")]
pub struct UploadQueueNotReadyError;
/// A client for accessing a timeline's data in remote storage.
///
/// This takes care of managing the number of connections, and balancing them
@@ -468,6 +480,20 @@ impl RemoteTimelineClient {
.ok()
}
/// Returns `Ok(Some(timestamp))` if the timeline has been archived, `Ok(None)` if the timeline hasn't been archived.
///
/// Return Err(_) if the remote index_part hasn't been downloaded yet, or the timeline hasn't been stopped yet.
pub(crate) fn archived_at_stopped_queue(
&self,
) -> Result<Option<NaiveDateTime>, UploadQueueNotReadyError> {
self.upload_queue
.lock()
.unwrap()
.stopped_mut()
.map(|q| q.upload_queue_for_deletion.clean.0.archived_at)
.map_err(|_| UploadQueueNotReadyError)
}
fn update_remote_physical_size_gauge(&self, current_remote_index_part: Option<&IndexPart>) {
let size: u64 = if let Some(current_remote_index_part) = current_remote_index_part {
current_remote_index_part
@@ -2198,6 +2224,17 @@ pub fn remote_tenant_path(tenant_shard_id: &TenantShardId) -> RemotePath {
RemotePath::from_string(&path).expect("Failed to construct path")
}
pub fn remote_tenant_manifest_path(
tenant_shard_id: &TenantShardId,
generation: Generation,
) -> RemotePath {
let path = format!(
"tenants/{tenant_shard_id}/tenant-manifest{}.json",
generation.get_suffix()
);
RemotePath::from_string(&path).expect("Failed to construct path")
}
pub fn remote_timelines_path(tenant_shard_id: &TenantShardId) -> RemotePath {
let path = format!("tenants/{tenant_shard_id}/{TIMELINES_SEGMENT_NAME}");
RemotePath::from_string(&path).expect("Failed to construct path")

View File

@@ -34,10 +34,11 @@ use utils::id::{TenantId, TimelineId};
use utils::pausable_failpoint;
use super::index::{IndexPart, LayerFileMetadata};
use super::manifest::TenantManifest;
use super::{
parse_remote_index_path, remote_index_path, remote_initdb_archive_path,
remote_initdb_preserved_archive_path, remote_tenant_path, FAILED_DOWNLOAD_WARN_THRESHOLD,
FAILED_REMOTE_OP_RETRIES, INITDB_PATH,
remote_initdb_preserved_archive_path, remote_tenant_manifest_path, remote_tenant_path,
FAILED_DOWNLOAD_WARN_THRESHOLD, FAILED_REMOTE_OP_RETRIES, INITDB_PATH,
};
///
@@ -338,19 +339,15 @@ pub async fn list_remote_timelines(
list_identifiers::<TimelineId>(storage, remote_path, cancel).await
}
async fn do_download_index_part(
async fn do_download_remote_path_retry_forever(
storage: &GenericRemoteStorage,
tenant_shard_id: &TenantShardId,
timeline_id: &TimelineId,
index_generation: Generation,
remote_path: &RemotePath,
cancel: &CancellationToken,
) -> Result<(IndexPart, Generation, SystemTime), DownloadError> {
let remote_path = remote_index_path(tenant_shard_id, timeline_id, index_generation);
let (index_part_bytes, index_part_mtime) = download_retry_forever(
) -> Result<(Vec<u8>, SystemTime), DownloadError> {
download_retry_forever(
|| async {
let download = storage
.download(&remote_path, &DownloadOpts::default(), cancel)
.download(remote_path, &DownloadOpts::default(), cancel)
.await?;
let mut bytes = Vec::new();
@@ -365,7 +362,39 @@ async fn do_download_index_part(
&format!("download {remote_path:?}"),
cancel,
)
.await?;
.await
}
pub async fn do_download_tenant_manifest(
storage: &GenericRemoteStorage,
tenant_shard_id: &TenantShardId,
cancel: &CancellationToken,
) -> Result<(TenantManifest, Generation), DownloadError> {
// TODO: generation support
let generation = super::TENANT_MANIFEST_GENERATION;
let remote_path = remote_tenant_manifest_path(tenant_shard_id, generation);
let (manifest_bytes, _manifest_bytes_mtime) =
do_download_remote_path_retry_forever(storage, &remote_path, cancel).await?;
let tenant_manifest = TenantManifest::from_json_bytes(&manifest_bytes)
.with_context(|| format!("deserialize tenant manifest file at {remote_path:?}"))
.map_err(DownloadError::Other)?;
Ok((tenant_manifest, generation))
}
async fn do_download_index_part(
storage: &GenericRemoteStorage,
tenant_shard_id: &TenantShardId,
timeline_id: &TimelineId,
index_generation: Generation,
cancel: &CancellationToken,
) -> Result<(IndexPart, Generation, SystemTime), DownloadError> {
let remote_path = remote_index_path(tenant_shard_id, timeline_id, index_generation);
let (index_part_bytes, index_part_mtime) =
do_download_remote_path_retry_forever(storage, &remote_path, cancel).await?;
let index_part: IndexPart = serde_json::from_slice(&index_part_bytes)
.with_context(|| format!("deserialize index part file at {remote_path:?}"))

View File

@@ -0,0 +1,53 @@
use chrono::NaiveDateTime;
use serde::{Deserialize, Serialize};
use utils::{id::TimelineId, lsn::Lsn};
/// Tenant-shard scoped manifest
#[derive(Clone, Serialize, Deserialize)]
pub struct TenantManifest {
/// Debugging aid describing the version of this manifest.
/// Can also be used for distinguishing breaking changes later on.
pub version: usize,
/// The list of offloaded timelines together with enough information
/// to not have to actually load them.
///
/// Note: the timelines mentioned in this list might be deleted, i.e.
/// we don't hold an invariant that the references aren't dangling.
/// Existence of index-part.json is the actual indicator of timeline existence.
pub offloaded_timelines: Vec<OffloadedTimelineManifest>,
}
/// The remote level representation of an offloaded timeline.
///
/// Very similar to [`pageserver_api::models::OffloadedTimelineInfo`],
/// but the two datastructures serve different needs, this is for a persistent disk format
/// that must be backwards compatible, while the other is only for informative purposes.
#[derive(Clone, Serialize, Deserialize, Copy)]
pub struct OffloadedTimelineManifest {
pub timeline_id: TimelineId,
/// Whether the timeline has a parent it has been branched off from or not
pub ancestor_timeline_id: Option<TimelineId>,
/// Whether to retain the branch lsn at the ancestor or not
pub ancestor_retain_lsn: Option<Lsn>,
/// The time point when the timeline was archived
pub archived_at: NaiveDateTime,
}
pub const LATEST_TENANT_MANIFEST_VERSION: usize = 1;
impl TenantManifest {
pub(crate) fn empty() -> Self {
Self {
version: LATEST_TENANT_MANIFEST_VERSION,
offloaded_timelines: vec![],
}
}
pub(crate) fn from_json_bytes(bytes: &[u8]) -> Result<Self, serde_json::Error> {
serde_json::from_slice::<Self>(bytes)
}
pub(crate) fn to_json_bytes(&self) -> serde_json::Result<Vec<u8>> {
serde_json::to_vec(self)
}
}

View File

@@ -13,9 +13,11 @@ use tokio_util::sync::CancellationToken;
use utils::{backoff, pausable_failpoint};
use super::index::IndexPart;
use super::manifest::TenantManifest;
use super::Generation;
use crate::tenant::remote_timeline_client::{
remote_index_path, remote_initdb_archive_path, remote_initdb_preserved_archive_path,
remote_tenant_manifest_path,
};
use remote_storage::{GenericRemoteStorage, RemotePath, TimeTravelError};
use utils::id::{TenantId, TimelineId};
@@ -55,6 +57,37 @@ pub(crate) async fn upload_index_part<'a>(
.await
.with_context(|| format!("upload index part for '{tenant_shard_id} / {timeline_id}'"))
}
/// Serializes and uploads the given tenant manifest data to the remote storage.
pub(crate) async fn upload_tenant_manifest(
storage: &GenericRemoteStorage,
tenant_shard_id: &TenantShardId,
generation: Generation,
tenant_manifest: &TenantManifest,
cancel: &CancellationToken,
) -> anyhow::Result<()> {
tracing::trace!("uploading new tenant manifest");
fail_point!("before-upload-manifest", |_| {
bail!("failpoint before-upload-manifest")
});
pausable_failpoint!("before-upload-manifest-pausable");
let serialized = tenant_manifest.to_json_bytes()?;
let serialized = Bytes::from(serialized);
let tenant_manifest_site = serialized.len();
let remote_path = remote_tenant_manifest_path(tenant_shard_id, generation);
storage
.upload_storage_object(
futures::stream::once(futures::future::ready(Ok(serialized))),
tenant_manifest_site,
&remote_path,
cancel,
)
.await
.with_context(|| format!("upload tenant manifest for '{tenant_shard_id}'"))
}
/// Attempts to upload given layer files.
/// No extra checks for overlapping files is made and any files that are already present remotely will be overwritten, if submitted during the upload.

View File

@@ -371,7 +371,7 @@ pub struct Timeline {
/// Prevent two tasks from deleting the timeline at the same time. If held, the
/// timeline is being deleted. If 'true', the timeline has already been deleted.
pub delete_progress: Arc<tokio::sync::Mutex<DeleteTimelineFlow>>,
pub delete_progress: TimelineDeleteProgress,
eviction_task_timeline_state: tokio::sync::Mutex<EvictionTaskTimelineState>,
@@ -426,6 +426,8 @@ pub struct Timeline {
pub(crate) attach_wal_lag_cooldown: Arc<OnceLock<WalLagCooldown>>,
}
pub type TimelineDeleteProgress = Arc<tokio::sync::Mutex<DeleteTimelineFlow>>;
pub struct WalReceiverInfo {
pub wal_source_connconf: PgConnectionConfig,
pub last_received_msg_lsn: Lsn,
@@ -2250,7 +2252,7 @@ impl Timeline {
eviction_task_timeline_state: tokio::sync::Mutex::new(
EvictionTaskTimelineState::default(),
),
delete_progress: Arc::new(tokio::sync::Mutex::new(DeleteTimelineFlow::default())),
delete_progress: TimelineDeleteProgress::default(),
cancel,
gate: Gate::default(),

View File

@@ -14,7 +14,9 @@ use crate::{
task_mgr::{self, TaskKind},
tenant::{
metadata::TimelineMetadata,
remote_timeline_client::{PersistIndexPartWithDeletedFlagError, RemoteTimelineClient},
remote_timeline_client::{
self, PersistIndexPartWithDeletedFlagError, RemoteTimelineClient,
},
CreateTimelineCause, DeleteTimelineError, Tenant, TimelineOrOffloaded,
},
};
@@ -25,12 +27,9 @@ use super::{Timeline, TimelineResources};
/// during attach or pageserver restart.
/// See comment in persist_index_part_with_deleted_flag.
async fn set_deleted_in_remote_index(
timeline: &TimelineOrOffloaded,
remote_client: &Arc<RemoteTimelineClient>,
) -> Result<(), DeleteTimelineError> {
let res = timeline
.remote_client()
.persist_index_part_with_deleted_flag()
.await;
let res = remote_client.persist_index_part_with_deleted_flag().await;
match res {
// If we (now, or already) marked it successfully as deleted, we can proceed
Ok(()) | Err(PersistIndexPartWithDeletedFlagError::AlreadyDeleted(_)) => (),
@@ -129,12 +128,10 @@ pub(super) async fn delete_local_timeline_directory(
}
/// Removes remote layers and an index file after them.
async fn delete_remote_layers_and_index(timeline: &TimelineOrOffloaded) -> anyhow::Result<()> {
timeline
.remote_client()
.delete_all()
.await
.context("delete_all")
async fn delete_remote_layers_and_index(
remote_client: &Arc<RemoteTimelineClient>,
) -> anyhow::Result<()> {
remote_client.delete_all().await.context("delete_all")
}
/// It is important that this gets called when DeletionGuard is being held.
@@ -179,6 +176,32 @@ async fn remove_maybe_offloaded_timeline_from_tenant(
Ok(())
}
/// It is important that this gets called when DeletionGuard is being held.
/// For more context see comments in [`DeleteTimelineFlow::prepare`]
async fn upload_new_tenant_manifest(
tenant: &Tenant,
_: &DeletionGuard, // using it as a witness
) -> anyhow::Result<()> {
// This is susceptible to race conditions, i.e. we won't continue deletions if there is a crash
// between the deletion of the index-part.json and reaching of this code.
// So indeed, the tenant manifest might refer to an offloaded timeline which has already been deleted.
// However, we handle this case in tenant loading code so the next time we attach, the issue is
// resolved.
let manifest = tenant.tenant_manifest();
// TODO: generation support
let generation = remote_timeline_client::TENANT_MANIFEST_GENERATION;
remote_timeline_client::upload_tenant_manifest(
&tenant.remote_storage,
&tenant.tenant_shard_id,
generation,
&manifest,
&tenant.cancel,
)
.await?;
Ok(())
}
/// Orchestrates timeline shut down of all timeline tasks, removes its in-memory structures,
/// and deletes its data from both disk and s3.
/// The sequence of steps:
@@ -235,7 +258,8 @@ impl DeleteTimelineFlow {
))?
});
set_deleted_in_remote_index(&timeline).await?;
let remote_client = timeline.remote_client_maybe_construct(tenant);
set_deleted_in_remote_index(&remote_client).await?;
fail::fail_point!("timeline-delete-before-schedule", |_| {
Err(anyhow::anyhow!(
@@ -243,7 +267,13 @@ impl DeleteTimelineFlow {
))?
});
Self::schedule_background(guard, tenant.conf, Arc::clone(tenant), timeline);
Self::schedule_background(
guard,
tenant.conf,
Arc::clone(tenant),
timeline,
remote_client,
);
Ok(())
}
@@ -301,8 +331,9 @@ impl DeleteTimelineFlow {
guard.mark_in_progress()?;
let remote_client = timeline.remote_client.clone();
let timeline = TimelineOrOffloaded::Timeline(timeline);
Self::schedule_background(guard, tenant.conf, tenant, timeline);
Self::schedule_background(guard, tenant.conf, tenant, timeline, remote_client);
Ok(())
}
@@ -380,6 +411,7 @@ impl DeleteTimelineFlow {
conf: &'static PageServerConf,
tenant: Arc<Tenant>,
timeline: TimelineOrOffloaded,
remote_client: Arc<RemoteTimelineClient>,
) {
let tenant_shard_id = timeline.tenant_shard_id();
let timeline_id = timeline.timeline_id();
@@ -391,7 +423,7 @@ impl DeleteTimelineFlow {
Some(timeline_id),
"timeline_delete",
async move {
if let Err(err) = Self::background(guard, conf, &tenant, &timeline).await {
if let Err(err) = Self::background(guard, conf, &tenant, &timeline, remote_client).await {
error!("Error: {err:#}");
if let TimelineOrOffloaded::Timeline(timeline) = timeline {
timeline.set_broken(format!("{err:#}"))
@@ -408,6 +440,7 @@ impl DeleteTimelineFlow {
conf: &PageServerConf,
tenant: &Tenant,
timeline: &TimelineOrOffloaded,
remote_client: Arc<RemoteTimelineClient>,
) -> Result<(), DeleteTimelineError> {
// Offloaded timelines have no local state
// TODO: once we persist offloaded information, delete the timeline from there, too
@@ -415,12 +448,14 @@ impl DeleteTimelineFlow {
delete_local_timeline_directory(conf, tenant.tenant_shard_id, timeline).await?;
}
delete_remote_layers_and_index(timeline).await?;
delete_remote_layers_and_index(&remote_client).await?;
pausable_failpoint!("in_progress_delete");
remove_maybe_offloaded_timeline_from_tenant(tenant, timeline, &guard).await?;
upload_new_tenant_manifest(tenant, &guard).await?;
*guard = Self::Finished;
Ok(())

View File

@@ -1,17 +1,17 @@
use std::sync::Arc;
use crate::tenant::{OffloadedTimeline, Tenant, TimelineOrOffloaded};
use super::{
delete::{delete_local_timeline_directory, DeleteTimelineFlow, DeletionGuard},
Timeline,
};
use super::delete::{delete_local_timeline_directory, DeleteTimelineFlow, DeletionGuard};
use super::Timeline;
use crate::span::debug_assert_current_span_has_tenant_and_timeline_id;
use crate::tenant::{remote_timeline_client, OffloadedTimeline, Tenant, TimelineOrOffloaded};
pub(crate) async fn offload_timeline(
tenant: &Tenant,
timeline: &Arc<Timeline>,
) -> anyhow::Result<()> {
debug_assert_current_span_has_tenant_and_timeline_id();
tracing::info!("offloading archived timeline");
let (timeline, guard) = DeleteTimelineFlow::prepare(tenant, timeline.timeline_id)?;
let TimelineOrOffloaded::Timeline(timeline) = timeline else {
@@ -19,14 +19,28 @@ pub(crate) async fn offload_timeline(
return Ok(());
};
let is_archived = timeline.is_archived();
match is_archived {
Some(true) => (),
Some(false) => {
tracing::warn!(?is_archived, "tried offloading a non-archived timeline");
anyhow::bail!("timeline isn't archived");
}
None => {
tracing::warn!(
?is_archived,
"tried offloading a timeline where manifest is not yet available"
);
anyhow::bail!("timeline manifest hasn't been loaded yet");
}
}
// Now that the Timeline is in Stopping state, request all the related tasks to shut down.
timeline.shutdown(super::ShutdownMode::Hard).await;
// TODO extend guard mechanism above with method
// to make deletions possible while offloading is in progress
// TODO mark timeline as offloaded in S3
let conf = &tenant.conf;
delete_local_timeline_directory(conf, tenant.tenant_shard_id, &timeline).await?;
@@ -36,10 +50,31 @@ pub(crate) async fn offload_timeline(
let mut offloaded_timelines = tenant.timelines_offloaded.lock().unwrap();
offloaded_timelines.insert(
timeline.timeline_id,
Arc::new(OffloadedTimeline::from_timeline(&timeline)),
Arc::new(
OffloadedTimeline::from_timeline(&timeline)
.expect("we checked above that timeline was ready"),
),
);
}
// Last step: mark timeline as offloaded in S3
// TODO: maybe move this step above, right above deletion of the local timeline directory,
// then there is no potential race condition where we partially offload a timeline, and
// at the next restart attach it again.
// For that to happen, we'd need to make the manifest reflect our *intended* state,
// not our actual state of offloaded timelines.
let manifest = tenant.tenant_manifest();
// TODO: generation support
let generation = remote_timeline_client::TENANT_MANIFEST_GENERATION;
remote_timeline_client::upload_tenant_manifest(
&tenant.remote_storage,
&tenant.tenant_shard_id,
generation,
&manifest,
&tenant.cancel,
)
.await?;
Ok(())
}

View File

@@ -4,8 +4,11 @@ import pytest
from fixtures.common_types import TenantId, TimelineArchivalState, TimelineId
from fixtures.neon_fixtures import (
NeonEnvBuilder,
last_flush_lsn_upload,
)
from fixtures.pageserver.http import PageserverApiException
from fixtures.pageserver.utils import assert_prefix_empty, assert_prefix_not_empty
from fixtures.remote_storage import s3_storage
from fixtures.utils import wait_until
@@ -168,7 +171,7 @@ def test_timeline_offloading(neon_env_builder: NeonEnvBuilder, manual_offload: b
state=TimelineArchivalState.ARCHIVED,
)
def timeline_offloaded(timeline_id: TimelineId) -> bool:
def timeline_offloaded_logged(timeline_id: TimelineId) -> bool:
return (
env.pageserver.log_contains(f".*{timeline_id}.* offloading archived timeline.*")
is not None
@@ -186,12 +189,12 @@ def test_timeline_offloading(neon_env_builder: NeonEnvBuilder, manual_offload: b
def parent_offloaded():
if manual_offload:
ps_http.timeline_offload(tenant_id=tenant_id, timeline_id=parent_timeline_id)
assert timeline_offloaded(parent_timeline_id)
assert timeline_offloaded_logged(parent_timeline_id)
def leaf_offloaded():
if manual_offload:
ps_http.timeline_offload(tenant_id=tenant_id, timeline_id=leaf_timeline_id)
assert timeline_offloaded(leaf_timeline_id)
assert timeline_offloaded_logged(leaf_timeline_id)
wait_until(30, 1, leaf_offloaded)
wait_until(30, 1, parent_offloaded)
@@ -218,4 +221,118 @@ def test_timeline_offloading(neon_env_builder: NeonEnvBuilder, manual_offload: b
sum_again = endpoint.safe_psql("SELECT sum(key) from foo where key > 50")
assert sum == sum_again
assert not timeline_offloaded(initial_timeline_id)
assert not timeline_offloaded_logged(initial_timeline_id)
def test_timeline_offload_persist(neon_env_builder: NeonEnvBuilder):
"""
Test for persistence of timeline offload state
"""
remote_storage_kind = s3_storage()
neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind)
env = neon_env_builder.init_start()
ps_http = env.pageserver.http_client()
# Turn off gc and compaction loops: we want to issue them manually for better reliability
tenant_id, root_timeline_id = env.create_tenant(
conf={
"gc_period": "0s",
"compaction_period": "0s",
"checkpoint_distance": f"{1024 ** 2}",
}
)
# Create a branch and archive it
child_timeline_id = env.create_branch("test_archived_branch_persisted", tenant_id)
with env.endpoints.create_start(
"test_archived_branch_persisted", tenant_id=tenant_id
) as endpoint:
endpoint.safe_psql_many(
[
"CREATE TABLE foo(key serial primary key, t text default 'data_content')",
"INSERT INTO foo SELECT FROM generate_series(1,2048)",
]
)
sum = endpoint.safe_psql("SELECT sum(key) from foo where key < 500")
last_flush_lsn_upload(env, endpoint, tenant_id, child_timeline_id)
assert_prefix_not_empty(
neon_env_builder.pageserver_remote_storage,
prefix=f"tenants/{str(tenant_id)}/",
)
assert_prefix_empty(
neon_env_builder.pageserver_remote_storage,
prefix=f"tenants/{str(tenant_id)}/tenant-manifest",
)
ps_http.timeline_archival_config(
tenant_id,
child_timeline_id,
state=TimelineArchivalState.ARCHIVED,
)
leaf_detail = ps_http.timeline_detail(
tenant_id,
child_timeline_id,
)
assert leaf_detail["is_archived"] is True
def timeline_offloaded_api(timeline_id: TimelineId) -> bool:
# TODO add a proper API to check if a timeline has been offloaded or not
return not any(
timeline["timeline_id"] == str(timeline_id)
for timeline in ps_http.timeline_list(tenant_id=tenant_id)
)
def child_offloaded():
ps_http.timeline_offload(tenant_id=tenant_id, timeline_id=child_timeline_id)
assert timeline_offloaded_api(child_timeline_id)
wait_until(30, 1, child_offloaded)
assert timeline_offloaded_api(child_timeline_id)
assert not timeline_offloaded_api(root_timeline_id)
assert_prefix_not_empty(
neon_env_builder.pageserver_remote_storage,
prefix=f"tenants/{str(tenant_id)}/tenant-manifest",
)
# Test persistence, is the timeline still offloaded?
env.pageserver.stop()
env.pageserver.start()
assert timeline_offloaded_api(child_timeline_id)
assert not timeline_offloaded_api(root_timeline_id)
ps_http.timeline_archival_config(
tenant_id,
child_timeline_id,
state=TimelineArchivalState.UNARCHIVED,
)
child_detail = ps_http.timeline_detail(
tenant_id,
child_timeline_id,
)
assert child_detail["is_archived"] is False
with env.endpoints.create_start(
"test_archived_branch_persisted", tenant_id=tenant_id
) as endpoint:
sum_again = endpoint.safe_psql("SELECT sum(key) from foo where key < 500")
assert sum == sum_again
assert_prefix_empty(
neon_env_builder.pageserver_remote_storage,
prefix=f"tenants/{str(env.initial_tenant)}/tenant-manifest",
)
assert not timeline_offloaded_api(root_timeline_id)
ps_http.tenant_delete(tenant_id)
assert_prefix_empty(
neon_env_builder.pageserver_remote_storage,
prefix=f"tenants/{str(tenant_id)}/",
)