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