TLA Line data Source code
1 : use std::{
2 : ops::{Deref, DerefMut},
3 : sync::Arc,
4 : };
5 :
6 : use anyhow::Context;
7 : use pageserver_api::models::TimelineState;
8 : use tokio::sync::OwnedMutexGuard;
9 : use tracing::{debug, error, info, instrument, warn, Instrument, Span};
10 : use utils::{
11 : crashsafe, fs_ext,
12 : id::{TenantId, TimelineId},
13 : };
14 :
15 : use crate::{
16 : config::PageServerConf,
17 : deletion_queue::DeletionQueueClient,
18 : task_mgr::{self, TaskKind},
19 : tenant::{
20 : metadata::TimelineMetadata,
21 : remote_timeline_client::{
22 : self, PersistIndexPartWithDeletedFlagError, RemoteTimelineClient,
23 : },
24 : CreateTimelineCause, DeleteTimelineError, Tenant,
25 : },
26 : InitializationOrder,
27 : };
28 :
29 : use super::{Timeline, TimelineResources};
30 :
31 : /// Now that the Timeline is in Stopping state, request all the related tasks to shut down.
32 CBC 225 : async fn stop_tasks(timeline: &Timeline) -> Result<(), DeleteTimelineError> {
33 : // Stop the walreceiver first.
34 UBC 0 : debug!("waiting for wal receiver to shutdown");
35 CBC 225 : let maybe_started_walreceiver = { timeline.walreceiver.lock().unwrap().take() };
36 225 : if let Some(walreceiver) = maybe_started_walreceiver {
37 175 : walreceiver.stop().await;
38 50 : }
39 UBC 0 : debug!("wal receiver shutdown confirmed");
40 :
41 : // Prevent new uploads from starting.
42 CBC 225 : if let Some(remote_client) = timeline.remote_client.as_ref() {
43 225 : let res = remote_client.stop();
44 225 : match res {
45 225 : Ok(()) => {}
46 UBC 0 : Err(e) => match e {
47 0 : remote_timeline_client::StopError::QueueUninitialized => {
48 0 : // This case shouldn't happen currently because the
49 0 : // load and attach code bails out if _any_ of the timeline fails to fetch its IndexPart.
50 0 : // That is, before we declare the Tenant as Active.
51 0 : // But we only allow calls to delete_timeline on Active tenants.
52 0 : return Err(DeleteTimelineError::Other(anyhow::anyhow!("upload queue is uninitialized, likely the timeline was in Broken state prior to this call because it failed to fetch IndexPart during load or attach, check the logs")));
53 : }
54 : },
55 : }
56 0 : }
57 :
58 : // Stop & wait for the remaining timeline tasks, including upload tasks.
59 : // NB: This and other delete_timeline calls do not run as a task_mgr task,
60 : // so, they are not affected by this shutdown_tasks() call.
61 CBC 225 : info!("waiting for timeline tasks to shutdown");
62 239 : task_mgr::shutdown_tasks(None, Some(timeline.tenant_id), Some(timeline.timeline_id)).await;
63 :
64 225 : fail::fail_point!("timeline-delete-before-index-deleted-at", |_| {
65 8 : Err(anyhow::anyhow!(
66 8 : "failpoint: timeline-delete-before-index-deleted-at"
67 8 : ))?
68 225 : });
69 217 : Ok(())
70 225 : }
71 :
72 : /// Mark timeline as deleted in S3 so we won't pick it up next time
73 : /// during attach or pageserver restart.
74 : /// See comment in persist_index_part_with_deleted_flag.
75 217 : async fn set_deleted_in_remote_index(timeline: &Timeline) -> Result<(), DeleteTimelineError> {
76 217 : if let Some(remote_client) = timeline.remote_client.as_ref() {
77 944 : match remote_client.persist_index_part_with_deleted_flag().await {
78 : // If we (now, or already) marked it successfully as deleted, we can proceed
79 217 : Ok(()) | Err(PersistIndexPartWithDeletedFlagError::AlreadyDeleted(_)) => (),
80 : // Bail out otherwise
81 : //
82 : // AlreadyInProgress shouldn't happen, because the 'delete_lock' prevents
83 : // two tasks from performing the deletion at the same time. The first task
84 : // that starts deletion should run it to completion.
85 UBC 0 : Err(e @ PersistIndexPartWithDeletedFlagError::AlreadyInProgress(_))
86 0 : | Err(e @ PersistIndexPartWithDeletedFlagError::Other(_)) => {
87 0 : return Err(DeleteTimelineError::Other(anyhow::anyhow!(e)));
88 : }
89 : }
90 0 : }
91 CBC 217 : Ok(())
92 217 : }
93 :
94 : // We delete local files first, so if pageserver restarts after local files deletion then remote deletion is not continued.
95 : // This can be solved with inversion of these steps. But even if these steps are inverted then, when index_part.json
96 : // gets deleted there is no way to distinguish between "this timeline is good, we just didnt upload it to remote"
97 : // and "this timeline is deleted we should continue with removal of local state". So to avoid the ambiguity we use a mark file.
98 : // After index part is deleted presence of this mark file indentifies that it was a deletion intention.
99 : // So we can just remove the mark file.
100 238 : async fn create_delete_mark(
101 238 : conf: &PageServerConf,
102 238 : tenant_id: TenantId,
103 238 : timeline_id: TimelineId,
104 238 : ) -> Result<(), DeleteTimelineError> {
105 238 : fail::fail_point!("timeline-delete-before-delete-mark", |_| {
106 UBC 0 : Err(anyhow::anyhow!(
107 0 : "failpoint: timeline-delete-before-delete-mark"
108 0 : ))?
109 CBC 238 : });
110 238 : let marker_path = conf.timeline_delete_mark_file_path(tenant_id, timeline_id);
111 238 :
112 238 : // Note: we're ok to replace existing file.
113 238 : let _ = std::fs::OpenOptions::new()
114 238 : .write(true)
115 238 : .create(true)
116 238 : .open(&marker_path)
117 238 : .with_context(|| format!("could not create delete marker file {marker_path:?}"))?;
118 :
119 238 : crashsafe::fsync_file_and_parent(&marker_path).context("sync_mark")?;
120 238 : Ok(())
121 238 : }
122 :
123 : /// Grab the layer_removal_cs lock, and actually perform the deletion.
124 : ///
125 : /// This lock prevents prevents GC or compaction from running at the same time.
126 : /// The GC task doesn't register itself with the timeline it's operating on,
127 : /// so it might still be running even though we called `shutdown_tasks`.
128 : ///
129 : /// Note that there are still other race conditions between
130 : /// GC, compaction and timeline deletion. See
131 : /// <https://github.com/neondatabase/neon/issues/2671>
132 : ///
133 : /// No timeout here, GC & Compaction should be responsive to the
134 : /// `TimelineState::Stopping` change.
135 234 : async fn delete_local_layer_files(
136 234 : conf: &PageServerConf,
137 234 : tenant_id: TenantId,
138 234 : timeline: &Timeline,
139 234 : ) -> anyhow::Result<()> {
140 234 : info!("waiting for layer_removal_cs.lock()");
141 234 : let layer_removal_guard = timeline.layer_removal_cs.lock().await;
142 234 : info!("got layer_removal_cs.lock(), deleting layer files");
143 :
144 : // NB: storage_sync upload tasks that reference these layers have been cancelled
145 : // by the caller.
146 :
147 234 : let local_timeline_directory = conf.timeline_path(&tenant_id, &timeline.timeline_id);
148 234 :
149 234 : fail::fail_point!("timeline-delete-before-rm", |_| {
150 9 : Err(anyhow::anyhow!("failpoint: timeline-delete-before-rm"))?
151 234 : });
152 :
153 : // NB: This need not be atomic because the deleted flag in the IndexPart
154 : // will be observed during tenant/timeline load. The deletion will be resumed there.
155 : //
156 : // For configurations without remote storage, we guarantee crash-safety by persising delete mark file.
157 : //
158 : // Note that here we do not bail out on std::io::ErrorKind::NotFound.
159 : // This can happen if we're called a second time, e.g.,
160 : // because of a previous failure/cancellation at/after
161 : // failpoint timeline-delete-after-rm.
162 : //
163 : // It can also happen if we race with tenant detach, because,
164 : // it doesn't grab the layer_removal_cs lock.
165 : //
166 : // For now, log and continue.
167 : // warn! level is technically not appropriate for the
168 : // first case because we should expect retries to happen.
169 : // But the error is so rare, it seems better to get attention if it happens.
170 : //
171 : // Note that metadata removal is skipped, this is not technically needed,
172 : // but allows to reuse timeline loading code during resumed deletion.
173 : // (we always expect that metadata is in place when timeline is being loaded)
174 :
175 : #[cfg(feature = "testing")]
176 225 : let mut counter = 0;
177 225 :
178 225 : // Timeline directory may not exist if we failed to delete mark file and request was retried.
179 225 : if !local_timeline_directory.exists() {
180 10 : return Ok(());
181 215 : }
182 215 :
183 215 : let metadata_path = conf.metadata_path(&tenant_id, &timeline.timeline_id);
184 :
185 5265 : for entry in walkdir::WalkDir::new(&local_timeline_directory).contents_first(true) {
186 : #[cfg(feature = "testing")]
187 : {
188 5265 : counter += 1;
189 5265 : if counter == 2 {
190 213 : fail::fail_point!("timeline-delete-during-rm", |_| {
191 7 : Err(anyhow::anyhow!("failpoint: timeline-delete-during-rm"))?
192 213 : });
193 5052 : }
194 : }
195 :
196 5258 : let entry = entry?;
197 5258 : if entry.path() == metadata_path {
198 UBC 0 : debug!("found metadata, skipping");
199 CBC 206 : continue;
200 5052 : }
201 5052 :
202 5052 : if entry.path() == local_timeline_directory {
203 : // Keeping directory because metedata file is still there
204 UBC 0 : debug!("found timeline dir itself, skipping");
205 CBC 208 : continue;
206 4844 : }
207 :
208 4844 : let metadata = match entry.metadata() {
209 4844 : Ok(metadata) => metadata,
210 UBC 0 : Err(e) => {
211 0 : if crate::is_walkdir_io_not_found(&e) {
212 0 : warn!(
213 0 : timeline_dir=?local_timeline_directory,
214 0 : path=?entry.path().display(),
215 0 : "got not found err while removing timeline dir, proceeding anyway"
216 0 : );
217 0 : continue;
218 0 : }
219 0 : anyhow::bail!(e);
220 : }
221 : };
222 :
223 CBC 4844 : if metadata.is_dir() {
224 UBC 0 : warn!(path=%entry.path().display(), "unexpected directory under timeline dir");
225 0 : tokio::fs::remove_dir(entry.path()).await
226 : } else {
227 CBC 4844 : tokio::fs::remove_file(entry.path()).await
228 : }
229 4844 : .with_context(|| format!("Failed to remove: {}", entry.path().display()))?;
230 : }
231 :
232 208 : info!("finished deleting layer files, releasing layer_removal_cs.lock()");
233 208 : drop(layer_removal_guard);
234 208 :
235 208 : fail::fail_point!("timeline-delete-after-rm", |_| {
236 4 : Err(anyhow::anyhow!("failpoint: timeline-delete-after-rm"))?
237 208 : });
238 :
239 204 : Ok(())
240 234 : }
241 :
242 : /// Removes remote layers and an index file after them.
243 214 : async fn delete_remote_layers_and_index(timeline: &Timeline) -> anyhow::Result<()> {
244 214 : if let Some(remote_client) = &timeline.remote_client {
245 1240 : remote_client.delete_all().await.context("delete_all")?
246 UBC 0 : };
247 :
248 CBC 198 : Ok(())
249 214 : }
250 :
251 : // This function removs remaining traces of a timeline on disk.
252 : // Namely: metadata file, timeline directory, delete mark.
253 : // Note: io::ErrorKind::NotFound are ignored for metadata and timeline dir.
254 : // delete mark should be present because it is the last step during deletion.
255 : // (nothing can fail after its deletion)
256 206 : async fn cleanup_remaining_timeline_fs_traces(
257 206 : conf: &PageServerConf,
258 206 : tenant_id: TenantId,
259 206 : timeline_id: TimelineId,
260 206 : ) -> anyhow::Result<()> {
261 : // Remove local metadata
262 206 : tokio::fs::remove_file(conf.metadata_path(&tenant_id, &timeline_id))
263 206 : .await
264 206 : .or_else(fs_ext::ignore_not_found)
265 206 : .context("remove metadata")?;
266 :
267 206 : fail::fail_point!("timeline-delete-after-rm-metadata", |_| {
268 4 : Err(anyhow::anyhow!(
269 4 : "failpoint: timeline-delete-after-rm-metadata"
270 4 : ))?
271 206 : });
272 :
273 : // Remove timeline dir
274 202 : tokio::fs::remove_dir(conf.timeline_path(&tenant_id, &timeline_id))
275 200 : .await
276 202 : .or_else(fs_ext::ignore_not_found)
277 202 : .context("timeline dir")?;
278 :
279 202 : fail::fail_point!("timeline-delete-after-rm-dir", |_| {
280 8 : Err(anyhow::anyhow!("failpoint: timeline-delete-after-rm-dir"))?
281 202 : });
282 :
283 : // Make sure previous deletions are ordered before mark removal.
284 : // Otherwise there is no guarantee that they reach the disk before mark deletion.
285 : // So its possible for mark to reach disk first and for other deletions
286 : // to be reordered later and thus missed if a crash occurs.
287 : // Note that we dont need to sync after mark file is removed
288 : // because we can tolerate the case when mark file reappears on startup.
289 194 : let timeline_path = conf.timelines_path(&tenant_id);
290 194 : crashsafe::fsync_async(timeline_path)
291 388 : .await
292 194 : .context("fsync_pre_mark_remove")?;
293 :
294 : // Remove delete mark
295 194 : tokio::fs::remove_file(conf.timeline_delete_mark_file_path(tenant_id, timeline_id))
296 194 : .await
297 194 : .context("remove delete mark")
298 206 : }
299 :
300 : /// It is important that this gets called when DeletionGuard is being held.
301 : /// For more context see comments in [`DeleteTimelineFlow::prepare`]
302 186 : async fn remove_timeline_from_tenant(
303 186 : tenant: &Tenant,
304 186 : timeline_id: TimelineId,
305 186 : _: &DeletionGuard, // using it as a witness
306 186 : ) -> anyhow::Result<()> {
307 186 : // Remove the timeline from the map.
308 186 : let mut timelines = tenant.timelines.lock().unwrap();
309 186 : let children_exist = timelines
310 186 : .iter()
311 317 : .any(|(_, entry)| entry.get_ancestor_timeline_id() == Some(timeline_id));
312 186 : // XXX this can happen because `branch_timeline` doesn't check `TimelineState::Stopping`.
313 186 : // We already deleted the layer files, so it's probably best to panic.
314 186 : // (Ideally, above remove_dir_all is atomic so we don't see this timeline after a restart)
315 186 : if children_exist {
316 UBC 0 : panic!("Timeline grew children while we removed layer files");
317 CBC 186 : }
318 186 :
319 186 : timelines
320 186 : .remove(&timeline_id)
321 186 : .expect("timeline that we were deleting was concurrently removed from 'timelines' map");
322 186 :
323 186 : drop(timelines);
324 186 :
325 186 : Ok(())
326 186 : }
327 :
328 : /// Orchestrates timeline shut down of all timeline tasks, removes its in-memory structures,
329 : /// and deletes its data from both disk and s3.
330 : /// The sequence of steps:
331 : /// 1. Set deleted_at in remote index part.
332 : /// 2. Create local mark file.
333 : /// 3. Delete local files except metadata (it is simpler this way, to be able to reuse timeline initialization code that expects metadata)
334 : /// 4. Delete remote layers
335 : /// 5. Delete index part
336 : /// 6. Delete meta, timeline directory
337 : /// 7. Delete mark file
338 : /// It is resumable from any step in case a crash/restart occurs.
339 : /// There are three entrypoints to the process:
340 : /// 1. [`DeleteTimelineFlow::run`] this is the main one called by a management api handler.
341 : /// 2. [`DeleteTimelineFlow::resume_deletion`] is called during restarts when local metadata is still present
342 : /// and we possibly neeed to continue deletion of remote files.
343 : /// 3. [`DeleteTimelineFlow::cleanup_remaining_timeline_fs_traces`] is used when we deleted remote
344 : /// index but still have local metadata, timeline directory and delete mark.
345 : /// Note the only other place that messes around timeline delete mark is the logic that scans directory with timelines during tenant load.
346 1302 : #[derive(Default)]
347 : pub enum DeleteTimelineFlow {
348 : #[default]
349 : NotStarted,
350 : InProgress,
351 : Finished,
352 : }
353 :
354 : impl DeleteTimelineFlow {
355 : // These steps are run in the context of management api request handler.
356 : // Long running steps are continued to run in the background.
357 : // NB: If this fails half-way through, and is retried, the retry will go through
358 : // all the same steps again. Make sure the code here is idempotent, and don't
359 : // error out if some of the shutdown tasks have already been completed!
360 454 : #[instrument(skip(tenant), fields(tenant_id=%tenant.tenant_id))]
361 : pub async fn run(
362 : tenant: &Arc<Tenant>,
363 : timeline_id: TimelineId,
364 : inplace: bool,
365 : ) -> Result<(), DeleteTimelineError> {
366 : let (timeline, mut guard) = Self::prepare(tenant, timeline_id)?;
367 :
368 : guard.mark_in_progress()?;
369 :
370 : stop_tasks(&timeline).await?;
371 :
372 : set_deleted_in_remote_index(&timeline).await?;
373 :
374 : create_delete_mark(tenant.conf, timeline.tenant_id, timeline.timeline_id).await?;
375 :
376 : fail::fail_point!("timeline-delete-before-schedule", |_| {
377 4 : Err(anyhow::anyhow!(
378 4 : "failpoint: timeline-delete-before-schedule"
379 4 : ))?
380 4 : });
381 :
382 : if inplace {
383 : Self::background(guard, tenant.conf, tenant, &timeline).await?
384 : } else {
385 : Self::schedule_background(guard, tenant.conf, Arc::clone(tenant), timeline);
386 : }
387 :
388 : Ok(())
389 : }
390 :
391 246 : fn mark_in_progress(&mut self) -> anyhow::Result<()> {
392 246 : match self {
393 UBC 0 : Self::Finished => anyhow::bail!("Bug. Is in finished state"),
394 CBC 26 : Self::InProgress { .. } => { /* We're in a retry */ }
395 220 : Self::NotStarted => { /* Fresh start */ }
396 : }
397 :
398 246 : *self = Self::InProgress;
399 246 :
400 246 : Ok(())
401 246 : }
402 :
403 : /// Shortcut to create Timeline in stopping state and spawn deletion task.
404 : /// See corresponding parts of [`crate::tenant::delete::DeleteTenantFlow`]
405 84 : #[instrument(skip_all, fields(%timeline_id))]
406 : pub async fn resume_deletion(
407 : tenant: Arc<Tenant>,
408 : timeline_id: TimelineId,
409 : local_metadata: &TimelineMetadata,
410 : remote_client: Option<RemoteTimelineClient>,
411 : deletion_queue_client: DeletionQueueClient,
412 : init_order: Option<&InitializationOrder>,
413 : ) -> anyhow::Result<()> {
414 : // Note: here we even skip populating layer map. Timeline is essentially uninitialized.
415 : // RemoteTimelineClient is the only functioning part.
416 : let timeline = tenant
417 : .create_timeline_struct(
418 : timeline_id,
419 : local_metadata,
420 : None, // Ancestor is not needed for deletion.
421 : TimelineResources {
422 : remote_client,
423 : deletion_queue_client,
424 : },
425 : init_order,
426 : // Important. We dont pass ancestor above because it can be missing.
427 : // Thus we need to skip the validation here.
428 : CreateTimelineCause::Delete,
429 : )
430 : .context("create_timeline_struct")?;
431 :
432 : let mut guard = DeletionGuard(
433 : Arc::clone(&timeline.delete_progress)
434 : .try_lock_owned()
435 : .expect("cannot happen because we're the only owner"),
436 : );
437 :
438 : // We meed to do this because when console retries delete request we shouldnt answer with 404
439 : // because 404 means successful deletion.
440 : {
441 : let mut locked = tenant.timelines.lock().unwrap();
442 : locked.insert(timeline_id, Arc::clone(&timeline));
443 : }
444 :
445 : guard.mark_in_progress()?;
446 :
447 : // Note that delete mark can be missing on resume
448 : // because we create delete mark after we set deleted_at in the index part.
449 : create_delete_mark(tenant.conf, tenant.tenant_id, timeline_id).await?;
450 :
451 : Self::schedule_background(guard, tenant.conf, tenant, timeline);
452 :
453 : Ok(())
454 : }
455 :
456 24 : #[instrument(skip_all, fields(%timeline_id))]
457 : pub async fn cleanup_remaining_timeline_fs_traces(
458 : tenant: &Tenant,
459 : timeline_id: TimelineId,
460 : ) -> anyhow::Result<()> {
461 : let r =
462 : cleanup_remaining_timeline_fs_traces(tenant.conf, tenant.tenant_id, timeline_id).await;
463 8 : info!("Done");
464 : r
465 : }
466 :
467 237 : fn prepare(
468 237 : tenant: &Tenant,
469 237 : timeline_id: TimelineId,
470 237 : ) -> Result<(Arc<Timeline>, DeletionGuard), DeleteTimelineError> {
471 237 : // Note the interaction between this guard and deletion guard.
472 237 : // Here we attempt to lock deletion guard when we're holding a lock on timelines.
473 237 : // This is important because when you take into account `remove_timeline_from_tenant`
474 237 : // we remove timeline from memory when we still hold the deletion guard.
475 237 : // So here when timeline deletion is finished timeline wont be present in timelines map at all
476 237 : // which makes the following sequence impossible:
477 237 : // T1: get preempted right before the try_lock on `Timeline::delete_progress`
478 237 : // T2: do a full deletion, acquire and drop `Timeline::delete_progress`
479 237 : // T1: acquire deletion lock, do another `DeleteTimelineFlow::run`
480 237 : // For more context see this discussion: `https://github.com/neondatabase/neon/pull/4552#discussion_r1253437346`
481 237 : let timelines = tenant.timelines.lock().unwrap();
482 :
483 237 : let timeline = match timelines.get(&timeline_id) {
484 233 : Some(t) => t,
485 4 : None => return Err(DeleteTimelineError::NotFound),
486 : };
487 :
488 : // Ensure that there are no child timelines **attached to that pageserver**,
489 : // because detach removes files, which will break child branches
490 233 : let children: Vec<TimelineId> = timelines
491 233 : .iter()
492 421 : .filter_map(|(id, entry)| {
493 421 : if entry.get_ancestor_timeline_id() == Some(timeline_id) {
494 1 : Some(*id)
495 : } else {
496 420 : None
497 : }
498 421 : })
499 233 : .collect();
500 233 :
501 233 : if !children.is_empty() {
502 1 : return Err(DeleteTimelineError::HasChildren(children));
503 232 : }
504 232 :
505 232 : // Note that using try_lock here is important to avoid a deadlock.
506 232 : // Here we take lock on timelines and then the deletion guard.
507 232 : // At the end of the operation we're holding the guard and need to lock timelines map
508 232 : // to remove the timeline from it.
509 232 : // Always if you have two locks that are taken in different order this can result in a deadlock.
510 232 :
511 232 : let delete_progress = Arc::clone(&timeline.delete_progress);
512 232 : let delete_lock_guard = match delete_progress.try_lock_owned() {
513 225 : Ok(guard) => DeletionGuard(guard),
514 : Err(_) => {
515 : // Unfortunately if lock fails arc is consumed.
516 7 : return Err(DeleteTimelineError::AlreadyInProgress(Arc::clone(
517 7 : &timeline.delete_progress,
518 7 : )));
519 : }
520 : };
521 :
522 225 : timeline.set_state(TimelineState::Stopping);
523 225 :
524 225 : Ok((Arc::clone(timeline), delete_lock_guard))
525 237 : }
526 :
527 106 : fn schedule_background(
528 106 : guard: DeletionGuard,
529 106 : conf: &'static PageServerConf,
530 106 : tenant: Arc<Tenant>,
531 106 : timeline: Arc<Timeline>,
532 106 : ) {
533 106 : let tenant_id = timeline.tenant_id;
534 106 : let timeline_id = timeline.timeline_id;
535 106 :
536 106 : task_mgr::spawn(
537 106 : task_mgr::BACKGROUND_RUNTIME.handle(),
538 106 : TaskKind::TimelineDeletionWorker,
539 106 : Some(tenant_id),
540 106 : Some(timeline_id),
541 106 : "timeline_delete",
542 : false,
543 106 : async move {
544 3276 : if let Err(err) = Self::background(guard, conf, &tenant, &timeline).await {
545 33 : error!("Error: {err:#}");
546 33 : timeline.set_broken(format!("{err:#}"))
547 73 : };
548 106 : Ok(())
549 106 : }
550 106 : .instrument({
551 106 : let span =
552 106 : tracing::info_span!(parent: None, "delete_timeline", tenant_id=%tenant_id, timeline_id=%timeline_id);
553 106 : span.follows_from(Span::current());
554 106 : span
555 106 : }),
556 106 : );
557 106 : }
558 :
559 234 : async fn background(
560 234 : mut guard: DeletionGuard,
561 234 : conf: &PageServerConf,
562 234 : tenant: &Tenant,
563 234 : timeline: &Timeline,
564 234 : ) -> Result<(), DeleteTimelineError> {
565 4827 : delete_local_layer_files(conf, tenant.tenant_id, timeline).await?;
566 :
567 1240 : delete_remote_layers_and_index(timeline).await?;
568 :
569 198 : pausable_failpoint!("in_progress_delete");
570 :
571 948 : cleanup_remaining_timeline_fs_traces(conf, tenant.tenant_id, timeline.timeline_id).await?;
572 :
573 186 : remove_timeline_from_tenant(tenant, timeline.timeline_id, &guard).await?;
574 :
575 186 : *guard = Self::Finished;
576 186 :
577 186 : Ok(())
578 234 : }
579 :
580 4 : pub(crate) fn is_finished(&self) -> bool {
581 4 : matches!(self, Self::Finished)
582 4 : }
583 : }
584 :
585 : struct DeletionGuard(OwnedMutexGuard<DeleteTimelineFlow>);
586 :
587 : impl Deref for DeletionGuard {
588 : type Target = DeleteTimelineFlow;
589 :
590 UBC 0 : fn deref(&self) -> &Self::Target {
591 0 : &self.0
592 0 : }
593 : }
594 :
595 : impl DerefMut for DeletionGuard {
596 CBC 432 : fn deref_mut(&mut self) -> &mut Self::Target {
597 432 : &mut self.0
598 432 : }
599 : }
|