diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index feb136384325..104234841c82 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -33,9 +33,7 @@ use utils::{ use crate::tenant::timeline::GetVectoredImpl; use crate::tenant::vectored_blob_io::MaxVectoredReadBytes; use crate::tenant::{config::TenantConfOpt, timeline::GetImpl}; -use crate::tenant::{ - TENANTS_SEGMENT_NAME, TENANT_DELETED_MARKER_FILE_NAME, TIMELINES_SEGMENT_NAME, -}; +use crate::tenant::{TENANTS_SEGMENT_NAME, TIMELINES_SEGMENT_NAME}; use crate::{disk_usage_eviction_task::DiskUsageEvictionTaskConfig, virtual_file::io_engine}; use crate::{tenant::config::TenantConf, virtual_file}; use crate::{ @@ -855,14 +853,6 @@ impl PageServerConf { ) } - pub(crate) fn tenant_deleted_mark_file_path( - &self, - tenant_shard_id: &TenantShardId, - ) -> Utf8PathBuf { - self.tenant_path(tenant_shard_id) - .join(TENANT_DELETED_MARKER_FILE_NAME) - } - pub fn traces_path(&self) -> Utf8PathBuf { self.workdir.join("traces") } diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index b5713a8cb441..cfa507fed000 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -329,14 +329,11 @@ impl From for ApiError { } } -impl From for ApiError { - fn from(value: crate::tenant::delete::DeleteTenantError) -> Self { - use crate::tenant::delete::DeleteTenantError::*; +impl From for ApiError { + fn from(value: crate::tenant::mgr::DeleteTenantError) -> Self { + use crate::tenant::mgr::DeleteTenantError::*; match value { - Get(g) => ApiError::from(g), - Timeline(t) => ApiError::from(t), SlotError(e) => e.into(), - SlotUpsertError(e) => e.into(), Other(o) => ApiError::InternalServerError(o), Cancelled => ApiError::ShuttingDown, } diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index ace95af10ac3..6a748f61e7e1 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -55,11 +55,9 @@ use self::config::AttachedLocationConfig; use self::config::AttachmentMode; use self::config::LocationConf; use self::config::TenantConf; -use self::delete::DeleteTenantFlow; use self::metadata::TimelineMetadata; use self::mgr::GetActiveTenantError; use self::mgr::GetTenantError; -use self::mgr::TenantsMap; use self::remote_timeline_client::upload::upload_index_part; use self::remote_timeline_client::RemoteTimelineClient; use self::timeline::uninit::TimelineCreateGuard; @@ -137,7 +135,6 @@ pub mod remote_timeline_client; pub mod storage_layer; pub mod config; -pub mod delete; pub mod mgr; pub mod secondary; pub mod tasks; @@ -161,8 +158,6 @@ pub const TENANTS_SEGMENT_NAME: &str = "tenants"; /// Parts of the `.neon/tenants//timelines/` directory prefix. pub const TIMELINES_SEGMENT_NAME: &str = "timelines"; -pub const TENANT_DELETED_MARKER_FILE_NAME: &str = "deleted"; - /// References to shared objects that are passed into each tenant, such /// as the shared remote storage client and process initialization state. #[derive(Clone)] @@ -207,7 +202,6 @@ struct TimelinePreload { } pub(crate) struct TenantPreload { - deleting: bool, timelines: HashMap, } @@ -286,8 +280,6 @@ pub struct Tenant { /// background warmup. pub(crate) activate_now_sem: tokio::sync::Semaphore, - pub(crate) delete_progress: Arc>, - // Cancellation token fires when we have entered shutdown(). This is a parent of // Timelines' cancellation token. pub(crate) cancel: CancellationToken, @@ -654,7 +646,6 @@ impl Tenant { attached_conf: AttachedTenantConf, shard_identity: ShardIdentity, init_order: Option, - tenants: &'static std::sync::RwLock, mode: SpawnMode, ctx: &RequestContext, ) -> anyhow::Result> { @@ -828,52 +819,6 @@ impl Tenant { // Remote preload is complete. drop(remote_load_completion); - let pending_deletion = { - match DeleteTenantFlow::should_resume_deletion( - conf, - preload.as_ref().map(|p| p.deleting).unwrap_or(false), - &tenant_clone, - ) - .await - { - Ok(should_resume_deletion) => should_resume_deletion, - Err(err) => { - make_broken(&tenant_clone, anyhow::anyhow!(err), BrokenVerbosity::Error); - return Ok(()); - } - } - }; - - info!("pending_deletion {}", pending_deletion.is_some()); - - if let Some(deletion) = pending_deletion { - // as we are no longer loading, signal completion by dropping - // the completion while we resume deletion - drop(_completion); - let background_jobs_can_start = - init_order.as_ref().map(|x| &x.background_jobs_can_start); - if let Some(background) = background_jobs_can_start { - info!("waiting for backgound jobs barrier"); - background.clone().wait().await; - info!("ready for backgound jobs barrier"); - } - - let deleted = DeleteTenantFlow::resume_from_attach( - deletion, - &tenant_clone, - preload, - tenants, - &ctx, - ) - .await; - - if let Err(e) = deleted { - make_broken(&tenant_clone, anyhow::anyhow!(e), BrokenVerbosity::Error); - } - - return Ok(()); - } - // We will time the duration of the attach phase unless this is a creation (attach will do no work) let attached = { let _attach_timer = match mode { @@ -931,21 +876,13 @@ impl Tenant { ) .await?; - let deleting = other_keys.contains(TENANT_DELETED_MARKER_FILE_NAME); - info!( - "found {} timelines, deleting={}", - remote_timeline_ids.len(), - deleting - ); + info!("found {} timelines", remote_timeline_ids.len(),); for k in other_keys { - if k != TENANT_DELETED_MARKER_FILE_NAME { - warn!("Unexpected non timeline key {k}"); - } + warn!("Unexpected non timeline key {k}"); } Ok(TenantPreload { - deleting, timelines: Self::load_timeline_metadata( self, remote_timeline_ids, @@ -974,7 +911,6 @@ impl Tenant { let preload = match (preload, mode) { (Some(p), _) => p, (None, SpawnMode::Create) => TenantPreload { - deleting: false, timelines: HashMap::new(), }, (None, _) => { @@ -2628,7 +2564,6 @@ impl Tenant { cached_synthetic_tenant_size: Arc::new(AtomicU64::new(0)), eviction_task_tenant_state: tokio::sync::Mutex::new(EvictionTaskTenantState::default()), activate_now_sem: tokio::sync::Semaphore::new(0), - delete_progress: Arc::new(tokio::sync::Mutex::new(DeleteTenantFlow::default())), cancel: CancellationToken::default(), gate: Gate::default(), timeline_get_throttle: Arc::new(throttle::Throttle::new( diff --git a/pageserver/src/tenant/delete.rs b/pageserver/src/tenant/delete.rs deleted file mode 100644 index d9da3157b7fb..000000000000 --- a/pageserver/src/tenant/delete.rs +++ /dev/null @@ -1,426 +0,0 @@ -use std::sync::Arc; - -use anyhow::Context; -use camino::{Utf8Path, Utf8PathBuf}; -use pageserver_api::{models::TenantState, shard::TenantShardId}; -use remote_storage::{GenericRemoteStorage, RemotePath, TimeoutOrCancel}; -use tokio::sync::OwnedMutexGuard; -use tokio_util::sync::CancellationToken; -use tracing::{error, Instrument}; - -use utils::{backoff, completion, crashsafe, fs_ext, id::TimelineId, pausable_failpoint}; - -use crate::{ - config::PageServerConf, - context::RequestContext, - task_mgr::{self}, - tenant::{ - mgr::{TenantSlot, TenantsMapRemoveResult}, - remote_timeline_client::remote_heatmap_path, - }, -}; - -use super::{ - mgr::{GetTenantError, TenantSlotError, TenantSlotUpsertError, TenantsMap}, - remote_timeline_client::{FAILED_REMOTE_OP_RETRIES, FAILED_UPLOAD_WARN_THRESHOLD}, - timeline::delete::DeleteTimelineFlow, - tree_sort_timelines, DeleteTimelineError, Tenant, TenantPreload, -}; - -#[derive(Debug, thiserror::Error)] -pub(crate) enum DeleteTenantError { - #[error("GetTenant {0}")] - Get(#[from] GetTenantError), - - #[error("Tenant map slot error {0}")] - SlotError(#[from] TenantSlotError), - - #[error("Tenant map slot upsert error {0}")] - SlotUpsertError(#[from] TenantSlotUpsertError), - - #[error("Timeline {0}")] - Timeline(#[from] DeleteTimelineError), - - #[error("Cancelled")] - Cancelled, - - #[error(transparent)] - Other(#[from] anyhow::Error), -} - -type DeletionGuard = tokio::sync::OwnedMutexGuard; - -fn remote_tenant_delete_mark_path( - conf: &PageServerConf, - tenant_shard_id: &TenantShardId, -) -> anyhow::Result { - let tenant_remote_path = conf - .tenant_path(tenant_shard_id) - .strip_prefix(&conf.workdir) - .context("Failed to strip workdir prefix") - .and_then(RemotePath::new) - .context("tenant path")?; - Ok(tenant_remote_path.join(Utf8Path::new("timelines/deleted"))) -} - -async fn schedule_ordered_timeline_deletions( - tenant: &Arc, -) -> Result>, TimelineId)>, DeleteTenantError> { - // Tenant is stopping at this point. We know it will be deleted. - // No new timelines should be created. - // Tree sort timelines to delete from leafs to the root. - // NOTE: by calling clone we release the mutex which creates a possibility for a race: pending deletion - // can complete and remove timeline from the map in between our call to clone - // and `DeleteTimelineFlow::run`, so `run` wont find timeline in `timelines` map. - // timelines.lock is currently synchronous so we cant hold it across await point. - // So just ignore NotFound error if we get it from `run`. - // Beware: in case it becomes async and we try to hold it here, `run` also locks it, which can create a deadlock. - let timelines = tenant.timelines.lock().unwrap().clone(); - let sorted = - tree_sort_timelines(timelines, |t| t.get_ancestor_timeline_id()).context("tree sort")?; - - let mut already_running_deletions = vec![]; - - for (timeline_id, _) in sorted.into_iter().rev() { - let span = tracing::info_span!("timeline_delete", %timeline_id); - let res = DeleteTimelineFlow::run(tenant, timeline_id, true) - .instrument(span) - .await; - if let Err(e) = res { - match e { - DeleteTimelineError::NotFound => { - // Timeline deletion finished after call to clone above but before call - // to `DeleteTimelineFlow::run` and removed timeline from the map. - continue; - } - DeleteTimelineError::AlreadyInProgress(guard) => { - already_running_deletions.push((guard, timeline_id)); - continue; - } - e => return Err(DeleteTenantError::Timeline(e)), - } - } - } - - Ok(already_running_deletions) -} - -async fn ensure_timelines_dir_empty(timelines_path: &Utf8Path) -> Result<(), DeleteTenantError> { - // Assert timelines dir is empty. - if !fs_ext::is_directory_empty(timelines_path).await? { - // Display first 10 items in directory - let list = fs_ext::list_dir(timelines_path).await.context("list_dir")?; - let list = &list.into_iter().take(10).collect::>(); - return Err(DeleteTenantError::Other(anyhow::anyhow!( - "Timelines directory is not empty after all timelines deletion: {list:?}" - ))); - } - - Ok(()) -} - -async fn remove_tenant_remote_delete_mark( - conf: &PageServerConf, - remote_storage: &GenericRemoteStorage, - tenant_shard_id: &TenantShardId, - cancel: &CancellationToken, -) -> Result<(), DeleteTenantError> { - let path = remote_tenant_delete_mark_path(conf, tenant_shard_id)?; - backoff::retry( - || async { remote_storage.delete(&path, cancel).await }, - TimeoutOrCancel::caused_by_cancel, - FAILED_UPLOAD_WARN_THRESHOLD, - FAILED_REMOTE_OP_RETRIES, - "remove_tenant_remote_delete_mark", - cancel, - ) - .await - .ok_or_else(|| anyhow::Error::new(TimeoutOrCancel::Cancel)) - .and_then(|x| x) - .context("remove_tenant_remote_delete_mark")?; - Ok(()) -} - -// Cleanup fs traces: tenant config, timelines dir local delete mark, tenant dir -async fn cleanup_remaining_fs_traces( - conf: &PageServerConf, - tenant_shard_id: &TenantShardId, -) -> Result<(), DeleteTenantError> { - let rm = |p: Utf8PathBuf, is_dir: bool| async move { - if is_dir { - tokio::fs::remove_dir(&p).await - } else { - tokio::fs::remove_file(&p).await - } - .or_else(fs_ext::ignore_not_found) - .with_context(|| format!("failed to delete {p}")) - }; - - rm(conf.tenant_config_path(tenant_shard_id), false).await?; - rm(conf.tenant_location_config_path(tenant_shard_id), false).await?; - - fail::fail_point!("tenant-delete-before-remove-timelines-dir", |_| { - Err(anyhow::anyhow!( - "failpoint: tenant-delete-before-remove-timelines-dir" - ))? - }); - - rm(conf.timelines_path(tenant_shard_id), true).await?; - - fail::fail_point!("tenant-delete-before-remove-deleted-mark", |_| { - Err(anyhow::anyhow!( - "failpoint: tenant-delete-before-remove-deleted-mark" - ))? - }); - - // Make sure previous deletions are ordered before mark removal. - // Otherwise there is no guarantee that they reach the disk before mark deletion. - // So its possible for mark to reach disk first and for other deletions - // to be reordered later and thus missed if a crash occurs. - // Note that we dont need to sync after mark file is removed - // because we can tolerate the case when mark file reappears on startup. - let tenant_path = &conf.tenant_path(tenant_shard_id); - if tenant_path.exists() { - crashsafe::fsync_async(&conf.tenant_path(tenant_shard_id)) - .await - .context("fsync_pre_mark_remove")?; - } - - rm(conf.tenant_deleted_mark_file_path(tenant_shard_id), false).await?; - - rm(conf.tenant_heatmap_path(tenant_shard_id), false).await?; - - fail::fail_point!("tenant-delete-before-remove-tenant-dir", |_| { - Err(anyhow::anyhow!( - "failpoint: tenant-delete-before-remove-tenant-dir" - ))? - }); - - rm(conf.tenant_path(tenant_shard_id), true).await?; - - Ok(()) -} - -#[derive(Default)] -pub enum DeleteTenantFlow { - #[default] - NotStarted, - InProgress, - Finished, -} - -impl DeleteTenantFlow { - pub(crate) async fn should_resume_deletion( - conf: &'static PageServerConf, - remote_mark_exists: bool, - tenant: &Tenant, - ) -> Result, DeleteTenantError> { - let acquire = |t: &Tenant| { - Some( - Arc::clone(&t.delete_progress) - .try_lock_owned() - .expect("we're the only owner during init"), - ) - }; - - if remote_mark_exists { - return Ok(acquire(tenant)); - } - - // Check local mark first, if its there there is no need to go to s3 to check whether remote one exists. - if conf - .tenant_deleted_mark_file_path(&tenant.tenant_shard_id) - .exists() - { - Ok(acquire(tenant)) - } else { - Ok(None) - } - } - - pub(crate) async fn resume_from_attach( - guard: DeletionGuard, - tenant: &Arc, - preload: Option, - tenants: &'static std::sync::RwLock, - ctx: &RequestContext, - ) -> Result<(), DeleteTenantError> { - let (_, progress) = completion::channel(); - - tenant - .set_stopping(progress, false, true) - .await - .expect("cant be stopping or broken"); - - tenant - .attach(preload, super::SpawnMode::Eager, ctx) - .await - .context("attach")?; - - Self::background( - guard, - tenant.conf, - tenant.remote_storage.clone(), - tenants, - tenant, - ) - .await - } - - async fn background( - mut guard: OwnedMutexGuard, - conf: &PageServerConf, - remote_storage: GenericRemoteStorage, - tenants: &'static std::sync::RwLock, - tenant: &Arc, - ) -> Result<(), DeleteTenantError> { - // Tree sort timelines, schedule delete for them. Mention retries from the console side. - // Note that if deletion fails we dont mark timelines as broken, - // the whole tenant will become broken as by `Self::schedule_background` logic - let already_running_timeline_deletions = schedule_ordered_timeline_deletions(tenant) - .await - .context("schedule_ordered_timeline_deletions")?; - - fail::fail_point!("tenant-delete-before-polling-ongoing-deletions", |_| { - Err(anyhow::anyhow!( - "failpoint: tenant-delete-before-polling-ongoing-deletions" - ))? - }); - - // Wait for deletions that were already running at the moment when tenant deletion was requested. - // When we can lock deletion guard it means that corresponding timeline deletion finished. - for (guard, timeline_id) in already_running_timeline_deletions { - let flow = guard.lock().await; - if !flow.is_finished() { - return Err(DeleteTenantError::Other(anyhow::anyhow!( - "already running timeline deletion failed: {timeline_id}" - ))); - } - } - - // Remove top-level tenant objects that don't belong to a timeline, such as heatmap - let heatmap_path = remote_heatmap_path(&tenant.tenant_shard_id()); - if let Some(Err(e)) = backoff::retry( - || async { - remote_storage - .delete(&heatmap_path, &task_mgr::shutdown_token()) - .await - }, - TimeoutOrCancel::caused_by_cancel, - FAILED_UPLOAD_WARN_THRESHOLD, - FAILED_REMOTE_OP_RETRIES, - "remove_remote_tenant_heatmap", - &task_mgr::shutdown_token(), - ) - .await - { - tracing::warn!("Failed to delete heatmap at {heatmap_path}: {e}"); - } - - let timelines_path = conf.timelines_path(&tenant.tenant_shard_id); - // May not exist if we fail in cleanup_remaining_fs_traces after removing it - if timelines_path.exists() { - // sanity check to guard against layout changes - ensure_timelines_dir_empty(&timelines_path) - .await - .context("timelines dir not empty")?; - } - - remove_tenant_remote_delete_mark( - conf, - &remote_storage, - &tenant.tenant_shard_id, - &task_mgr::shutdown_token(), - ) - .await?; - - pausable_failpoint!("tenant-delete-before-cleanup-remaining-fs-traces-pausable"); - fail::fail_point!("tenant-delete-before-cleanup-remaining-fs-traces", |_| { - Err(anyhow::anyhow!( - "failpoint: tenant-delete-before-cleanup-remaining-fs-traces" - ))? - }); - - cleanup_remaining_fs_traces(conf, &tenant.tenant_shard_id) - .await - .context("cleanup_remaining_fs_traces")?; - - { - // This block is simply removing the TenantSlot for this tenant. It requires a loop because - // we might conflict with a TenantSlot::InProgress marker and need to wait for it. - // - // This complexity will go away when we simplify how deletion works: - // https://github.com/neondatabase/neon/issues/5080 - loop { - // Under the TenantMap lock, try to remove the tenant. We usually succeed, but if - // we encounter an InProgress marker, yield the barrier it contains and wait on it. - let barrier = { - let mut locked = tenants.write().unwrap(); - let removed = locked.remove(tenant.tenant_shard_id); - - // FIXME: we should not be modifying this from outside of mgr.rs. - // This will go away when we simplify deletion (https://github.com/neondatabase/neon/issues/5080) - - // Update stats - match &removed { - TenantsMapRemoveResult::Occupied(slot) => { - crate::metrics::TENANT_MANAGER.slot_removed(slot); - } - TenantsMapRemoveResult::InProgress(barrier) => { - crate::metrics::TENANT_MANAGER - .slot_removed(&TenantSlot::InProgress(barrier.clone())); - } - TenantsMapRemoveResult::Vacant => { - // Nothing changed in map, no metric update - } - } - - match removed { - TenantsMapRemoveResult::Occupied(TenantSlot::Attached(tenant)) => { - match tenant.current_state() { - TenantState::Stopping { .. } | TenantState::Broken { .. } => { - // Expected: we put the tenant into stopping state before we start deleting it - } - state => { - // Unexpected state - tracing::warn!( - "Tenant in unexpected state {state} after deletion" - ); - } - } - break; - } - TenantsMapRemoveResult::Occupied(TenantSlot::Secondary(_)) => { - // This is unexpected: this secondary tenants should not have been created, and we - // are not in a position to shut it down from here. - tracing::warn!("Tenant transitioned to secondary mode while deleting!"); - break; - } - TenantsMapRemoveResult::Occupied(TenantSlot::InProgress(_)) => { - unreachable!("TenantsMap::remove handles InProgress separately, should never return it here"); - } - TenantsMapRemoveResult::Vacant => { - tracing::warn!( - "Tenant removed from TenantsMap before deletion completed" - ); - break; - } - TenantsMapRemoveResult::InProgress(barrier) => { - // An InProgress entry was found, we must wait on its barrier - barrier - } - } - }; - - tracing::info!( - "Waiting for competing operation to complete before deleting state for tenant" - ); - barrier.wait().await; - } - } - - *guard = Self::Finished; - - Ok(()) - } -} diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 326086a3ccdf..4fcdf1405288 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -51,7 +51,6 @@ use utils::fs_ext::PathExt; use utils::generation::Generation; use utils::id::{TenantId, TimelineId}; -use super::delete::DeleteTenantError; use super::remote_timeline_client::remote_tenant_path; use super::secondary::SecondaryTenant; use super::timeline::detach_ancestor::PreparedTimelineDetach; @@ -109,12 +108,6 @@ pub(crate) enum TenantsMap { ShuttingDown(BTreeMap), } -pub(crate) enum TenantsMapRemoveResult { - Occupied(TenantSlot), - Vacant, - InProgress(utils::completion::Barrier), -} - /// When resolving a TenantId to a shard, we may be looking for the 0th /// shard, or we might be looking for whichever shard holds a particular page. #[derive(Copy, Clone)] @@ -191,26 +184,6 @@ impl TenantsMap { } } - /// Only for use from DeleteTenantFlow. This method directly removes a TenantSlot from the map. - /// - /// The normal way to remove a tenant is using a SlotGuard, which will gracefully remove the guarded - /// slot if the enclosed tenant is shutdown. - pub(crate) fn remove(&mut self, tenant_shard_id: TenantShardId) -> TenantsMapRemoveResult { - use std::collections::btree_map::Entry; - match self { - TenantsMap::Initializing => TenantsMapRemoveResult::Vacant, - TenantsMap::Open(m) | TenantsMap::ShuttingDown(m) => match m.entry(tenant_shard_id) { - Entry::Occupied(entry) => match entry.get() { - TenantSlot::InProgress(barrier) => { - TenantsMapRemoveResult::InProgress(barrier.clone()) - } - _ => TenantsMapRemoveResult::Occupied(entry.remove()), - }, - Entry::Vacant(_entry) => TenantsMapRemoveResult::Vacant, - }, - } - } - #[cfg(all(debug_assertions, not(test)))] pub(crate) fn len(&self) -> usize { match self { @@ -460,6 +433,18 @@ async fn init_load_tenant_configs( Ok(configs) } +#[derive(Debug, thiserror::Error)] +pub(crate) enum DeleteTenantError { + #[error("Tenant map slot error {0}")] + SlotError(#[from] TenantSlotError), + + #[error("Cancelled")] + Cancelled, + + #[error(transparent)] + Other(#[from] anyhow::Error), +} + /// Initialize repositories with locally available timelines. /// Timelines that are only partially available locally (remote storage has more data than this pageserver) /// are scheduled for download and added to the tenant once download is completed. @@ -629,7 +614,6 @@ pub async fn init_tenant_mgr( AttachedTenantConf::new(location_conf.tenant_conf, attached_conf), shard_identity, Some(init_order.clone()), - &TENANTS, SpawnMode::Lazy, &ctx, ) { @@ -685,7 +669,6 @@ fn tenant_spawn( location_conf: AttachedTenantConf, shard_identity: ShardIdentity, init_order: Option, - tenants: &'static std::sync::RwLock, mode: SpawnMode, ctx: &RequestContext, ) -> anyhow::Result> { @@ -712,7 +695,6 @@ fn tenant_spawn( location_conf, shard_identity, init_order, - tenants, mode, ctx, ) { @@ -1161,7 +1143,6 @@ impl TenantManager { attached_conf, shard_identity, None, - self.tenants, spawn_mode, ctx, )?; @@ -1283,7 +1264,6 @@ impl TenantManager { AttachedTenantConf::try_from(config)?, shard_identity, None, - self.tenants, SpawnMode::Eager, ctx, )?; @@ -1634,7 +1614,7 @@ impl TenantManager { for child_shard_id in &child_shards { let child_shard_id = *child_shard_id; let child_shard = { - let locked = TENANTS.read().unwrap(); + let locked = self.tenants.read().unwrap(); let peek_slot = tenant_map_peek_slot(&locked, &child_shard_id, TenantSlotPeekMode::Read)?; peek_slot.and_then(|s| s.get_attached()).cloned() @@ -1866,7 +1846,7 @@ impl TenantManager { deletion_queue_client: &DeletionQueueClient, ) -> Result<(), TenantStateError> { let tmp_path = self - .detach_tenant0(conf, &TENANTS, tenant_shard_id, deletion_queue_client) + .detach_tenant0(conf, tenant_shard_id, deletion_queue_client) .await?; spawn_background_purge(tmp_path); @@ -1876,7 +1856,6 @@ impl TenantManager { async fn detach_tenant0( &self, conf: &'static PageServerConf, - tenants: &std::sync::RwLock, tenant_shard_id: TenantShardId, deletion_queue_client: &DeletionQueueClient, ) -> Result { @@ -1890,7 +1869,7 @@ impl TenantManager { }; let removal_result = remove_tenant_from_memory( - tenants, + self.tenants, tenant_shard_id, tenant_dir_rename_operation(tenant_shard_id), ) @@ -1906,7 +1885,7 @@ impl TenantManager { pub(crate) fn list_tenants( &self, ) -> Result, TenantMapListError> { - let tenants = TENANTS.read().unwrap(); + let tenants = self.tenants.read().unwrap(); let m = match &*tenants { TenantsMap::Initializing => return Err(TenantMapListError::Initializing), TenantsMap::Open(m) | TenantsMap::ShuttingDown(m) => m, @@ -2007,7 +1986,6 @@ impl TenantManager { AttachedTenantConf::try_from(config)?, shard_identity, None, - self.tenants, SpawnMode::Eager, ctx, )?; diff --git a/pageserver/src/tenant/timeline/delete.rs b/pageserver/src/tenant/timeline/delete.rs index 441298f3e9a2..6d747d424dde 100644 --- a/pageserver/src/tenant/timeline/delete.rs +++ b/pageserver/src/tenant/timeline/delete.rs @@ -255,7 +255,6 @@ impl DeleteTimelineFlow { } /// Shortcut to create Timeline in stopping state and spawn deletion task. - /// See corresponding parts of [`crate::tenant::delete::DeleteTenantFlow`] #[instrument(skip_all, fields(%timeline_id))] pub async fn resume_deletion( tenant: Arc, @@ -420,10 +419,6 @@ impl DeleteTimelineFlow { Ok(()) } - pub(crate) fn is_finished(&self) -> bool { - matches!(self, Self::Finished) - } - pub(crate) fn is_not_started(&self) -> bool { matches!(self, Self::NotStarted) }