Line data Source code
1 : use std::sync::Arc;
2 :
3 : use anyhow::Context;
4 : use camino::{Utf8Path, Utf8PathBuf};
5 : use pageserver_api::{models::TenantState, shard::TenantShardId};
6 : use remote_storage::{GenericRemoteStorage, RemotePath, TimeoutOrCancel};
7 : use tokio::sync::OwnedMutexGuard;
8 : use tokio_util::sync::CancellationToken;
9 : use tracing::{error, instrument, Instrument};
10 :
11 : use utils::{backoff, completion, crashsafe, fs_ext, id::TimelineId};
12 :
13 : use crate::{
14 : config::PageServerConf,
15 : context::RequestContext,
16 : task_mgr::{self, TaskKind},
17 : tenant::mgr::{TenantSlot, TenantsMapRemoveResult},
18 : };
19 :
20 : use super::{
21 : mgr::{GetTenantError, TenantSlotError, TenantSlotUpsertError, TenantsMap},
22 : remote_timeline_client::{FAILED_REMOTE_OP_RETRIES, FAILED_UPLOAD_WARN_THRESHOLD},
23 : span,
24 : timeline::delete::DeleteTimelineFlow,
25 : tree_sort_timelines, DeleteTimelineError, Tenant, TenantPreload,
26 : };
27 :
28 0 : #[derive(Debug, thiserror::Error)]
29 : pub(crate) enum DeleteTenantError {
30 : #[error("GetTenant {0}")]
31 : Get(#[from] GetTenantError),
32 :
33 : #[error("Tenant not attached")]
34 : NotAttached,
35 :
36 : #[error("Invalid state {0}. Expected Active or Broken")]
37 : InvalidState(TenantState),
38 :
39 : #[error("Tenant deletion is already in progress")]
40 : AlreadyInProgress,
41 :
42 : #[error("Tenant map slot error {0}")]
43 : SlotError(#[from] TenantSlotError),
44 :
45 : #[error("Tenant map slot upsert error {0}")]
46 : SlotUpsertError(#[from] TenantSlotUpsertError),
47 :
48 : #[error("Timeline {0}")]
49 : Timeline(#[from] DeleteTimelineError),
50 :
51 : #[error("Cancelled")]
52 : Cancelled,
53 :
54 : #[error(transparent)]
55 : Other(#[from] anyhow::Error),
56 : }
57 :
58 : type DeletionGuard = tokio::sync::OwnedMutexGuard<DeleteTenantFlow>;
59 :
60 0 : fn remote_tenant_delete_mark_path(
61 0 : conf: &PageServerConf,
62 0 : tenant_shard_id: &TenantShardId,
63 0 : ) -> anyhow::Result<RemotePath> {
64 0 : let tenant_remote_path = conf
65 0 : .tenant_path(tenant_shard_id)
66 0 : .strip_prefix(&conf.workdir)
67 0 : .context("Failed to strip workdir prefix")
68 0 : .and_then(RemotePath::new)
69 0 : .context("tenant path")?;
70 0 : Ok(tenant_remote_path.join(Utf8Path::new("timelines/deleted")))
71 0 : }
72 :
73 0 : async fn create_remote_delete_mark(
74 0 : conf: &PageServerConf,
75 0 : remote_storage: &GenericRemoteStorage,
76 0 : tenant_shard_id: &TenantShardId,
77 0 : cancel: &CancellationToken,
78 0 : ) -> Result<(), DeleteTenantError> {
79 0 : let remote_mark_path = remote_tenant_delete_mark_path(conf, tenant_shard_id)?;
80 :
81 0 : let data: &[u8] = &[];
82 0 : backoff::retry(
83 0 : || async {
84 0 : let data = bytes::Bytes::from_static(data);
85 0 : let stream = futures::stream::once(futures::future::ready(Ok(data)));
86 0 : remote_storage
87 0 : .upload(stream, 0, &remote_mark_path, None, cancel)
88 0 : .await
89 0 : },
90 0 : TimeoutOrCancel::caused_by_cancel,
91 0 : FAILED_UPLOAD_WARN_THRESHOLD,
92 0 : FAILED_REMOTE_OP_RETRIES,
93 0 : "mark_upload",
94 0 : cancel,
95 0 : )
96 0 : .await
97 0 : .ok_or_else(|| anyhow::Error::new(TimeoutOrCancel::Cancel))
98 0 : .and_then(|x| x)
99 0 : .context("mark_upload")?;
100 :
101 0 : Ok(())
102 0 : }
103 :
104 0 : async fn create_local_delete_mark(
105 0 : conf: &PageServerConf,
106 0 : tenant_shard_id: &TenantShardId,
107 0 : ) -> Result<(), DeleteTenantError> {
108 0 : let marker_path = conf.tenant_deleted_mark_file_path(tenant_shard_id);
109 0 :
110 0 : // Note: we're ok to replace existing file.
111 0 : let _ = std::fs::OpenOptions::new()
112 0 : .write(true)
113 0 : .create(true)
114 0 : .open(&marker_path)
115 0 : .with_context(|| format!("could not create delete marker file {marker_path:?}"))?;
116 :
117 0 : crashsafe::fsync_file_and_parent(&marker_path).context("sync_mark")?;
118 :
119 0 : Ok(())
120 0 : }
121 :
122 0 : async fn schedule_ordered_timeline_deletions(
123 0 : tenant: &Arc<Tenant>,
124 0 : ) -> Result<Vec<(Arc<tokio::sync::Mutex<DeleteTimelineFlow>>, TimelineId)>, DeleteTenantError> {
125 0 : // Tenant is stopping at this point. We know it will be deleted.
126 0 : // No new timelines should be created.
127 0 : // Tree sort timelines to delete from leafs to the root.
128 0 : // NOTE: by calling clone we release the mutex which creates a possibility for a race: pending deletion
129 0 : // can complete and remove timeline from the map in between our call to clone
130 0 : // and `DeleteTimelineFlow::run`, so `run` wont find timeline in `timelines` map.
131 0 : // timelines.lock is currently synchronous so we cant hold it across await point.
132 0 : // So just ignore NotFound error if we get it from `run`.
133 0 : // Beware: in case it becomes async and we try to hold it here, `run` also locks it, which can create a deadlock.
134 0 : let timelines = tenant.timelines.lock().unwrap().clone();
135 0 : let sorted =
136 0 : tree_sort_timelines(timelines, |t| t.get_ancestor_timeline_id()).context("tree sort")?;
137 :
138 0 : let mut already_running_deletions = vec![];
139 :
140 0 : for (timeline_id, _) in sorted.into_iter().rev() {
141 0 : let span = tracing::info_span!("timeline_delete", %timeline_id);
142 0 : let res = DeleteTimelineFlow::run(tenant, timeline_id, true)
143 0 : .instrument(span)
144 0 : .await;
145 0 : if let Err(e) = res {
146 0 : match e {
147 : DeleteTimelineError::NotFound => {
148 : // Timeline deletion finished after call to clone above but before call
149 : // to `DeleteTimelineFlow::run` and removed timeline from the map.
150 0 : continue;
151 : }
152 0 : DeleteTimelineError::AlreadyInProgress(guard) => {
153 0 : already_running_deletions.push((guard, timeline_id));
154 0 : continue;
155 : }
156 0 : e => return Err(DeleteTenantError::Timeline(e)),
157 : }
158 0 : }
159 : }
160 :
161 0 : Ok(already_running_deletions)
162 0 : }
163 :
164 0 : async fn ensure_timelines_dir_empty(timelines_path: &Utf8Path) -> Result<(), DeleteTenantError> {
165 0 : // Assert timelines dir is empty.
166 0 : if !fs_ext::is_directory_empty(timelines_path).await? {
167 : // Display first 10 items in directory
168 0 : let list = fs_ext::list_dir(timelines_path).await.context("list_dir")?;
169 0 : let list = &list.into_iter().take(10).collect::<Vec<_>>();
170 0 : return Err(DeleteTenantError::Other(anyhow::anyhow!(
171 0 : "Timelines directory is not empty after all timelines deletion: {list:?}"
172 0 : )));
173 0 : }
174 0 :
175 0 : Ok(())
176 0 : }
177 :
178 0 : async fn remove_tenant_remote_delete_mark(
179 0 : conf: &PageServerConf,
180 0 : remote_storage: Option<&GenericRemoteStorage>,
181 0 : tenant_shard_id: &TenantShardId,
182 0 : cancel: &CancellationToken,
183 0 : ) -> Result<(), DeleteTenantError> {
184 0 : if let Some(remote_storage) = remote_storage {
185 0 : let path = remote_tenant_delete_mark_path(conf, tenant_shard_id)?;
186 0 : backoff::retry(
187 0 : || async { remote_storage.delete(&path, cancel).await },
188 0 : TimeoutOrCancel::caused_by_cancel,
189 0 : FAILED_UPLOAD_WARN_THRESHOLD,
190 0 : FAILED_REMOTE_OP_RETRIES,
191 0 : "remove_tenant_remote_delete_mark",
192 0 : cancel,
193 0 : )
194 0 : .await
195 0 : .ok_or_else(|| anyhow::Error::new(TimeoutOrCancel::Cancel))
196 0 : .and_then(|x| x)
197 0 : .context("remove_tenant_remote_delete_mark")?;
198 0 : }
199 0 : Ok(())
200 0 : }
201 :
202 : // Cleanup fs traces: tenant config, timelines dir local delete mark, tenant dir
203 0 : async fn cleanup_remaining_fs_traces(
204 0 : conf: &PageServerConf,
205 0 : tenant_shard_id: &TenantShardId,
206 0 : ) -> Result<(), DeleteTenantError> {
207 0 : let rm = |p: Utf8PathBuf, is_dir: bool| async move {
208 0 : if is_dir {
209 0 : tokio::fs::remove_dir(&p).await
210 : } else {
211 0 : tokio::fs::remove_file(&p).await
212 : }
213 0 : .or_else(fs_ext::ignore_not_found)
214 0 : .with_context(|| format!("failed to delete {p}"))
215 0 : };
216 :
217 0 : rm(conf.tenant_config_path(tenant_shard_id), false).await?;
218 0 : rm(conf.tenant_location_config_path(tenant_shard_id), false).await?;
219 :
220 0 : fail::fail_point!("tenant-delete-before-remove-timelines-dir", |_| {
221 0 : Err(anyhow::anyhow!(
222 0 : "failpoint: tenant-delete-before-remove-timelines-dir"
223 0 : ))?
224 0 : });
225 :
226 0 : rm(conf.timelines_path(tenant_shard_id), true).await?;
227 :
228 0 : fail::fail_point!("tenant-delete-before-remove-deleted-mark", |_| {
229 0 : Err(anyhow::anyhow!(
230 0 : "failpoint: tenant-delete-before-remove-deleted-mark"
231 0 : ))?
232 0 : });
233 :
234 : // Make sure previous deletions are ordered before mark removal.
235 : // Otherwise there is no guarantee that they reach the disk before mark deletion.
236 : // So its possible for mark to reach disk first and for other deletions
237 : // to be reordered later and thus missed if a crash occurs.
238 : // Note that we dont need to sync after mark file is removed
239 : // because we can tolerate the case when mark file reappears on startup.
240 0 : let tenant_path = &conf.tenant_path(tenant_shard_id);
241 0 : if tenant_path.exists() {
242 0 : crashsafe::fsync_async(&conf.tenant_path(tenant_shard_id))
243 0 : .await
244 0 : .context("fsync_pre_mark_remove")?;
245 0 : }
246 :
247 0 : rm(conf.tenant_deleted_mark_file_path(tenant_shard_id), false).await?;
248 :
249 0 : rm(conf.tenant_heatmap_path(tenant_shard_id), false).await?;
250 :
251 0 : fail::fail_point!("tenant-delete-before-remove-tenant-dir", |_| {
252 0 : Err(anyhow::anyhow!(
253 0 : "failpoint: tenant-delete-before-remove-tenant-dir"
254 0 : ))?
255 0 : });
256 :
257 0 : rm(conf.tenant_path(tenant_shard_id), true).await?;
258 :
259 0 : Ok(())
260 0 : }
261 :
262 : /// Orchestrates tenant shut down of all tasks, removes its in-memory structures,
263 : /// and deletes its data from both disk and s3.
264 : /// The sequence of steps:
265 : /// 1. Upload remote deletion mark.
266 : /// 2. Create local mark file.
267 : /// 3. Shutdown tasks
268 : /// 4. Run ordered timeline deletions
269 : /// 5. Wait for timeline deletion operations that were scheduled before tenant deletion was requested
270 : /// 6. Remove remote mark
271 : /// 7. Cleanup remaining fs traces, tenant dir, config, timelines dir, local delete mark
272 : /// It is resumable from any step in case a crash/restart occurs.
273 : /// There are two entrypoints to the process:
274 : /// 1. [`DeleteTenantFlow::run`] this is the main one called by a management api handler.
275 : /// 2. [`DeleteTenantFlow::resume_from_attach`] is called when deletion is resumed tenant is found to be deleted during attach process.
276 : /// Note the only other place that messes around timeline delete mark is the `Tenant::spawn_load` function.
277 88 : #[derive(Default)]
278 : pub enum DeleteTenantFlow {
279 : #[default]
280 : NotStarted,
281 : InProgress,
282 : Finished,
283 : }
284 :
285 : impl DeleteTenantFlow {
286 : // These steps are run in the context of management api request handler.
287 : // Long running steps are continued to run in the background.
288 : // NB: If this fails half-way through, and is retried, the retry will go through
289 : // all the same steps again. Make sure the code here is idempotent, and don't
290 : // error out if some of the shutdown tasks have already been completed!
291 : // NOTE: static needed for background part.
292 : // We assume that calling code sets up the span with tenant_id.
293 0 : #[instrument(skip_all)]
294 : pub(crate) async fn run(
295 : conf: &'static PageServerConf,
296 : remote_storage: Option<GenericRemoteStorage>,
297 : tenants: &'static std::sync::RwLock<TenantsMap>,
298 : tenant: Arc<Tenant>,
299 : ) -> Result<(), DeleteTenantError> {
300 : span::debug_assert_current_span_has_tenant_id();
301 :
302 0 : pausable_failpoint!("tenant-delete-before-run");
303 :
304 : let mut guard = Self::prepare(&tenant).await?;
305 :
306 : if let Err(e) = Self::run_inner(&mut guard, conf, remote_storage.as_ref(), &tenant).await {
307 : tenant.set_broken(format!("{e:#}")).await;
308 : return Err(e);
309 : }
310 :
311 : Self::schedule_background(guard, conf, remote_storage, tenants, tenant);
312 :
313 : Ok(())
314 : }
315 :
316 : // Helper function needed to be able to match once on returned error and transition tenant into broken state.
317 : // This is needed because tenant.shutwodn is not idempotent. If tenant state is set to stopping another call to tenant.shutdown
318 : // will result in an error, but here we need to be able to retry shutdown when tenant deletion is retried.
319 : // So the solution is to set tenant state to broken.
320 0 : async fn run_inner(
321 0 : guard: &mut OwnedMutexGuard<Self>,
322 0 : conf: &'static PageServerConf,
323 0 : remote_storage: Option<&GenericRemoteStorage>,
324 0 : tenant: &Tenant,
325 0 : ) -> Result<(), DeleteTenantError> {
326 0 : guard.mark_in_progress()?;
327 :
328 0 : fail::fail_point!("tenant-delete-before-create-remote-mark", |_| {
329 0 : Err(anyhow::anyhow!(
330 0 : "failpoint: tenant-delete-before-create-remote-mark"
331 0 : ))?
332 0 : });
333 :
334 : // IDEA: implement detach as delete without remote storage. Then they would use the same lock (deletion_progress) so wont contend.
335 : // Though sounds scary, different mark name?
336 : // Detach currently uses remove_dir_all so in case of a crash we can end up in a weird state.
337 0 : if let Some(remote_storage) = &remote_storage {
338 0 : create_remote_delete_mark(
339 0 : conf,
340 0 : remote_storage,
341 0 : &tenant.tenant_shard_id,
342 0 : // Can't use tenant.cancel, it's already shut down. TODO: wire in an appropriate token
343 0 : &CancellationToken::new(),
344 0 : )
345 0 : .await
346 0 : .context("remote_mark")?
347 0 : }
348 :
349 0 : fail::fail_point!("tenant-delete-before-create-local-mark", |_| {
350 0 : Err(anyhow::anyhow!(
351 0 : "failpoint: tenant-delete-before-create-local-mark"
352 0 : ))?
353 0 : });
354 :
355 0 : create_local_delete_mark(conf, &tenant.tenant_shard_id)
356 0 : .await
357 0 : .context("local delete mark")?;
358 :
359 0 : fail::fail_point!("tenant-delete-before-background", |_| {
360 0 : Err(anyhow::anyhow!(
361 0 : "failpoint: tenant-delete-before-background"
362 0 : ))?
363 0 : });
364 :
365 0 : Ok(())
366 0 : }
367 :
368 0 : fn mark_in_progress(&mut self) -> anyhow::Result<()> {
369 0 : match self {
370 0 : Self::Finished => anyhow::bail!("Bug. Is in finished state"),
371 0 : Self::InProgress { .. } => { /* We're in a retry */ }
372 0 : Self::NotStarted => { /* Fresh start */ }
373 : }
374 :
375 0 : *self = Self::InProgress;
376 0 :
377 0 : Ok(())
378 0 : }
379 :
380 0 : pub(crate) async fn should_resume_deletion(
381 0 : conf: &'static PageServerConf,
382 0 : remote_mark_exists: bool,
383 0 : tenant: &Tenant,
384 0 : ) -> Result<Option<DeletionGuard>, DeleteTenantError> {
385 0 : let acquire = |t: &Tenant| {
386 0 : Some(
387 0 : Arc::clone(&t.delete_progress)
388 0 : .try_lock_owned()
389 0 : .expect("we're the only owner during init"),
390 0 : )
391 0 : };
392 :
393 0 : if remote_mark_exists {
394 0 : return Ok(acquire(tenant));
395 0 : }
396 0 :
397 0 : // Check local mark first, if its there there is no need to go to s3 to check whether remote one exists.
398 0 : if conf
399 0 : .tenant_deleted_mark_file_path(&tenant.tenant_shard_id)
400 0 : .exists()
401 : {
402 0 : Ok(acquire(tenant))
403 : } else {
404 0 : Ok(None)
405 : }
406 0 : }
407 :
408 0 : pub(crate) async fn resume_from_attach(
409 0 : guard: DeletionGuard,
410 0 : tenant: &Arc<Tenant>,
411 0 : preload: Option<TenantPreload>,
412 0 : tenants: &'static std::sync::RwLock<TenantsMap>,
413 0 : ctx: &RequestContext,
414 0 : ) -> Result<(), DeleteTenantError> {
415 0 : let (_, progress) = completion::channel();
416 0 :
417 0 : tenant
418 0 : .set_stopping(progress, false, true)
419 0 : .await
420 0 : .expect("cant be stopping or broken");
421 0 :
422 0 : tenant
423 0 : .attach(preload, super::SpawnMode::Normal, ctx)
424 0 : .await
425 0 : .context("attach")?;
426 :
427 0 : Self::background(
428 0 : guard,
429 0 : tenant.conf,
430 0 : tenant.remote_storage.clone(),
431 0 : tenants,
432 0 : tenant,
433 0 : )
434 0 : .await
435 0 : }
436 :
437 0 : async fn prepare(
438 0 : tenant: &Arc<Tenant>,
439 0 : ) -> Result<tokio::sync::OwnedMutexGuard<Self>, DeleteTenantError> {
440 0 : // FIXME: unsure about active only. Our init jobs may not be cancellable properly,
441 0 : // so at least for now allow deletions only for active tenants. TODO recheck
442 0 : // Broken and Stopping is needed for retries.
443 0 : if !matches!(
444 0 : tenant.current_state(),
445 : TenantState::Active | TenantState::Broken { .. }
446 : ) {
447 0 : return Err(DeleteTenantError::InvalidState(tenant.current_state()));
448 0 : }
449 :
450 0 : let guard = Arc::clone(&tenant.delete_progress)
451 0 : .try_lock_owned()
452 0 : .map_err(|_| DeleteTenantError::AlreadyInProgress)?;
453 :
454 0 : fail::fail_point!("tenant-delete-before-shutdown", |_| {
455 0 : Err(anyhow::anyhow!("failpoint: tenant-delete-before-shutdown"))?
456 0 : });
457 :
458 : // make pageserver shutdown not to wait for our completion
459 0 : let (_, progress) = completion::channel();
460 0 :
461 0 : // It would be good to only set stopping here and continue shutdown in the background, but shutdown is not idempotent.
462 0 : // i e it is an error to do:
463 0 : // tenant.set_stopping
464 0 : // tenant.shutdown
465 0 : // Its also bad that we're holding tenants.read here.
466 0 : // TODO relax set_stopping to be idempotent?
467 0 : if tenant.shutdown(progress, false).await.is_err() {
468 0 : return Err(DeleteTenantError::Other(anyhow::anyhow!(
469 0 : "tenant shutdown is already in progress"
470 0 : )));
471 0 : }
472 0 :
473 0 : Ok(guard)
474 0 : }
475 :
476 0 : fn schedule_background(
477 0 : guard: OwnedMutexGuard<Self>,
478 0 : conf: &'static PageServerConf,
479 0 : remote_storage: Option<GenericRemoteStorage>,
480 0 : tenants: &'static std::sync::RwLock<TenantsMap>,
481 0 : tenant: Arc<Tenant>,
482 0 : ) {
483 0 : let tenant_shard_id = tenant.tenant_shard_id;
484 0 :
485 0 : task_mgr::spawn(
486 0 : task_mgr::BACKGROUND_RUNTIME.handle(),
487 0 : TaskKind::TimelineDeletionWorker,
488 0 : Some(tenant_shard_id),
489 0 : None,
490 0 : "tenant_delete",
491 : false,
492 0 : async move {
493 0 : if let Err(err) =
494 0 : Self::background(guard, conf, remote_storage, tenants, &tenant).await
495 : {
496 0 : error!("Error: {err:#}");
497 0 : tenant.set_broken(format!("{err:#}")).await;
498 0 : };
499 0 : Ok(())
500 0 : }
501 0 : .instrument(tracing::info_span!(parent: None, "delete_tenant", tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug())),
502 : );
503 0 : }
504 :
505 0 : async fn background(
506 0 : mut guard: OwnedMutexGuard<Self>,
507 0 : conf: &PageServerConf,
508 0 : remote_storage: Option<GenericRemoteStorage>,
509 0 : tenants: &'static std::sync::RwLock<TenantsMap>,
510 0 : tenant: &Arc<Tenant>,
511 0 : ) -> Result<(), DeleteTenantError> {
512 : // Tree sort timelines, schedule delete for them. Mention retries from the console side.
513 : // Note that if deletion fails we dont mark timelines as broken,
514 : // the whole tenant will become broken as by `Self::schedule_background` logic
515 0 : let already_running_timeline_deletions = schedule_ordered_timeline_deletions(tenant)
516 0 : .await
517 0 : .context("schedule_ordered_timeline_deletions")?;
518 :
519 0 : fail::fail_point!("tenant-delete-before-polling-ongoing-deletions", |_| {
520 0 : Err(anyhow::anyhow!(
521 0 : "failpoint: tenant-delete-before-polling-ongoing-deletions"
522 0 : ))?
523 0 : });
524 :
525 : // Wait for deletions that were already running at the moment when tenant deletion was requested.
526 : // When we can lock deletion guard it means that corresponding timeline deletion finished.
527 0 : for (guard, timeline_id) in already_running_timeline_deletions {
528 0 : let flow = guard.lock().await;
529 0 : if !flow.is_finished() {
530 0 : return Err(DeleteTenantError::Other(anyhow::anyhow!(
531 0 : "already running timeline deletion failed: {timeline_id}"
532 0 : )));
533 0 : }
534 : }
535 :
536 0 : let timelines_path = conf.timelines_path(&tenant.tenant_shard_id);
537 0 : // May not exist if we fail in cleanup_remaining_fs_traces after removing it
538 0 : if timelines_path.exists() {
539 : // sanity check to guard against layout changes
540 0 : ensure_timelines_dir_empty(&timelines_path)
541 0 : .await
542 0 : .context("timelines dir not empty")?;
543 0 : }
544 :
545 0 : remove_tenant_remote_delete_mark(
546 0 : conf,
547 0 : remote_storage.as_ref(),
548 0 : &tenant.tenant_shard_id,
549 0 : // Can't use tenant.cancel, it's already shut down. TODO: wire in an appropriate token
550 0 : &CancellationToken::new(),
551 0 : )
552 0 : .await?;
553 :
554 0 : pausable_failpoint!("tenant-delete-before-cleanup-remaining-fs-traces-pausable");
555 0 : fail::fail_point!("tenant-delete-before-cleanup-remaining-fs-traces", |_| {
556 0 : Err(anyhow::anyhow!(
557 0 : "failpoint: tenant-delete-before-cleanup-remaining-fs-traces"
558 0 : ))?
559 0 : });
560 :
561 0 : cleanup_remaining_fs_traces(conf, &tenant.tenant_shard_id)
562 0 : .await
563 0 : .context("cleanup_remaining_fs_traces")?;
564 :
565 : {
566 0 : pausable_failpoint!("tenant-delete-before-map-remove");
567 :
568 : // This block is simply removing the TenantSlot for this tenant. It requires a loop because
569 : // we might conflict with a TenantSlot::InProgress marker and need to wait for it.
570 : //
571 : // This complexity will go away when we simplify how deletion works:
572 : // https://github.com/neondatabase/neon/issues/5080
573 : loop {
574 : // Under the TenantMap lock, try to remove the tenant. We usually succeed, but if
575 : // we encounter an InProgress marker, yield the barrier it contains and wait on it.
576 0 : let barrier = {
577 0 : let mut locked = tenants.write().unwrap();
578 0 : let removed = locked.remove(tenant.tenant_shard_id);
579 0 :
580 0 : // FIXME: we should not be modifying this from outside of mgr.rs.
581 0 : // This will go away when we simplify deletion (https://github.com/neondatabase/neon/issues/5080)
582 0 : crate::metrics::TENANT_MANAGER
583 0 : .tenant_slots
584 0 : .set(locked.len() as u64);
585 :
586 0 : match removed {
587 0 : TenantsMapRemoveResult::Occupied(TenantSlot::Attached(tenant)) => {
588 0 : match tenant.current_state() {
589 0 : TenantState::Stopping { .. } | TenantState::Broken { .. } => {
590 0 : // Expected: we put the tenant into stopping state before we start deleting it
591 0 : }
592 0 : state => {
593 : // Unexpected state
594 0 : tracing::warn!(
595 0 : "Tenant in unexpected state {state} after deletion"
596 0 : );
597 : }
598 : }
599 0 : break;
600 : }
601 : TenantsMapRemoveResult::Occupied(TenantSlot::Secondary(_)) => {
602 : // This is unexpected: this secondary tenants should not have been created, and we
603 : // are not in a position to shut it down from here.
604 0 : tracing::warn!("Tenant transitioned to secondary mode while deleting!");
605 0 : break;
606 : }
607 : TenantsMapRemoveResult::Occupied(TenantSlot::InProgress(_)) => {
608 0 : unreachable!("TenantsMap::remove handles InProgress separately, should never return it here");
609 : }
610 : TenantsMapRemoveResult::Vacant => {
611 0 : tracing::warn!(
612 0 : "Tenant removed from TenantsMap before deletion completed"
613 0 : );
614 0 : break;
615 : }
616 0 : TenantsMapRemoveResult::InProgress(barrier) => {
617 0 : // An InProgress entry was found, we must wait on its barrier
618 0 : barrier
619 : }
620 : }
621 : };
622 :
623 0 : tracing::info!(
624 0 : "Waiting for competing operation to complete before deleting state for tenant"
625 0 : );
626 0 : barrier.wait().await;
627 : }
628 : }
629 :
630 0 : *guard = Self::Finished;
631 0 :
632 0 : Ok(())
633 0 : }
634 : }
|