Line data Source code
1 : mod deleter;
2 : mod list_writer;
3 : mod validator;
4 :
5 : use std::collections::HashMap;
6 : use std::sync::Arc;
7 : use std::time::Duration;
8 :
9 : use crate::control_plane_client::ControlPlaneGenerationsApi;
10 : use crate::metrics;
11 : use crate::tenant::remote_timeline_client::remote_layer_path;
12 : use crate::tenant::remote_timeline_client::remote_timeline_path;
13 : use crate::tenant::remote_timeline_client::LayerFileMetadata;
14 : use crate::virtual_file::MaybeFatalIo;
15 : use crate::virtual_file::VirtualFile;
16 : use anyhow::Context;
17 : use camino::Utf8PathBuf;
18 : use pageserver_api::shard::TenantShardId;
19 : use remote_storage::{GenericRemoteStorage, RemotePath};
20 : use serde::Deserialize;
21 : use serde::Serialize;
22 : use thiserror::Error;
23 : use tokio;
24 : use tokio_util::sync::CancellationToken;
25 : use tracing::Instrument;
26 : use tracing::{self, debug, error};
27 : use utils::crashsafe::path_with_suffix_extension;
28 : use utils::generation::Generation;
29 : use utils::id::TimelineId;
30 : use utils::lsn::AtomicLsn;
31 : use utils::lsn::Lsn;
32 :
33 : use self::deleter::Deleter;
34 : use self::list_writer::DeletionOp;
35 : use self::list_writer::ListWriter;
36 : use self::list_writer::RecoverOp;
37 : use self::validator::Validator;
38 : use deleter::DeleterMessage;
39 : use list_writer::ListWriterQueueMessage;
40 : use validator::ValidatorQueueMessage;
41 :
42 : use crate::{config::PageServerConf, tenant::storage_layer::LayerFileName};
43 :
44 : // TODO: configurable for how long to wait before executing deletions
45 :
46 : /// We aggregate object deletions from many tenants in one place, for several reasons:
47 : /// - Coalesce deletions into fewer DeleteObjects calls
48 : /// - Enable Tenant/Timeline lifetimes to be shorter than the time it takes
49 : /// to flush any outstanding deletions.
50 : /// - Globally control throughput of deletions, as these are a low priority task: do
51 : /// not compete with the same S3 clients/connections used for higher priority uploads.
52 : /// - Enable gating deletions on validation of a tenant's generation number, to make
53 : /// it safe to multi-attach tenants (see docs/rfcs/025-generation-numbers.md)
54 : ///
55 : /// There are two kinds of deletion: deferred and immediate. A deferred deletion
56 : /// may be intentionally delayed to protect passive readers of S3 data, and is
57 : /// subject to a generation number validation step. An immediate deletion is
58 : /// ready to execute immediately, and is only queued up so that it can be coalesced
59 : /// with other deletions in flight.
60 : ///
61 : /// Deferred deletions pass through three steps:
62 : /// - ListWriter: accumulate deletion requests from Timelines, and batch them up into
63 : /// DeletionLists, which are persisted to disk.
64 : /// - Validator: accumulate deletion lists, and validate them en-masse prior to passing
65 : /// the keys in the list onward for actual deletion. Also validate remote_consistent_lsn
66 : /// updates for running timelines.
67 : /// - Deleter: accumulate object keys that the validator has validated, and execute them in
68 : /// batches of 1000 keys via DeleteObjects.
69 : ///
70 : /// Non-deferred deletions, such as during timeline deletion, bypass the first
71 : /// two stages and are passed straight into the Deleter.
72 : ///
73 : /// Internally, each stage is joined by a channel to the next. On disk, there is only
74 : /// one queue (of DeletionLists), which is written by the frontend and consumed
75 : /// by the backend.
76 182 : #[derive(Clone)]
77 : pub struct DeletionQueue {
78 : client: DeletionQueueClient,
79 :
80 : // Parent cancellation token for the tokens passed into background workers
81 : cancel: CancellationToken,
82 : }
83 :
84 : /// Opaque wrapper around individual worker tasks, to avoid making the
85 : /// worker objects themselves public
86 : pub struct DeletionQueueWorkers<C>
87 : where
88 : C: ControlPlaneGenerationsApi + Send + Sync,
89 : {
90 : frontend: ListWriter,
91 : backend: Validator<C>,
92 : executor: Deleter,
93 : }
94 :
95 : impl<C> DeletionQueueWorkers<C>
96 : where
97 : C: ControlPlaneGenerationsApi + Send + Sync + 'static,
98 : {
99 632 : pub fn spawn_with(mut self, runtime: &tokio::runtime::Handle) -> tokio::task::JoinHandle<()> {
100 632 : let jh_frontend = runtime.spawn(async move {
101 632 : self.frontend
102 632 : .background()
103 632 : .instrument(tracing::info_span!(parent:None, "deletion frontend"))
104 3922 : .await
105 632 : });
106 632 : let jh_backend = runtime.spawn(async move {
107 632 : self.backend
108 632 : .background()
109 632 : .instrument(tracing::info_span!(parent:None, "deletion backend"))
110 1896 : .await
111 632 : });
112 632 : let jh_executor = runtime.spawn(async move {
113 632 : self.executor
114 632 : .background()
115 632 : .instrument(tracing::info_span!(parent:None, "deletion executor"))
116 11119 : .await
117 632 : });
118 632 :
119 632 : runtime.spawn({
120 632 : async move {
121 632 : jh_frontend.await.expect("error joining frontend worker");
122 3 : jh_backend.await.expect("error joining backend worker");
123 3 : drop(jh_executor.await.expect("error joining executor worker"));
124 632 : }
125 632 : })
126 632 : }
127 : }
128 :
129 : /// A FlushOp is just a oneshot channel, where we send the transmit side down
130 : /// another channel, and the receive side will receive a message when the channel
131 : /// we're flushing has reached the FlushOp we sent into it.
132 : ///
133 : /// The only extra behavior beyond the channel is that the notify() method does not
134 : /// return an error when the receive side has been dropped, because in this use case
135 : /// it is harmless (the code that initiated the flush no longer cares about the result).
136 0 : #[derive(Debug)]
137 : struct FlushOp {
138 : tx: tokio::sync::oneshot::Sender<()>,
139 : }
140 :
141 : impl FlushOp {
142 909 : fn new() -> (Self, tokio::sync::oneshot::Receiver<()>) {
143 909 : let (tx, rx) = tokio::sync::oneshot::channel::<()>();
144 909 : (Self { tx }, rx)
145 909 : }
146 :
147 907 : fn notify(self) {
148 907 : if self.tx.send(()).is_err() {
149 : // oneshot channel closed. This is legal: a client could be destroyed while waiting for a flush.
150 166 : debug!("deletion queue flush from dropped client");
151 741 : };
152 907 : }
153 : }
154 :
155 5506 : #[derive(Clone, Debug)]
156 : pub struct DeletionQueueClient {
157 : tx: tokio::sync::mpsc::UnboundedSender<ListWriterQueueMessage>,
158 : executor_tx: tokio::sync::mpsc::Sender<DeleterMessage>,
159 :
160 : lsn_table: Arc<std::sync::RwLock<VisibleLsnUpdates>>,
161 : }
162 :
163 200 : #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)]
164 : struct TenantDeletionList {
165 : /// For each Timeline, a list of key fragments to append to the timeline remote path
166 : /// when reconstructing a full key
167 : timelines: HashMap<TimelineId, Vec<String>>,
168 :
169 : /// The generation in which this deletion was emitted: note that this may not be the
170 : /// same as the generation of any layers being deleted. The generation of the layer
171 : /// has already been absorbed into the keys in `objects`
172 : generation: Generation,
173 : }
174 :
175 : impl TenantDeletionList {
176 40 : pub(crate) fn len(&self) -> usize {
177 52 : self.timelines.values().map(|v| v.len()).sum()
178 40 : }
179 : }
180 :
181 : /// Files ending with this suffix will be ignored and erased
182 : /// during recovery as startup.
183 : const TEMP_SUFFIX: &str = "tmp";
184 :
185 333 : #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)]
186 : struct DeletionList {
187 : /// Serialization version, for future use
188 : version: u8,
189 :
190 : /// Used for constructing a unique key for each deletion list we write out.
191 : sequence: u64,
192 :
193 : /// To avoid repeating tenant/timeline IDs in every key, we store keys in
194 : /// nested HashMaps by TenantTimelineID. Each Tenant only appears once
195 : /// with one unique generation ID: if someone tries to push a second generation
196 : /// ID for the same tenant, we will start a new DeletionList.
197 : tenants: HashMap<TenantShardId, TenantDeletionList>,
198 :
199 : /// Avoid having to walk `tenants` to calculate the number of keys in
200 : /// the nested deletion lists
201 : size: usize,
202 :
203 : /// Set to true when the list has undergone validation with the control
204 : /// plane and the remaining contents of `tenants` are valid. A list may
205 : /// also be implicitly marked valid by DeletionHeader.validated_sequence
206 : /// advancing to >= DeletionList.sequence
207 : #[serde(default)]
208 : #[serde(skip_serializing_if = "std::ops::Not::not")]
209 : validated: bool,
210 : }
211 :
212 90 : #[derive(Debug, Serialize, Deserialize)]
213 : struct DeletionHeader {
214 : /// Serialization version, for future use
215 : version: u8,
216 :
217 : /// The highest sequence number (inclusive) that has been validated. All deletion
218 : /// lists on disk with a sequence <= this value are safe to execute.
219 : validated_sequence: u64,
220 : }
221 :
222 : impl DeletionHeader {
223 : const VERSION_LATEST: u8 = 1;
224 :
225 38 : fn new(validated_sequence: u64) -> Self {
226 38 : Self {
227 38 : version: Self::VERSION_LATEST,
228 38 : validated_sequence,
229 38 : }
230 38 : }
231 :
232 38 : async fn save(&self, conf: &'static PageServerConf) -> anyhow::Result<()> {
233 0 : debug!("Saving deletion list header {:?}", self);
234 38 : let header_bytes = serde_json::to_vec(self).context("serialize deletion header")?;
235 38 : let header_path = conf.deletion_header_path();
236 38 : let temp_path = path_with_suffix_extension(&header_path, TEMP_SUFFIX);
237 38 : VirtualFile::crashsafe_overwrite(&header_path, &temp_path, &header_bytes)
238 8 : .await
239 38 : .maybe_fatal_err("save deletion header")?;
240 :
241 38 : Ok(())
242 38 : }
243 : }
244 :
245 : impl DeletionList {
246 : const VERSION_LATEST: u8 = 1;
247 692 : fn new(sequence: u64) -> Self {
248 692 : Self {
249 692 : version: Self::VERSION_LATEST,
250 692 : sequence,
251 692 : tenants: HashMap::new(),
252 692 : size: 0,
253 692 : validated: false,
254 692 : }
255 692 : }
256 :
257 804 : fn is_empty(&self) -> bool {
258 804 : self.tenants.is_empty()
259 804 : }
260 :
261 5436 : fn len(&self) -> usize {
262 5436 : self.size
263 5436 : }
264 :
265 : /// Returns true if the push was accepted, false if the caller must start a new
266 : /// deletion list.
267 4270 : fn push(
268 4270 : &mut self,
269 4270 : tenant: &TenantShardId,
270 4270 : timeline: &TimelineId,
271 4270 : generation: Generation,
272 4270 : objects: &mut Vec<RemotePath>,
273 4270 : ) -> bool {
274 4270 : if objects.is_empty() {
275 : // Avoid inserting an empty TimelineDeletionList: this preserves the property
276 : // that if we have no keys, then self.objects is empty (used in Self::is_empty)
277 426 : return true;
278 3844 : }
279 3844 :
280 3844 : let tenant_entry = self
281 3844 : .tenants
282 3844 : .entry(*tenant)
283 3844 : .or_insert_with(|| TenantDeletionList {
284 116 : timelines: HashMap::new(),
285 116 : generation,
286 3844 : });
287 3844 :
288 3844 : if tenant_entry.generation != generation {
289 : // Only one generation per tenant per list: signal to
290 : // caller to start a new list.
291 3 : return false;
292 3841 : }
293 3841 :
294 3841 : let timeline_entry = tenant_entry.timelines.entry(*timeline).or_default();
295 3841 :
296 3841 : let timeline_remote_path = remote_timeline_path(tenant, timeline);
297 3841 :
298 3841 : self.size += objects.len();
299 3841 : timeline_entry.extend(objects.drain(..).map(|p| {
300 3841 : p.strip_prefix(&timeline_remote_path)
301 3841 : .expect("Timeline paths always start with the timeline prefix")
302 3841 : .to_string()
303 3841 : }));
304 3841 : true
305 4270 : }
306 :
307 42 : fn into_remote_paths(self) -> Vec<RemotePath> {
308 42 : let mut result = Vec::new();
309 42 : for (tenant, tenant_deletions) in self.tenants.into_iter() {
310 40 : for (timeline, timeline_layers) in tenant_deletions.timelines.into_iter() {
311 40 : let timeline_remote_path = remote_timeline_path(&tenant, &timeline);
312 40 : result.extend(
313 40 : timeline_layers
314 40 : .into_iter()
315 1068 : .map(|l| timeline_remote_path.join(&Utf8PathBuf::from(l))),
316 40 : );
317 40 : }
318 : }
319 :
320 42 : result
321 42 : }
322 :
323 72 : async fn save(&self, conf: &'static PageServerConf) -> anyhow::Result<()> {
324 72 : let path = conf.deletion_list_path(self.sequence);
325 72 : let temp_path = path_with_suffix_extension(&path, TEMP_SUFFIX);
326 72 :
327 72 : let bytes = serde_json::to_vec(self).expect("Failed to serialize deletion list");
328 72 : VirtualFile::crashsafe_overwrite(&path, &temp_path, &bytes)
329 17 : .await
330 72 : .maybe_fatal_err("save deletion list")
331 72 : .map_err(Into::into)
332 72 : }
333 : }
334 :
335 : impl std::fmt::Display for DeletionList {
336 0 : fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
337 0 : write!(
338 0 : f,
339 0 : "DeletionList<seq={}, tenants={}, keys={}>",
340 0 : self.sequence,
341 0 : self.tenants.len(),
342 0 : self.size
343 0 : )
344 0 : }
345 : }
346 :
347 : struct PendingLsn {
348 : projected: Lsn,
349 : result_slot: Arc<AtomicLsn>,
350 : }
351 :
352 : struct TenantLsnState {
353 : timelines: HashMap<TimelineId, PendingLsn>,
354 :
355 : // In what generation was the most recent update proposed?
356 : generation: Generation,
357 : }
358 :
359 757 : #[derive(Default)]
360 : struct VisibleLsnUpdates {
361 : tenants: HashMap<TenantShardId, TenantLsnState>,
362 : }
363 :
364 : impl VisibleLsnUpdates {
365 714 : fn new() -> Self {
366 714 : Self {
367 714 : tenants: HashMap::new(),
368 714 : }
369 714 : }
370 : }
371 :
372 : impl std::fmt::Debug for VisibleLsnUpdates {
373 0 : fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
374 0 : write!(f, "VisibleLsnUpdates({} tenants)", self.tenants.len())
375 0 : }
376 : }
377 :
378 0 : #[derive(Error, Debug)]
379 : pub enum DeletionQueueError {
380 : #[error("Deletion queue unavailable during shutdown")]
381 : ShuttingDown,
382 : }
383 :
384 : impl DeletionQueueClient {
385 0 : pub(crate) fn broken() -> Self {
386 0 : // Channels whose receivers are immediately dropped.
387 0 : let (tx, _rx) = tokio::sync::mpsc::unbounded_channel();
388 0 : let (executor_tx, _executor_rx) = tokio::sync::mpsc::channel(1);
389 0 : Self {
390 0 : tx,
391 0 : executor_tx,
392 0 : lsn_table: Arc::default(),
393 0 : }
394 0 : }
395 :
396 : /// This is cancel-safe. If you drop the future before it completes, the message
397 : /// is not pushed, although in the context of the deletion queue it doesn't matter: once
398 : /// we decide to do a deletion the decision is always final.
399 5307 : fn do_push<T>(
400 5307 : &self,
401 5307 : queue: &tokio::sync::mpsc::UnboundedSender<T>,
402 5307 : msg: T,
403 5307 : ) -> Result<(), DeletionQueueError> {
404 5307 : match queue.send(msg) {
405 5307 : Ok(_) => Ok(()),
406 0 : Err(e) => {
407 0 : // This shouldn't happen, we should shut down all tenants before
408 0 : // we shut down the global delete queue. If we encounter a bug like this,
409 0 : // we may leak objects as deletions won't be processed.
410 0 : error!("Deletion queue closed while pushing, shutting down? ({e})");
411 0 : Err(DeletionQueueError::ShuttingDown)
412 : }
413 : }
414 5307 : }
415 :
416 631 : pub(crate) fn recover(
417 631 : &self,
418 631 : attached_tenants: HashMap<TenantShardId, Generation>,
419 631 : ) -> Result<(), DeletionQueueError> {
420 631 : self.do_push(
421 631 : &self.tx,
422 631 : ListWriterQueueMessage::Recover(RecoverOp { attached_tenants }),
423 631 : )
424 631 : }
425 :
426 : /// When a Timeline wishes to update the remote_consistent_lsn that it exposes to the outside
427 : /// world, it must validate its generation number before doing so. Rather than do this synchronously,
428 : /// we allow the timeline to publish updates at will via this API, and then read back what LSN was most
429 : /// recently validated separately.
430 : ///
431 : /// In this function we publish the LSN to the `projected` field of the timeline's entry in the VisibleLsnUpdates. The
432 : /// backend will later wake up and notice that the tenant's generation requires validation.
433 6026 : pub(crate) async fn update_remote_consistent_lsn(
434 6026 : &self,
435 6026 : tenant_shard_id: TenantShardId,
436 6026 : timeline_id: TimelineId,
437 6026 : current_generation: Generation,
438 6026 : lsn: Lsn,
439 6026 : result_slot: Arc<AtomicLsn>,
440 6026 : ) {
441 6026 : let mut locked = self
442 6026 : .lsn_table
443 6026 : .write()
444 6026 : .expect("Lock should never be poisoned");
445 6026 :
446 6026 : let tenant_entry = locked
447 6026 : .tenants
448 6026 : .entry(tenant_shard_id)
449 6026 : .or_insert(TenantLsnState {
450 6026 : timelines: HashMap::new(),
451 6026 : generation: current_generation,
452 6026 : });
453 6026 :
454 6026 : if tenant_entry.generation != current_generation {
455 2 : // Generation might have changed if we were detached and then re-attached: in this case,
456 2 : // state from the previous generation cannot be trusted.
457 2 : tenant_entry.timelines.clear();
458 2 : tenant_entry.generation = current_generation;
459 6024 : }
460 :
461 6026 : tenant_entry.timelines.insert(
462 6026 : timeline_id,
463 6026 : PendingLsn {
464 6026 : projected: lsn,
465 6026 : result_slot,
466 6026 : },
467 6026 : );
468 6026 : }
469 :
470 : /// Submit a list of layers for deletion: this function will return before the deletion is
471 : /// persistent, but it may be executed at any time after this function enters: do not push
472 : /// layers until you're sure they can be deleted safely (i.e. remote metadata no longer
473 : /// references them).
474 : ///
475 : /// The `current_generation` is the generation of this pageserver's current attachment. The
476 : /// generations in `layers` are the generations in which those layers were written.
477 4488 : pub(crate) async fn push_layers(
478 4488 : &self,
479 4488 : tenant_shard_id: TenantShardId,
480 4488 : timeline_id: TimelineId,
481 4488 : current_generation: Generation,
482 4488 : layers: Vec<(LayerFileName, LayerFileMetadata)>,
483 4488 : ) -> Result<(), DeletionQueueError> {
484 4488 : if current_generation.is_none() {
485 0 : debug!("Enqueuing deletions in legacy mode, skipping queue");
486 :
487 60 : let mut layer_paths = Vec::new();
488 120 : for (layer, meta) in layers {
489 60 : layer_paths.push(remote_layer_path(
490 60 : &tenant_shard_id.tenant_id,
491 60 : &timeline_id,
492 60 : meta.shard,
493 60 : &layer,
494 60 : meta.generation,
495 60 : ));
496 60 : }
497 60 : self.push_immediate(layer_paths).await?;
498 105 : return self.flush_immediate().await;
499 4428 : }
500 4428 :
501 4428 : self.push_layers_sync(tenant_shard_id, timeline_id, current_generation, layers)
502 4488 : }
503 :
504 : /// When a Tenant has a generation, push_layers is always synchronous because
505 : /// the ListValidator channel is an unbounded channel.
506 : ///
507 : /// This can be merged into push_layers when we remove the Generation-less mode
508 : /// support (`<https://github.com/neondatabase/neon/issues/5395>`)
509 4428 : pub(crate) fn push_layers_sync(
510 4428 : &self,
511 4428 : tenant_shard_id: TenantShardId,
512 4428 : timeline_id: TimelineId,
513 4428 : current_generation: Generation,
514 4428 : layers: Vec<(LayerFileName, LayerFileMetadata)>,
515 4428 : ) -> Result<(), DeletionQueueError> {
516 4428 : metrics::DELETION_QUEUE
517 4428 : .keys_submitted
518 4428 : .inc_by(layers.len() as u64);
519 4428 : self.do_push(
520 4428 : &self.tx,
521 4428 : ListWriterQueueMessage::Delete(DeletionOp {
522 4428 : tenant_shard_id,
523 4428 : timeline_id,
524 4428 : layers,
525 4428 : generation: current_generation,
526 4428 : objects: Vec::new(),
527 4428 : }),
528 4428 : )
529 4428 : }
530 :
531 : /// This is cancel-safe. If you drop the future the flush may still happen in the background.
532 248 : async fn do_flush<T>(
533 248 : &self,
534 248 : queue: &tokio::sync::mpsc::UnboundedSender<T>,
535 248 : msg: T,
536 248 : rx: tokio::sync::oneshot::Receiver<()>,
537 248 : ) -> Result<(), DeletionQueueError> {
538 248 : self.do_push(queue, msg)?;
539 249 : if rx.await.is_err() {
540 : // This shouldn't happen if tenants are shut down before deletion queue. If we
541 : // encounter a bug like this, then a flusher will incorrectly believe it has flushed
542 : // when it hasn't, possibly leading to leaking objects.
543 0 : error!("Deletion queue dropped flush op while client was still waiting");
544 0 : Err(DeletionQueueError::ShuttingDown)
545 : } else {
546 248 : Ok(())
547 : }
548 248 : }
549 :
550 : /// Wait until all previous deletions are persistent (either executed, or written to a DeletionList)
551 : ///
552 : /// This is cancel-safe. If you drop the future the flush may still happen in the background.
553 221 : pub async fn flush(&self) -> Result<(), DeletionQueueError> {
554 221 : let (flush_op, rx) = FlushOp::new();
555 221 : self.do_flush(&self.tx, ListWriterQueueMessage::Flush(flush_op), rx)
556 222 : .await
557 221 : }
558 :
559 : /// Issue a flush without waiting for it to complete. This is useful on advisory flushes where
560 : /// the caller wants to avoid the risk of waiting for lots of enqueued work, such as on tenant
561 : /// detach where flushing is nice but not necessary.
562 : ///
563 : /// This function provides no guarantees of work being done.
564 166 : pub fn flush_advisory(&self) {
565 166 : let (flush_op, _) = FlushOp::new();
566 166 :
567 166 : // Transmit the flush message, ignoring any result (such as a closed channel during shutdown).
568 166 : drop(self.tx.send(ListWriterQueueMessage::FlushExecute(flush_op)));
569 166 : }
570 :
571 : // Wait until all previous deletions are executed
572 27 : pub(crate) async fn flush_execute(&self) -> Result<(), DeletionQueueError> {
573 0 : debug!("flush_execute: flushing to deletion lists...");
574 : // Flush any buffered work to deletion lists
575 27 : self.flush().await?;
576 :
577 : // Flush the backend into the executor of deletion lists
578 27 : let (flush_op, rx) = FlushOp::new();
579 0 : debug!("flush_execute: flushing backend...");
580 27 : self.do_flush(&self.tx, ListWriterQueueMessage::FlushExecute(flush_op), rx)
581 27 : .await?;
582 0 : debug!("flush_execute: finished flushing backend...");
583 :
584 : // Flush any immediate-mode deletions (the above backend flush will only flush
585 : // the executor if deletions had flowed through the backend)
586 0 : debug!("flush_execute: flushing execution...");
587 27 : self.flush_immediate().await?;
588 0 : debug!("flush_execute: finished flushing execution...");
589 27 : Ok(())
590 27 : }
591 :
592 : /// This interface bypasses the persistent deletion queue, and any validation
593 : /// that this pageserver is still elegible to execute the deletions. It is for
594 : /// use in timeline deletions, where the control plane is telling us we may
595 : /// delete everything in the timeline.
596 : ///
597 : /// DO NOT USE THIS FROM GC OR COMPACTION CODE. Use the regular `push_layers`.
598 718 : pub(crate) async fn push_immediate(
599 718 : &self,
600 718 : objects: Vec<RemotePath>,
601 718 : ) -> Result<(), DeletionQueueError> {
602 718 : metrics::DELETION_QUEUE
603 718 : .keys_submitted
604 718 : .inc_by(objects.len() as u64);
605 718 : self.executor_tx
606 718 : .send(DeleterMessage::Delete(objects))
607 45 : .await
608 718 : .map_err(|_| DeletionQueueError::ShuttingDown)
609 718 : }
610 :
611 : /// Companion to push_immediate. When this returns Ok, all prior objects sent
612 : /// into push_immediate have been deleted from remote storage.
613 455 : pub(crate) async fn flush_immediate(&self) -> Result<(), DeletionQueueError> {
614 455 : let (flush_op, rx) = FlushOp::new();
615 455 : self.executor_tx
616 455 : .send(DeleterMessage::Flush(flush_op))
617 45 : .await
618 455 : .map_err(|_| DeletionQueueError::ShuttingDown)?;
619 :
620 455 : rx.await.map_err(|_| DeletionQueueError::ShuttingDown)
621 455 : }
622 : }
623 :
624 : impl DeletionQueue {
625 1256 : pub fn new_client(&self) -> DeletionQueueClient {
626 1256 : self.client.clone()
627 1256 : }
628 :
629 : /// Caller may use the returned object to construct clients with new_client.
630 : /// Caller should tokio::spawn the background() members of the two worker objects returned:
631 : /// we don't spawn those inside new() so that the caller can use their runtime/spans of choice.
632 : ///
633 : /// If remote_storage is None, then the returned workers will also be None.
634 632 : pub fn new<C>(
635 632 : remote_storage: Option<GenericRemoteStorage>,
636 632 : control_plane_client: Option<C>,
637 632 : conf: &'static PageServerConf,
638 632 : ) -> (Self, Option<DeletionQueueWorkers<C>>)
639 632 : where
640 632 : C: ControlPlaneGenerationsApi + Send + Sync,
641 632 : {
642 632 : // Unbounded channel: enables non-async functions to submit deletions. The actual length is
643 632 : // constrained by how promptly the ListWriter wakes up and drains it, which should be frequent
644 632 : // enough to avoid this taking pathologically large amount of memory.
645 632 : let (tx, rx) = tokio::sync::mpsc::unbounded_channel();
646 632 :
647 632 : // Shallow channel: it carries DeletionLists which each contain up to thousands of deletions
648 632 : let (backend_tx, backend_rx) = tokio::sync::mpsc::channel(16);
649 632 :
650 632 : // Shallow channel: it carries lists of paths, and we expect the main queueing to
651 632 : // happen in the backend (persistent), not in this queue.
652 632 : let (executor_tx, executor_rx) = tokio::sync::mpsc::channel(16);
653 632 :
654 632 : let lsn_table = Arc::new(std::sync::RwLock::new(VisibleLsnUpdates::new()));
655 632 :
656 632 : // The deletion queue has an independent cancellation token to
657 632 : // the general pageserver shutdown token, because it stays alive a bit
658 632 : // longer to flush after Tenants have all been torn down.
659 632 : let cancel = CancellationToken::new();
660 :
661 632 : let remote_storage = match remote_storage {
662 : None => {
663 0 : return (
664 0 : Self {
665 0 : client: DeletionQueueClient {
666 0 : tx,
667 0 : executor_tx,
668 0 : lsn_table: lsn_table.clone(),
669 0 : },
670 0 : cancel,
671 0 : },
672 0 : None,
673 0 : )
674 : }
675 632 : Some(r) => r,
676 632 : };
677 632 :
678 632 : (
679 632 : Self {
680 632 : client: DeletionQueueClient {
681 632 : tx,
682 632 : executor_tx: executor_tx.clone(),
683 632 : lsn_table: lsn_table.clone(),
684 632 : },
685 632 : cancel: cancel.clone(),
686 632 : },
687 632 : Some(DeletionQueueWorkers {
688 632 : frontend: ListWriter::new(conf, rx, backend_tx, cancel.clone()),
689 632 : backend: Validator::new(
690 632 : conf,
691 632 : backend_rx,
692 632 : executor_tx,
693 632 : control_plane_client,
694 632 : lsn_table.clone(),
695 632 : cancel.clone(),
696 632 : ),
697 632 : executor: Deleter::new(remote_storage, executor_rx, cancel.clone()),
698 632 : }),
699 632 : )
700 632 : }
701 :
702 182 : pub async fn shutdown(&mut self, timeout: Duration) {
703 183 : match tokio::time::timeout(timeout, self.client.flush()).await {
704 : Ok(Ok(())) => {
705 182 : tracing::info!("Deletion queue flushed successfully on shutdown")
706 : }
707 : Ok(Err(DeletionQueueError::ShuttingDown)) => {
708 : // This is not harmful for correctness, but is unexpected: the deletion
709 : // queue's workers should stay alive as long as there are any client handles instantiated.
710 0 : tracing::warn!("Deletion queue stopped prematurely");
711 : }
712 0 : Err(_timeout) => {
713 0 : tracing::warn!("Timed out flushing deletion queue on shutdown")
714 : }
715 : }
716 :
717 : // We only cancel _after_ flushing: otherwise we would be shutting down the
718 : // components that do the flush.
719 182 : self.cancel.cancel();
720 182 : }
721 : }
722 :
723 : #[cfg(test)]
724 : mod test {
725 : use camino::Utf8Path;
726 : use hex_literal::hex;
727 : use pageserver_api::shard::ShardIndex;
728 : use std::{io::ErrorKind, time::Duration};
729 : use tracing::info;
730 :
731 : use remote_storage::{RemoteStorageConfig, RemoteStorageKind};
732 : use tokio::task::JoinHandle;
733 :
734 : use crate::{
735 : control_plane_client::RetryForeverError,
736 : repository::Key,
737 : tenant::{
738 : harness::TenantHarness, remote_timeline_client::remote_timeline_path,
739 : storage_layer::DeltaFileName,
740 : },
741 : };
742 :
743 : use super::*;
744 : pub const TIMELINE_ID: TimelineId =
745 : TimelineId::from_array(hex!("11223344556677881122334455667788"));
746 :
747 : pub const EXAMPLE_LAYER_NAME: LayerFileName = LayerFileName::Delta(DeltaFileName {
748 : key_range: Key::from_i128(0x0)..Key::from_i128(0xFFFFFFFFFFFFFFFF),
749 : lsn_range: Lsn(0x00000000016B59D8)..Lsn(0x00000000016B5A51),
750 : });
751 :
752 : // When you need a second layer in a test.
753 : pub const EXAMPLE_LAYER_NAME_ALT: LayerFileName = LayerFileName::Delta(DeltaFileName {
754 : key_range: Key::from_i128(0x0)..Key::from_i128(0xFFFFFFFFFFFFFFFF),
755 : lsn_range: Lsn(0x00000000016B5A51)..Lsn(0x00000000016B5A61),
756 : });
757 :
758 : struct TestSetup {
759 : harness: TenantHarness,
760 : remote_fs_dir: Utf8PathBuf,
761 : storage: GenericRemoteStorage,
762 : mock_control_plane: MockControlPlane,
763 : deletion_queue: DeletionQueue,
764 : worker_join: JoinHandle<()>,
765 : }
766 :
767 : impl TestSetup {
768 : /// Simulate a pageserver restart by destroying and recreating the deletion queue
769 2 : async fn restart(&mut self) {
770 2 : let (deletion_queue, workers) = DeletionQueue::new(
771 2 : Some(self.storage.clone()),
772 2 : Some(self.mock_control_plane.clone()),
773 2 : self.harness.conf,
774 2 : );
775 :
776 0 : tracing::debug!("Spawning worker for new queue queue");
777 2 : let worker_join = workers
778 2 : .unwrap()
779 2 : .spawn_with(&tokio::runtime::Handle::current());
780 2 :
781 2 : let old_worker_join = std::mem::replace(&mut self.worker_join, worker_join);
782 2 : let old_deletion_queue = std::mem::replace(&mut self.deletion_queue, deletion_queue);
783 :
784 0 : tracing::debug!("Joining worker from previous queue");
785 2 : old_deletion_queue.cancel.cancel();
786 2 : old_worker_join
787 2 : .await
788 2 : .expect("Failed to join workers for previous deletion queue");
789 2 : }
790 :
791 6 : fn set_latest_generation(&self, gen: Generation) {
792 6 : let tenant_shard_id = self.harness.tenant_shard_id;
793 6 : self.mock_control_plane
794 6 : .latest_generation
795 6 : .lock()
796 6 : .unwrap()
797 6 : .insert(tenant_shard_id, gen);
798 6 : }
799 :
800 : /// Returns remote layer file name, suitable for use in assert_remote_files
801 6 : fn write_remote_layer(
802 6 : &self,
803 6 : file_name: LayerFileName,
804 6 : gen: Generation,
805 6 : ) -> anyhow::Result<String> {
806 6 : let tenant_shard_id = self.harness.tenant_shard_id;
807 6 : let relative_remote_path = remote_timeline_path(&tenant_shard_id, &TIMELINE_ID);
808 6 : let remote_timeline_path = self.remote_fs_dir.join(relative_remote_path.get_path());
809 6 : std::fs::create_dir_all(&remote_timeline_path)?;
810 6 : let remote_layer_file_name = format!("{}{}", file_name, gen.get_suffix());
811 6 :
812 6 : let content: Vec<u8> = format!("placeholder contents of {file_name}").into();
813 6 :
814 6 : std::fs::write(
815 6 : remote_timeline_path.join(remote_layer_file_name.clone()),
816 6 : content,
817 6 : )?;
818 :
819 6 : Ok(remote_layer_file_name)
820 6 : }
821 : }
822 :
823 8 : #[derive(Debug, Clone)]
824 : struct MockControlPlane {
825 : pub latest_generation: std::sync::Arc<std::sync::Mutex<HashMap<TenantShardId, Generation>>>,
826 : }
827 :
828 : impl MockControlPlane {
829 6 : fn new() -> Self {
830 6 : Self {
831 6 : latest_generation: Arc::default(),
832 6 : }
833 6 : }
834 : }
835 :
836 : impl ControlPlaneGenerationsApi for MockControlPlane {
837 : #[allow(clippy::diverging_sub_expression)] // False positive via async_trait
838 0 : async fn re_attach(&self) -> Result<HashMap<TenantShardId, Generation>, RetryForeverError> {
839 0 : unimplemented!()
840 0 : }
841 8 : async fn validate(
842 8 : &self,
843 8 : tenants: Vec<(TenantShardId, Generation)>,
844 8 : ) -> Result<HashMap<TenantShardId, bool>, RetryForeverError> {
845 8 : let mut result = HashMap::new();
846 8 :
847 8 : let latest_generation = self.latest_generation.lock().unwrap();
848 :
849 16 : for (tenant_shard_id, generation) in tenants {
850 8 : if let Some(latest) = latest_generation.get(&tenant_shard_id) {
851 8 : result.insert(tenant_shard_id, *latest == generation);
852 8 : }
853 : }
854 :
855 8 : Ok(result)
856 8 : }
857 : }
858 :
859 6 : fn setup(test_name: &str) -> anyhow::Result<TestSetup> {
860 6 : let test_name = Box::leak(Box::new(format!("deletion_queue__{test_name}")));
861 6 : let harness = TenantHarness::create(test_name)?;
862 :
863 : // We do not load() the harness: we only need its config and remote_storage
864 :
865 : // Set up a GenericRemoteStorage targetting a directory
866 6 : let remote_fs_dir = harness.conf.workdir.join("remote_fs");
867 6 : std::fs::create_dir_all(remote_fs_dir)?;
868 6 : let remote_fs_dir = harness.conf.workdir.join("remote_fs").canonicalize_utf8()?;
869 6 : let storage_config = RemoteStorageConfig {
870 6 : storage: RemoteStorageKind::LocalFs(remote_fs_dir.clone()),
871 6 : };
872 6 : let storage = GenericRemoteStorage::from_config(&storage_config).unwrap();
873 6 :
874 6 : let mock_control_plane = MockControlPlane::new();
875 6 :
876 6 : let (deletion_queue, worker) = DeletionQueue::new(
877 6 : Some(storage.clone()),
878 6 : Some(mock_control_plane.clone()),
879 6 : harness.conf,
880 6 : );
881 6 :
882 6 : let worker = worker.unwrap();
883 6 : let worker_join = worker.spawn_with(&tokio::runtime::Handle::current());
884 6 :
885 6 : Ok(TestSetup {
886 6 : harness,
887 6 : remote_fs_dir,
888 6 : storage,
889 6 : mock_control_plane,
890 6 : deletion_queue,
891 6 : worker_join,
892 6 : })
893 6 : }
894 :
895 : // TODO: put this in a common location so that we can share with remote_timeline_client's tests
896 18 : fn assert_remote_files(expected: &[&str], remote_path: &Utf8Path) {
897 18 : let mut expected: Vec<String> = expected.iter().map(|x| String::from(*x)).collect();
898 18 : expected.sort();
899 18 :
900 18 : let mut found: Vec<String> = Vec::new();
901 18 : let dir = match std::fs::read_dir(remote_path) {
902 18 : Ok(d) => d,
903 0 : Err(e) => {
904 0 : if e.kind() == ErrorKind::NotFound {
905 0 : if expected.is_empty() {
906 : // We are asserting prefix is empty: it is expected that the dir is missing
907 0 : return;
908 : } else {
909 0 : assert_eq!(expected, Vec::<String>::new());
910 0 : unreachable!();
911 : }
912 : } else {
913 0 : panic!("Unexpected error listing {remote_path}: {e}");
914 : }
915 : }
916 : };
917 :
918 18 : for entry in dir.flatten() {
919 16 : let entry_name = entry.file_name();
920 16 : let fname = entry_name.to_str().unwrap();
921 16 : found.push(String::from(fname));
922 16 : }
923 18 : found.sort();
924 18 :
925 18 : assert_eq!(expected, found);
926 18 : }
927 :
928 10 : fn assert_local_files(expected: &[&str], directory: &Utf8Path) {
929 10 : let dir = match std::fs::read_dir(directory) {
930 8 : Ok(d) => d,
931 : Err(_) => {
932 2 : assert_eq!(expected, &Vec::<String>::new());
933 2 : return;
934 : }
935 : };
936 8 : let mut found = Vec::new();
937 18 : for dentry in dir {
938 10 : let dentry = dentry.unwrap();
939 10 : let file_name = dentry.file_name();
940 10 : let file_name_str = file_name.to_string_lossy();
941 10 : found.push(file_name_str.to_string());
942 10 : }
943 8 : found.sort();
944 8 : assert_eq!(expected, found);
945 10 : }
946 :
947 2 : #[tokio::test]
948 2 : async fn deletion_queue_smoke() -> anyhow::Result<()> {
949 2 : // Basic test that the deletion queue processes the deletions we pass into it
950 2 : let ctx = setup("deletion_queue_smoke").expect("Failed test setup");
951 2 : let client = ctx.deletion_queue.new_client();
952 2 : client.recover(HashMap::new())?;
953 2 :
954 2 : let layer_file_name_1: LayerFileName = "000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D8-00000000016B5A51".parse().unwrap();
955 2 : let tenant_shard_id = ctx.harness.tenant_shard_id;
956 2 :
957 2 : let content: Vec<u8> = "victim1 contents".into();
958 2 : let relative_remote_path = remote_timeline_path(&tenant_shard_id, &TIMELINE_ID);
959 2 : let remote_timeline_path = ctx.remote_fs_dir.join(relative_remote_path.get_path());
960 2 : let deletion_prefix = ctx.harness.conf.deletion_prefix();
961 2 :
962 2 : // Exercise the distinction between the generation of the layers
963 2 : // we delete, and the generation of the running Tenant.
964 2 : let layer_generation = Generation::new(0xdeadbeef);
965 2 : let now_generation = Generation::new(0xfeedbeef);
966 2 : let layer_metadata =
967 2 : LayerFileMetadata::new(0xf00, layer_generation, ShardIndex::unsharded());
968 2 :
969 2 : let remote_layer_file_name_1 =
970 2 : format!("{}{}", layer_file_name_1, layer_generation.get_suffix());
971 2 :
972 2 : // Set mock control plane state to valid for our generation
973 2 : ctx.set_latest_generation(now_generation);
974 2 :
975 2 : // Inject a victim file to remote storage
976 2 : info!("Writing");
977 2 : std::fs::create_dir_all(&remote_timeline_path)?;
978 2 : std::fs::write(
979 2 : remote_timeline_path.join(remote_layer_file_name_1.clone()),
980 2 : content,
981 2 : )?;
982 2 : assert_remote_files(&[&remote_layer_file_name_1], &remote_timeline_path);
983 2 :
984 2 : // File should still be there after we push it to the queue (we haven't pushed enough to flush anything)
985 2 : info!("Pushing");
986 2 : client
987 2 : .push_layers(
988 2 : tenant_shard_id,
989 2 : TIMELINE_ID,
990 2 : now_generation,
991 2 : [(layer_file_name_1.clone(), layer_metadata)].to_vec(),
992 2 : )
993 2 : .await?;
994 2 : assert_remote_files(&[&remote_layer_file_name_1], &remote_timeline_path);
995 2 :
996 2 : assert_local_files(&[], &deletion_prefix);
997 2 :
998 2 : // File should still be there after we write a deletion list (we haven't pushed enough to execute anything)
999 2 : info!("Flushing");
1000 2 : client.flush().await?;
1001 2 : assert_remote_files(&[&remote_layer_file_name_1], &remote_timeline_path);
1002 2 : assert_local_files(&["0000000000000001-01.list"], &deletion_prefix);
1003 2 :
1004 2 : // File should go away when we execute
1005 2 : info!("Flush-executing");
1006 6 : client.flush_execute().await?;
1007 2 : assert_remote_files(&[], &remote_timeline_path);
1008 2 : assert_local_files(&["header-01"], &deletion_prefix);
1009 2 :
1010 2 : // Flushing on an empty queue should succeed immediately, and not write any lists
1011 2 : info!("Flush-executing on empty");
1012 6 : client.flush_execute().await?;
1013 2 : assert_local_files(&["header-01"], &deletion_prefix);
1014 2 :
1015 2 : Ok(())
1016 2 : }
1017 :
1018 2 : #[tokio::test]
1019 2 : async fn deletion_queue_validation() -> anyhow::Result<()> {
1020 2 : let ctx = setup("deletion_queue_validation").expect("Failed test setup");
1021 2 : let client = ctx.deletion_queue.new_client();
1022 2 : client.recover(HashMap::new())?;
1023 2 :
1024 2 : // Generation that the control plane thinks is current
1025 2 : let latest_generation = Generation::new(0xdeadbeef);
1026 2 : // Generation that our DeletionQueue thinks the tenant is running with
1027 2 : let stale_generation = latest_generation.previous();
1028 2 : // Generation that our example layer file was written with
1029 2 : let layer_generation = stale_generation.previous();
1030 2 : let layer_metadata =
1031 2 : LayerFileMetadata::new(0xf00, layer_generation, ShardIndex::unsharded());
1032 2 :
1033 2 : ctx.set_latest_generation(latest_generation);
1034 2 :
1035 2 : let tenant_shard_id = ctx.harness.tenant_shard_id;
1036 2 : let relative_remote_path = remote_timeline_path(&tenant_shard_id, &TIMELINE_ID);
1037 2 : let remote_timeline_path = ctx.remote_fs_dir.join(relative_remote_path.get_path());
1038 2 :
1039 2 : // Initial state: a remote layer exists
1040 2 : let remote_layer_name = ctx.write_remote_layer(EXAMPLE_LAYER_NAME, layer_generation)?;
1041 2 : assert_remote_files(&[&remote_layer_name], &remote_timeline_path);
1042 2 :
1043 2 : tracing::debug!("Pushing...");
1044 2 : client
1045 2 : .push_layers(
1046 2 : tenant_shard_id,
1047 2 : TIMELINE_ID,
1048 2 : stale_generation,
1049 2 : [(EXAMPLE_LAYER_NAME.clone(), layer_metadata.clone())].to_vec(),
1050 2 : )
1051 2 : .await?;
1052 2 :
1053 2 : // We enqueued the operation in a stale generation: it should have failed validation
1054 2 : tracing::debug!("Flushing...");
1055 6 : tokio::time::timeout(Duration::from_secs(5), client.flush_execute()).await??;
1056 2 : assert_remote_files(&[&remote_layer_name], &remote_timeline_path);
1057 2 :
1058 2 : tracing::debug!("Pushing...");
1059 2 : client
1060 2 : .push_layers(
1061 2 : tenant_shard_id,
1062 2 : TIMELINE_ID,
1063 2 : latest_generation,
1064 2 : [(EXAMPLE_LAYER_NAME.clone(), layer_metadata.clone())].to_vec(),
1065 2 : )
1066 2 : .await?;
1067 2 :
1068 2 : // We enqueued the operation in a fresh generation: it should have passed validation
1069 2 : tracing::debug!("Flushing...");
1070 6 : tokio::time::timeout(Duration::from_secs(5), client.flush_execute()).await??;
1071 2 : assert_remote_files(&[], &remote_timeline_path);
1072 2 :
1073 2 : Ok(())
1074 2 : }
1075 :
1076 2 : #[tokio::test]
1077 2 : async fn deletion_queue_recovery() -> anyhow::Result<()> {
1078 2 : // Basic test that the deletion queue processes the deletions we pass into it
1079 2 : let mut ctx = setup("deletion_queue_recovery").expect("Failed test setup");
1080 2 : let client = ctx.deletion_queue.new_client();
1081 2 : client.recover(HashMap::new())?;
1082 2 :
1083 2 : let tenant_shard_id = ctx.harness.tenant_shard_id;
1084 2 :
1085 2 : let relative_remote_path = remote_timeline_path(&tenant_shard_id, &TIMELINE_ID);
1086 2 : let remote_timeline_path = ctx.remote_fs_dir.join(relative_remote_path.get_path());
1087 2 : let deletion_prefix = ctx.harness.conf.deletion_prefix();
1088 2 :
1089 2 : let layer_generation = Generation::new(0xdeadbeef);
1090 2 : let now_generation = Generation::new(0xfeedbeef);
1091 2 : let layer_metadata =
1092 2 : LayerFileMetadata::new(0xf00, layer_generation, ShardIndex::unsharded());
1093 2 :
1094 2 : // Inject a deletion in the generation before generation_now: after restart,
1095 2 : // this deletion should _not_ get executed (only the immediately previous
1096 2 : // generation gets that treatment)
1097 2 : let remote_layer_file_name_historical =
1098 2 : ctx.write_remote_layer(EXAMPLE_LAYER_NAME, layer_generation)?;
1099 2 : client
1100 2 : .push_layers(
1101 2 : tenant_shard_id,
1102 2 : TIMELINE_ID,
1103 2 : now_generation.previous(),
1104 2 : [(EXAMPLE_LAYER_NAME.clone(), layer_metadata.clone())].to_vec(),
1105 2 : )
1106 2 : .await?;
1107 2 :
1108 2 : // Inject a deletion in the generation before generation_now: after restart,
1109 2 : // this deletion should get executed, because we execute deletions in the
1110 2 : // immediately previous generation on the same node.
1111 2 : let remote_layer_file_name_previous =
1112 2 : ctx.write_remote_layer(EXAMPLE_LAYER_NAME_ALT, layer_generation)?;
1113 2 : client
1114 2 : .push_layers(
1115 2 : tenant_shard_id,
1116 2 : TIMELINE_ID,
1117 2 : now_generation,
1118 2 : [(EXAMPLE_LAYER_NAME_ALT.clone(), layer_metadata.clone())].to_vec(),
1119 2 : )
1120 2 : .await?;
1121 2 :
1122 2 : client.flush().await?;
1123 2 : assert_remote_files(
1124 2 : &[
1125 2 : &remote_layer_file_name_historical,
1126 2 : &remote_layer_file_name_previous,
1127 2 : ],
1128 2 : &remote_timeline_path,
1129 2 : );
1130 2 :
1131 2 : // Different generatinos for the same tenant will cause two separate
1132 2 : // deletion lists to be emitted.
1133 2 : assert_local_files(
1134 2 : &["0000000000000001-01.list", "0000000000000002-01.list"],
1135 2 : &deletion_prefix,
1136 2 : );
1137 2 :
1138 2 : // Simulate a node restart: the latest generation advances
1139 2 : let now_generation = now_generation.next();
1140 2 : ctx.set_latest_generation(now_generation);
1141 2 :
1142 2 : // Restart the deletion queue
1143 2 : drop(client);
1144 2 : ctx.restart().await;
1145 2 : let client = ctx.deletion_queue.new_client();
1146 2 : client.recover(HashMap::from([(tenant_shard_id, now_generation)]))?;
1147 2 :
1148 2 : info!("Flush-executing");
1149 6 : client.flush_execute().await?;
1150 2 : // The deletion from immediately prior generation was executed, the one from
1151 2 : // an older generation was not.
1152 2 : assert_remote_files(&[&remote_layer_file_name_historical], &remote_timeline_path);
1153 2 : Ok(())
1154 2 : }
1155 : }
1156 :
1157 : /// A lightweight queue which can issue ordinary DeletionQueueClient objects, but doesn't do any persistence
1158 : /// or coalescing, and doesn't actually execute any deletions unless you call pump() to kick it.
1159 : #[cfg(test)]
1160 : pub(crate) mod mock {
1161 : use tracing::info;
1162 :
1163 : use crate::tenant::remote_timeline_client::remote_layer_path;
1164 :
1165 : use super::*;
1166 : use std::sync::{
1167 : atomic::{AtomicUsize, Ordering},
1168 : Arc,
1169 : };
1170 :
1171 : pub struct ConsumerState {
1172 : rx: tokio::sync::mpsc::UnboundedReceiver<ListWriterQueueMessage>,
1173 : executor_rx: tokio::sync::mpsc::Receiver<DeleterMessage>,
1174 : }
1175 :
1176 : impl ConsumerState {
1177 2 : async fn consume(&mut self, remote_storage: &GenericRemoteStorage) -> usize {
1178 2 : let mut executed = 0;
1179 :
1180 2 : info!("Executing all pending deletions");
1181 :
1182 : // Transform all executor messages to generic frontend messages
1183 2 : while let Ok(msg) = self.executor_rx.try_recv() {
1184 0 : match msg {
1185 0 : DeleterMessage::Delete(objects) => {
1186 0 : for path in objects {
1187 0 : match remote_storage.delete(&path).await {
1188 : Ok(_) => {
1189 0 : debug!("Deleted {path}");
1190 : }
1191 0 : Err(e) => {
1192 0 : error!("Failed to delete {path}, leaking object! ({e})");
1193 : }
1194 : }
1195 0 : executed += 1;
1196 : }
1197 : }
1198 0 : DeleterMessage::Flush(flush_op) => {
1199 0 : flush_op.notify();
1200 0 : }
1201 : }
1202 : }
1203 :
1204 4 : while let Ok(msg) = self.rx.try_recv() {
1205 2 : match msg {
1206 2 : ListWriterQueueMessage::Delete(op) => {
1207 2 : let mut objects = op.objects;
1208 4 : for (layer, meta) in op.layers {
1209 2 : objects.push(remote_layer_path(
1210 2 : &op.tenant_shard_id.tenant_id,
1211 2 : &op.timeline_id,
1212 2 : meta.shard,
1213 2 : &layer,
1214 2 : meta.generation,
1215 2 : ));
1216 2 : }
1217 :
1218 4 : for path in objects {
1219 2 : info!("Executing deletion {path}");
1220 2 : match remote_storage.delete(&path).await {
1221 : Ok(_) => {
1222 0 : debug!("Deleted {path}");
1223 : }
1224 0 : Err(e) => {
1225 0 : error!("Failed to delete {path}, leaking object! ({e})");
1226 : }
1227 : }
1228 2 : executed += 1;
1229 : }
1230 : }
1231 0 : ListWriterQueueMessage::Flush(op) => {
1232 0 : op.notify();
1233 0 : }
1234 0 : ListWriterQueueMessage::FlushExecute(op) => {
1235 0 : // We have already executed all prior deletions because mock does them inline
1236 0 : op.notify();
1237 0 : }
1238 0 : ListWriterQueueMessage::Recover(_) => {
1239 0 : // no-op in mock
1240 0 : }
1241 : }
1242 2 : info!("All pending deletions have been executed");
1243 : }
1244 :
1245 2 : executed
1246 2 : }
1247 : }
1248 :
1249 : pub struct MockDeletionQueue {
1250 : tx: tokio::sync::mpsc::UnboundedSender<ListWriterQueueMessage>,
1251 : executor_tx: tokio::sync::mpsc::Sender<DeleterMessage>,
1252 : executed: Arc<AtomicUsize>,
1253 : remote_storage: Option<GenericRemoteStorage>,
1254 : consumer: std::sync::Mutex<ConsumerState>,
1255 : lsn_table: Arc<std::sync::RwLock<VisibleLsnUpdates>>,
1256 : }
1257 :
1258 : impl MockDeletionQueue {
1259 82 : pub fn new(remote_storage: Option<GenericRemoteStorage>) -> Self {
1260 82 : let (tx, rx) = tokio::sync::mpsc::unbounded_channel();
1261 82 : let (executor_tx, executor_rx) = tokio::sync::mpsc::channel(16384);
1262 82 :
1263 82 : let executed = Arc::new(AtomicUsize::new(0));
1264 82 :
1265 82 : Self {
1266 82 : tx,
1267 82 : executor_tx,
1268 82 : executed,
1269 82 : remote_storage,
1270 82 : consumer: std::sync::Mutex::new(ConsumerState { rx, executor_rx }),
1271 82 : lsn_table: Arc::new(std::sync::RwLock::new(VisibleLsnUpdates::new())),
1272 82 : }
1273 82 : }
1274 :
1275 : #[allow(clippy::await_holding_lock)]
1276 2 : pub async fn pump(&self) {
1277 2 : if let Some(remote_storage) = &self.remote_storage {
1278 : // Permit holding mutex across await, because this is only ever
1279 : // called once at a time in tests.
1280 2 : let mut locked = self.consumer.lock().unwrap();
1281 2 : let count = locked.consume(remote_storage).await;
1282 2 : self.executed.fetch_add(count, Ordering::Relaxed);
1283 0 : }
1284 2 : }
1285 :
1286 94 : pub(crate) fn new_client(&self) -> DeletionQueueClient {
1287 94 : DeletionQueueClient {
1288 94 : tx: self.tx.clone(),
1289 94 : executor_tx: self.executor_tx.clone(),
1290 94 : lsn_table: self.lsn_table.clone(),
1291 94 : }
1292 94 : }
1293 : }
1294 :
1295 : /// Test round-trip serialization/deserialization, and test stability of the format
1296 : /// vs. a static expected string for the serialized version.
1297 2 : #[test]
1298 2 : fn deletion_list_serialization() -> anyhow::Result<()> {
1299 2 : let tenant_id = "ad6c1a56f5680419d3a16ff55d97ec3c"
1300 2 : .to_string()
1301 2 : .parse::<TenantShardId>()?;
1302 2 : let timeline_id = "be322c834ed9e709e63b5c9698691910"
1303 2 : .to_string()
1304 2 : .parse::<TimelineId>()?;
1305 2 : let generation = Generation::new(123);
1306 :
1307 2 : let object =
1308 2 : RemotePath::from_string(&format!("tenants/{tenant_id}/timelines/{timeline_id}/foo"))?;
1309 2 : let mut objects = [object].to_vec();
1310 2 :
1311 2 : let mut example = DeletionList::new(1);
1312 2 : example.push(&tenant_id, &timeline_id, generation, &mut objects);
1313 :
1314 2 : let encoded = serde_json::to_string(&example)?;
1315 :
1316 2 : let expected = "{\"version\":1,\"sequence\":1,\"tenants\":{\"ad6c1a56f5680419d3a16ff55d97ec3c\":{\"timelines\":{\"be322c834ed9e709e63b5c9698691910\":[\"foo\"]},\"generation\":123}},\"size\":1}".to_string();
1317 2 : assert_eq!(encoded, expected);
1318 :
1319 2 : let decoded = serde_json::from_str::<DeletionList>(&encoded)?;
1320 2 : assert_eq!(example, decoded);
1321 :
1322 2 : Ok(())
1323 2 : }
1324 : }
|