diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index ced4bb5af474..754b9814fffb 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -49,7 +49,6 @@ use self::config::AttachmentMode; use self::config::LocationConf; use self::config::TenantConf; use self::delete::DeleteTenantFlow; -use self::metadata::LoadMetadataError; use self::metadata::TimelineMetadata; use self::mgr::GetActiveTenantError; use self::mgr::GetTenantError; @@ -77,7 +76,6 @@ use crate::task_mgr; use crate::task_mgr::TaskKind; use crate::tenant::config::LocationMode; use crate::tenant::config::TenantConfOpt; -use crate::tenant::metadata::load_metadata; pub use crate::tenant::remote_timeline_client::index::IndexPart; use crate::tenant::remote_timeline_client::remote_initdb_archive_path; use crate::tenant::remote_timeline_client::MaybeDeletedIndexPart; @@ -94,7 +92,6 @@ use std::fmt::Debug; use std::fmt::Display; use std::fs; use std::fs::File; -use std::io; use std::ops::Bound::Included; use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering; @@ -488,11 +485,6 @@ impl From for InitdbError { } } -struct TenantDirectoryScan { - sorted_timelines_to_load: Vec<(TimelineId, TimelineMetadata)>, - timelines_to_resume_deletion: Vec<(TimelineId, Option)>, -} - enum CreateTimelineCause { Load, Delete, @@ -928,9 +920,7 @@ impl Tenant { timelines: HashMap::new(), }, (None, SpawnMode::Normal) => { - // Deprecated dev mode: load from local disk state instead of remote storage - // https://github.com/neondatabase/neon/issues/5624 - return self.load_local(ctx).await; + anyhow::bail!("local-only deployment is no longer supported, https://github.com/neondatabase/neon/issues/5624"); } }; @@ -1198,149 +1188,6 @@ impl Tenant { )) } - fn scan_and_sort_timelines_dir(self: Arc) -> anyhow::Result { - let mut timelines_to_load: HashMap = HashMap::new(); - // Note timelines_to_resume_deletion needs to be separate because it can be not sortable - // from the point of `tree_sort_timelines`. I e some parents can be missing because deletion - // completed in non topological order (for example because parent has smaller number of layer files in it) - let mut timelines_to_resume_deletion: Vec<(TimelineId, Option)> = vec![]; - - let timelines_dir = self.conf.timelines_path(&self.tenant_shard_id); - - for entry in timelines_dir - .read_dir_utf8() - .context("list timelines directory for tenant")? - { - let entry = entry.context("read timeline dir entry")?; - let timeline_dir = entry.path(); - - if crate::is_temporary(timeline_dir) { - info!("Found temporary timeline directory, removing: {timeline_dir}"); - if let Err(e) = std::fs::remove_dir_all(timeline_dir) { - error!("Failed to remove temporary directory '{timeline_dir}': {e:?}"); - } - } else if is_uninit_mark(timeline_dir) { - if !timeline_dir.exists() { - warn!("Timeline dir entry become invalid: {timeline_dir}"); - continue; - } - - let timeline_uninit_mark_file = &timeline_dir; - info!( - "Found an uninit mark file {timeline_uninit_mark_file}, removing the timeline and its uninit mark", - ); - let timeline_id = - TimelineId::try_from(timeline_uninit_mark_file.file_stem()) - .with_context(|| { - format!( - "Could not parse timeline id out of the timeline uninit mark name {timeline_uninit_mark_file}", - ) - })?; - let timeline_dir = self.conf.timeline_path(&self.tenant_shard_id, &timeline_id); - if let Err(e) = - remove_timeline_and_uninit_mark(&timeline_dir, timeline_uninit_mark_file) - { - error!("Failed to clean up uninit marked timeline: {e:?}"); - } - } else if crate::is_delete_mark(timeline_dir) { - // If metadata exists, load as usual, continue deletion - let timeline_id = TimelineId::try_from(timeline_dir.file_stem()) - .with_context(|| { - format!( - "Could not parse timeline id out of the timeline uninit mark name {timeline_dir}", - ) - })?; - - info!("Found deletion mark for timeline {}", timeline_id); - - match load_metadata(self.conf, &self.tenant_shard_id, &timeline_id) { - Ok(metadata) => { - timelines_to_resume_deletion.push((timeline_id, Some(metadata))) - } - Err(e) => match &e { - LoadMetadataError::Read(r) => { - if r.kind() != io::ErrorKind::NotFound { - return Err(anyhow::anyhow!(e)).with_context(|| { - format!("Failed to load metadata for timeline_id {timeline_id}") - }); - } - - // If metadata doesnt exist it means that we've crashed without - // completing cleanup_remaining_timeline_fs_traces in DeleteTimelineFlow. - // So save timeline_id for later call to `DeleteTimelineFlow::cleanup_remaining_timeline_fs_traces`. - // We cant do it here because the method is async so we'd need block_on - // and here we're in spawn_blocking. cleanup_remaining_timeline_fs_traces uses fs operations - // so that basically results in a cycle: - // spawn_blocking - // - block_on - // - spawn_blocking - // which can lead to running out of threads in blocing pool. - timelines_to_resume_deletion.push((timeline_id, None)); - } - _ => { - return Err(anyhow::anyhow!(e)).with_context(|| { - format!("Failed to load metadata for timeline_id {timeline_id}") - }) - } - }, - } - } else { - if !timeline_dir.exists() { - warn!("Timeline dir entry become invalid: {timeline_dir}"); - continue; - } - let timeline_id = TimelineId::try_from(timeline_dir.file_name()) - .with_context(|| { - format!( - "Could not parse timeline id out of the timeline dir name {timeline_dir}", - ) - })?; - let timeline_uninit_mark_file = self - .conf - .timeline_uninit_mark_file_path(self.tenant_shard_id, timeline_id); - if timeline_uninit_mark_file.exists() { - info!( - %timeline_id, - "Found an uninit mark file, removing the timeline and its uninit mark", - ); - if let Err(e) = - remove_timeline_and_uninit_mark(timeline_dir, &timeline_uninit_mark_file) - { - error!("Failed to clean up uninit marked timeline: {e:?}"); - } - continue; - } - - let timeline_delete_mark_file = self - .conf - .timeline_delete_mark_file_path(self.tenant_shard_id, timeline_id); - if timeline_delete_mark_file.exists() { - // Cleanup should be done in `is_delete_mark` branch above - continue; - } - - let file_name = entry.file_name(); - if let Ok(timeline_id) = file_name.parse::() { - let metadata = load_metadata(self.conf, &self.tenant_shard_id, &timeline_id) - .context("failed to load metadata")?; - timelines_to_load.insert(timeline_id, metadata); - } else { - // A file or directory that doesn't look like a timeline ID - warn!("unexpected file or directory in timelines directory: {file_name}"); - } - } - } - - // Sort the array of timeline IDs into tree-order, so that parent comes before - // all its children. - tree_sort_timelines(timelines_to_load, |m| m.ancestor_timeline()).map(|sorted_timelines| { - TenantDirectoryScan { - sorted_timelines_to_load: sorted_timelines, - timelines_to_resume_deletion, - } - }) - } - async fn load_timeline_metadata( self: &Arc, timeline_ids: HashSet, @@ -1404,141 +1251,6 @@ impl Tenant { Ok(timeline_preloads) } - /// - /// Background task to load in-memory data structures for this tenant, from - /// files on disk. Used at pageserver startup. - /// - /// No background tasks are started as part of this routine. - async fn load_local(self: &Arc, ctx: &RequestContext) -> anyhow::Result<()> { - span::debug_assert_current_span_has_tenant_id(); - - debug!("loading tenant task"); - - // Load in-memory state to reflect the local files on disk - // - // Scan the directory, peek into the metadata file of each timeline, and - // collect a list of timelines and their ancestors. - let span = info_span!("blocking"); - let cloned = Arc::clone(self); - - let scan = tokio::task::spawn_blocking(move || { - let _g = span.entered(); - cloned.scan_and_sort_timelines_dir() - }) - .await - .context("load spawn_blocking") - .and_then(|res| res)?; - - // FIXME original collect_timeline_files contained one more check: - // 1. "Timeline has no ancestor and no layer files" - - // Process loadable timelines first - for (timeline_id, local_metadata) in scan.sorted_timelines_to_load { - if let Err(e) = self - .load_local_timeline(timeline_id, local_metadata, ctx, false) - .await - { - match e { - LoadLocalTimelineError::Load(source) => { - return Err(anyhow::anyhow!(source)).with_context(|| { - format!("Failed to load local timeline: {timeline_id}") - }) - } - LoadLocalTimelineError::ResumeDeletion(source) => { - // Make sure resumed deletion wont fail loading for entire tenant. - error!("Failed to resume timeline deletion: {source:#}") - } - } - } - } - - // Resume deletion ones with deleted_mark - for (timeline_id, maybe_local_metadata) in scan.timelines_to_resume_deletion { - match maybe_local_metadata { - None => { - // See comment in `scan_and_sort_timelines_dir`. - if let Err(e) = - DeleteTimelineFlow::cleanup_remaining_timeline_fs_traces(self, timeline_id) - .await - { - warn!( - "cannot clean up deleted timeline dir timeline_id: {} error: {:#}", - timeline_id, e - ); - } - } - Some(local_metadata) => { - if let Err(e) = self - .load_local_timeline(timeline_id, local_metadata, ctx, true) - .await - { - match e { - LoadLocalTimelineError::Load(source) => { - // We tried to load deleted timeline, this is a bug. - return Err(anyhow::anyhow!(source).context( - format!("This is a bug. We tried to load deleted timeline which is wrong and loading failed. Timeline: {timeline_id}") - )); - } - LoadLocalTimelineError::ResumeDeletion(source) => { - // Make sure resumed deletion wont fail loading for entire tenant. - error!("Failed to resume timeline deletion: {source:#}") - } - } - } - } - } - } - - trace!("Done"); - - Ok(()) - } - - /// Subroutine of `load_tenant`, to load an individual timeline - /// - /// NB: The parent is assumed to be already loaded! - #[instrument(skip(self, local_metadata, ctx))] - async fn load_local_timeline( - self: &Arc, - timeline_id: TimelineId, - local_metadata: TimelineMetadata, - ctx: &RequestContext, - found_delete_mark: bool, - ) -> Result<(), LoadLocalTimelineError> { - span::debug_assert_current_span_has_tenant_id(); - - let resources = self.build_timeline_resources(timeline_id); - - if found_delete_mark { - // There is no remote client, we found local metadata. - // Continue cleaning up local disk. - DeleteTimelineFlow::resume_deletion( - Arc::clone(self), - timeline_id, - &local_metadata, - None, - self.deletion_queue_client.clone(), - ) - .await - .context("resume deletion") - .map_err(LoadLocalTimelineError::ResumeDeletion)?; - return Ok(()); - } - - let ancestor = if let Some(ancestor_timeline_id) = local_metadata.ancestor_timeline() { - let ancestor_timeline = self.get_timeline(ancestor_timeline_id, false) - .with_context(|| anyhow::anyhow!("cannot find ancestor timeline {ancestor_timeline_id} for timeline {timeline_id}")) - .map_err(LoadLocalTimelineError::Load)?; - Some(ancestor_timeline) - } else { - None - }; - - self.timeline_init_and_sync(timeline_id, resources, None, local_metadata, ancestor, ctx) - .await - .map_err(LoadLocalTimelineError::Load) - } - pub(crate) fn tenant_shard_id(&self) -> TenantShardId { self.tenant_shard_id } @@ -3787,29 +3499,6 @@ impl Tenant { } } -fn remove_timeline_and_uninit_mark( - timeline_dir: &Utf8Path, - uninit_mark: &Utf8Path, -) -> anyhow::Result<()> { - fs::remove_dir_all(timeline_dir) - .or_else(|e| { - if e.kind() == std::io::ErrorKind::NotFound { - // we can leave the uninit mark without a timeline dir, - // just remove the mark then - Ok(()) - } else { - Err(e) - } - }) - .with_context(|| { - format!("Failed to remove unit marked timeline directory {timeline_dir}") - })?; - fs::remove_file(uninit_mark) - .with_context(|| format!("Failed to remove timeline uninit mark file {uninit_mark}"))?; - - Ok(()) -} - /// Create the cluster temporarily in 'initdbpath' directory inside the repository /// to get bootstrap data for timeline initialization. async fn run_initdb( @@ -3969,13 +3658,6 @@ pub(crate) mod harness { } } - #[cfg(test)] - #[derive(Debug)] - enum LoadMode { - Local, - Remote, - } - pub struct TenantHarness { pub conf: &'static PageServerConf, pub tenant_conf: TenantConf, @@ -4057,42 +3739,17 @@ pub(crate) mod harness { pub(crate) async fn load(&self) -> (Arc, RequestContext) { let ctx = RequestContext::new(TaskKind::UnitTest, DownloadBehavior::Error); ( - self.try_load(&ctx) + self.do_try_load(&ctx) .await .expect("failed to load test tenant"), ctx, ) } - /// For tests that specifically want to exercise the local load path, which does - /// not use remote storage. - pub(crate) async fn try_load_local( + #[instrument(skip_all, fields(tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug()))] + pub(crate) async fn do_try_load( &self, ctx: &RequestContext, - ) -> anyhow::Result> { - self.do_try_load(ctx, LoadMode::Local).await - } - - /// The 'load' in this function is either a local load or a normal attachment, - pub(crate) async fn try_load(&self, ctx: &RequestContext) -> anyhow::Result> { - // If we have nothing in remote storage, must use load_local instead of attach: attach - // will error out if there are no timelines. - // - // See https://github.com/neondatabase/neon/issues/5456 for how we will eliminate - // this weird state of a Tenant which exists but doesn't have any timelines. - let mode = match self.remote_empty() { - true => LoadMode::Local, - false => LoadMode::Remote, - }; - - self.do_try_load(ctx, mode).await - } - - #[instrument(skip_all, fields(tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug(), ?mode))] - async fn do_try_load( - &self, - ctx: &RequestContext, - mode: LoadMode, ) -> anyhow::Result> { let walredo_mgr = Arc::new(WalRedoManager::from(TestRedoManager)); @@ -4113,17 +3770,10 @@ pub(crate) mod harness { self.deletion_queue.new_client(), )); - match mode { - LoadMode::Local => { - tenant.load_local(ctx).await?; - } - LoadMode::Remote => { - let preload = tenant - .preload(&self.remote_storage, CancellationToken::new()) - .await?; - tenant.attach(Some(preload), SpawnMode::Normal, ctx).await?; - } - } + let preload = tenant + .preload(&self.remote_storage, CancellationToken::new()) + .await?; + tenant.attach(Some(preload), SpawnMode::Normal, ctx).await?; tenant.state.send_replace(TenantState::Active); for timeline in tenant.timelines.lock().unwrap().values() { @@ -4132,31 +3782,6 @@ pub(crate) mod harness { Ok(tenant) } - fn remote_empty(&self) -> bool { - let tenant_path = self.conf.tenant_path(&self.tenant_shard_id); - let remote_tenant_dir = self - .remote_fs_dir - .join(tenant_path.strip_prefix(&self.conf.workdir).unwrap()); - if std::fs::metadata(&remote_tenant_dir).is_err() { - return true; - } - - match std::fs::read_dir(remote_tenant_dir) - .unwrap() - .flatten() - .next() - { - Some(entry) => { - tracing::debug!( - "remote_empty: not empty, found file {}", - entry.file_name().to_string_lossy(), - ); - false - } - None => true, - } - } - pub fn timeline_path(&self, timeline_id: &TimelineId) -> Utf8PathBuf { self.conf.timeline_path(&self.tenant_shard_id, timeline_id) } @@ -4215,7 +3840,6 @@ mod tests { use crate::repository::{Key, Value}; use crate::tenant::harness::*; use crate::DEFAULT_PG_VERSION; - use crate::METADATA_FILE_NAME; use bytes::BytesMut; use hex_literal::hex; use once_cell::sync::Lazy; @@ -4757,60 +4381,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn corrupt_local_metadata() -> anyhow::Result<()> { - const TEST_NAME: &str = "corrupt_metadata"; - let harness = TenantHarness::create(TEST_NAME)?; - let (tenant, ctx) = harness.load().await; - - let tline = tenant - .create_test_timeline(TIMELINE_ID, Lsn(0x10), DEFAULT_PG_VERSION, &ctx) - .await?; - drop(tline); - // so that all uploads finish & we can call harness.try_load() below again - tenant - .shutdown(Default::default(), true) - .instrument(harness.span()) - .await - .ok() - .unwrap(); - drop(tenant); - - // Corrupt local metadata - let metadata_path = harness.timeline_path(&TIMELINE_ID).join(METADATA_FILE_NAME); - assert!(metadata_path.is_file()); - let mut metadata_bytes = std::fs::read(&metadata_path)?; - assert_eq!(metadata_bytes.len(), 512); - metadata_bytes[8] ^= 1; - std::fs::write(metadata_path, metadata_bytes)?; - - let err = harness.try_load_local(&ctx).await.expect_err("should fail"); - // get all the stack with all .context, not only the last one - let message = format!("{err:#}"); - let expected = "failed to load metadata"; - assert!( - message.contains(expected), - "message '{message}' expected to contain {expected}" - ); - - let mut found_error_message = false; - let mut err_source = err.source(); - while let Some(source) = err_source { - if source.to_string().contains("metadata checksum mismatch") { - found_error_message = true; - break; - } - err_source = source.source(); - } - assert!( - found_error_message, - "didn't find the corrupted metadata error in {}", - message - ); - - Ok(()) - } - #[tokio::test] async fn test_images() -> anyhow::Result<()> { let (tenant, ctx) = TenantHarness::create("test_images")?.load().await; diff --git a/pageserver/src/tenant/metadata.rs b/pageserver/src/tenant/metadata.rs index dcbe781f908a..1a20a237a78e 100644 --- a/pageserver/src/tenant/metadata.rs +++ b/pageserver/src/tenant/metadata.rs @@ -294,17 +294,6 @@ pub enum LoadMetadataError { Decode(#[from] anyhow::Error), } -pub fn load_metadata( - conf: &'static PageServerConf, - tenant_shard_id: &TenantShardId, - timeline_id: &TimelineId, -) -> Result { - let metadata_path = conf.metadata_path(tenant_shard_id, timeline_id); - let metadata_bytes = std::fs::read(metadata_path)?; - - Ok(TimelineMetadata::from_bytes(&metadata_bytes)?) -} - #[cfg(test)] mod tests { use super::*; diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 87cf0ac6ea41..7f7713a6c6e5 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -4849,7 +4849,7 @@ mod tests { TenantHarness::create("two_layer_eviction_attempts_at_the_same_time").unwrap(); let ctx = any_context(); - let tenant = harness.try_load(&ctx).await.unwrap(); + let tenant = harness.do_try_load(&ctx).await.unwrap(); let timeline = tenant .create_test_timeline(TimelineId::generate(), Lsn(0x10), 14, &ctx) .await