Line data Source code
1 : //! New compaction implementation. The algorithm itself is implemented in the
2 : //! compaction crate. This file implements the callbacks and structs that allow
3 : //! the algorithm to drive the process.
4 : //!
5 : //! The old legacy algorithm is implemented directly in `timeline.rs`.
6 :
7 : use std::collections::{BinaryHeap, HashMap, HashSet, VecDeque};
8 : use std::ops::{Deref, Range};
9 : use std::sync::Arc;
10 : use std::time::Instant;
11 :
12 : use super::layer_manager::LayerManager;
13 : use super::{
14 : CompactFlags, CompactOptions, CompactionError, CreateImageLayersError, DurationRecorder,
15 : GetVectoredError, ImageLayerCreationMode, LastImageLayerCreationStatus, RecordedDuration,
16 : Timeline,
17 : };
18 :
19 : use anyhow::{Context, anyhow};
20 : use bytes::Bytes;
21 : use enumset::EnumSet;
22 : use fail::fail_point;
23 : use futures::FutureExt;
24 : use itertools::Itertools;
25 : use once_cell::sync::Lazy;
26 : use pageserver_api::config::tenant_conf_defaults::DEFAULT_CHECKPOINT_DISTANCE;
27 : use pageserver_api::key::{KEY_SIZE, Key};
28 : use pageserver_api::keyspace::{KeySpace, ShardedRange};
29 : use pageserver_api::models::CompactInfoResponse;
30 : use pageserver_api::record::NeonWalRecord;
31 : use pageserver_api::shard::{ShardCount, ShardIdentity, TenantShardId};
32 : use pageserver_api::value::Value;
33 : use pageserver_compaction::helpers::{fully_contains, overlaps_with};
34 : use pageserver_compaction::interface::*;
35 : use serde::Serialize;
36 : use tokio::sync::{OwnedSemaphorePermit, Semaphore};
37 : use tokio_util::sync::CancellationToken;
38 : use tracing::{Instrument, debug, error, info, info_span, trace, warn};
39 : use utils::critical;
40 : use utils::id::TimelineId;
41 : use utils::lsn::Lsn;
42 :
43 : use crate::context::{AccessStatsBehavior, RequestContext, RequestContextBuilder};
44 : use crate::page_cache;
45 : use crate::statvfs::Statvfs;
46 : use crate::tenant::checks::check_valid_layermap;
47 : use crate::tenant::gc_block::GcBlock;
48 : use crate::tenant::layer_map::LayerMap;
49 : use crate::tenant::remote_timeline_client::WaitCompletionError;
50 : use crate::tenant::remote_timeline_client::index::GcCompactionState;
51 : use crate::tenant::storage_layer::batch_split_writer::{
52 : BatchWriterResult, SplitDeltaLayerWriter, SplitImageLayerWriter,
53 : };
54 : use crate::tenant::storage_layer::filter_iterator::FilterIterator;
55 : use crate::tenant::storage_layer::merge_iterator::MergeIterator;
56 : use crate::tenant::storage_layer::{
57 : AsLayerDesc, PersistentLayerDesc, PersistentLayerKey, ValueReconstructState,
58 : };
59 : use crate::tenant::timeline::{
60 : DeltaLayerWriter, ImageLayerCreationOutcome, ImageLayerWriter, IoConcurrency, Layer,
61 : ResidentLayer, drop_rlock,
62 : };
63 : use crate::tenant::{DeltaLayer, MaybeOffloaded, gc_block};
64 : use crate::virtual_file::{MaybeFatalIo, VirtualFile};
65 :
66 : /// Maximum number of deltas before generating an image layer in bottom-most compaction.
67 : const COMPACTION_DELTA_THRESHOLD: usize = 5;
68 :
69 : #[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)]
70 : pub struct GcCompactionJobId(pub usize);
71 :
72 : impl std::fmt::Display for GcCompactionJobId {
73 0 : fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
74 0 : write!(f, "{}", self.0)
75 0 : }
76 : }
77 :
78 : pub struct GcCompactionCombinedSettings {
79 : pub gc_compaction_enabled: bool,
80 : pub gc_compaction_initial_threshold_kb: u64,
81 : pub gc_compaction_ratio_percent: u64,
82 : }
83 :
84 : #[derive(Debug, Clone)]
85 : pub enum GcCompactionQueueItem {
86 : MetaJob {
87 : /// Compaction options
88 : options: CompactOptions,
89 : /// Whether the compaction is triggered automatically (determines whether we need to update L2 LSN)
90 : auto: bool,
91 : },
92 : SubCompactionJob(CompactOptions),
93 : Notify(GcCompactionJobId, Option<Lsn>),
94 : }
95 :
96 : impl GcCompactionQueueItem {
97 0 : pub fn into_compact_info_resp(
98 0 : self,
99 0 : id: GcCompactionJobId,
100 0 : running: bool,
101 0 : ) -> Option<CompactInfoResponse> {
102 0 : match self {
103 0 : GcCompactionQueueItem::MetaJob { options, .. } => Some(CompactInfoResponse {
104 0 : compact_key_range: options.compact_key_range,
105 0 : compact_lsn_range: options.compact_lsn_range,
106 0 : sub_compaction: options.sub_compaction,
107 0 : running,
108 0 : job_id: id.0,
109 0 : }),
110 0 : GcCompactionQueueItem::SubCompactionJob(options) => Some(CompactInfoResponse {
111 0 : compact_key_range: options.compact_key_range,
112 0 : compact_lsn_range: options.compact_lsn_range,
113 0 : sub_compaction: options.sub_compaction,
114 0 : running,
115 0 : job_id: id.0,
116 0 : }),
117 0 : GcCompactionQueueItem::Notify(_, _) => None,
118 : }
119 0 : }
120 : }
121 :
122 : #[derive(Default)]
123 : struct GcCompactionGuardItems {
124 : notify: Option<tokio::sync::oneshot::Sender<()>>,
125 : gc_guard: Option<gc_block::Guard>,
126 : permit: Option<OwnedSemaphorePermit>,
127 : }
128 :
129 : struct GcCompactionQueueInner {
130 : running: Option<(GcCompactionJobId, GcCompactionQueueItem)>,
131 : queued: VecDeque<(GcCompactionJobId, GcCompactionQueueItem)>,
132 : guards: HashMap<GcCompactionJobId, GcCompactionGuardItems>,
133 : last_id: GcCompactionJobId,
134 : }
135 :
136 : impl GcCompactionQueueInner {
137 0 : fn next_id(&mut self) -> GcCompactionJobId {
138 0 : let id = self.last_id;
139 0 : self.last_id = GcCompactionJobId(id.0 + 1);
140 0 : id
141 0 : }
142 : }
143 :
144 : /// A structure to store gc_compaction jobs.
145 : pub struct GcCompactionQueue {
146 : /// All items in the queue, and the currently-running job.
147 : inner: std::sync::Mutex<GcCompactionQueueInner>,
148 : /// Ensure only one thread is consuming the queue.
149 : consumer_lock: tokio::sync::Mutex<()>,
150 : }
151 :
152 0 : static CONCURRENT_GC_COMPACTION_TASKS: Lazy<Arc<Semaphore>> = Lazy::new(|| {
153 0 : // Only allow two timelines on one pageserver to run gc compaction at a time.
154 0 : Arc::new(Semaphore::new(2))
155 0 : });
156 :
157 : impl GcCompactionQueue {
158 0 : pub fn new() -> Self {
159 0 : GcCompactionQueue {
160 0 : inner: std::sync::Mutex::new(GcCompactionQueueInner {
161 0 : running: None,
162 0 : queued: VecDeque::new(),
163 0 : guards: HashMap::new(),
164 0 : last_id: GcCompactionJobId(0),
165 0 : }),
166 0 : consumer_lock: tokio::sync::Mutex::new(()),
167 0 : }
168 0 : }
169 :
170 0 : pub fn cancel_scheduled(&self) {
171 0 : let mut guard = self.inner.lock().unwrap();
172 0 : guard.queued.clear();
173 0 : // TODO: if there is a running job, we should keep the gc guard. However, currently, the cancel
174 0 : // API is only used for testing purposes, so we can drop everything here.
175 0 : guard.guards.clear();
176 0 : }
177 :
178 : /// Schedule a manual compaction job.
179 0 : pub fn schedule_manual_compaction(
180 0 : &self,
181 0 : options: CompactOptions,
182 0 : notify: Option<tokio::sync::oneshot::Sender<()>>,
183 0 : ) -> GcCompactionJobId {
184 0 : let mut guard = self.inner.lock().unwrap();
185 0 : let id = guard.next_id();
186 0 : guard.queued.push_back((
187 0 : id,
188 0 : GcCompactionQueueItem::MetaJob {
189 0 : options,
190 0 : auto: false,
191 0 : },
192 0 : ));
193 0 : guard.guards.entry(id).or_default().notify = notify;
194 0 : info!("scheduled compaction job id={}", id);
195 0 : id
196 0 : }
197 :
198 : /// Schedule an auto compaction job.
199 0 : fn schedule_auto_compaction(
200 0 : &self,
201 0 : options: CompactOptions,
202 0 : permit: OwnedSemaphorePermit,
203 0 : ) -> GcCompactionJobId {
204 0 : let mut guard = self.inner.lock().unwrap();
205 0 : let id = guard.next_id();
206 0 : guard.queued.push_back((
207 0 : id,
208 0 : GcCompactionQueueItem::MetaJob {
209 0 : options,
210 0 : auto: true,
211 0 : },
212 0 : ));
213 0 : guard.guards.entry(id).or_default().permit = Some(permit);
214 0 : id
215 0 : }
216 :
217 : /// Trigger an auto compaction.
218 0 : pub async fn trigger_auto_compaction(
219 0 : &self,
220 0 : timeline: &Arc<Timeline>,
221 0 : ) -> Result<(), CompactionError> {
222 0 : let GcCompactionCombinedSettings {
223 0 : gc_compaction_enabled,
224 0 : gc_compaction_initial_threshold_kb,
225 0 : gc_compaction_ratio_percent,
226 0 : } = timeline.get_gc_compaction_settings();
227 0 : if !gc_compaction_enabled {
228 0 : return Ok(());
229 0 : }
230 0 : if self.remaining_jobs_num() > 0 {
231 : // Only schedule auto compaction when the queue is empty
232 0 : return Ok(());
233 0 : }
234 0 : if timeline.ancestor_timeline().is_some() {
235 : // Do not trigger auto compaction for child timelines. We haven't tested
236 : // it enough in staging yet.
237 0 : return Ok(());
238 0 : }
239 0 : if timeline.get_gc_compaction_watermark() == Lsn::INVALID {
240 : // If the gc watermark is not set, we don't need to trigger auto compaction.
241 : // This check is the same as in `gc_compaction_split_jobs` but we don't log
242 : // here and we can also skip the computation of the trigger condition earlier.
243 0 : return Ok(());
244 0 : }
245 :
246 0 : let Ok(permit) = CONCURRENT_GC_COMPACTION_TASKS.clone().try_acquire_owned() else {
247 : // Only allow one compaction run at a time. TODO: As we do `try_acquire_owned`, we cannot ensure
248 : // the fairness of the lock across timelines. We should listen for both `acquire` and `l0_compaction_trigger`
249 : // to ensure the fairness while avoid starving other tasks.
250 0 : return Ok(());
251 : };
252 :
253 0 : let gc_compaction_state = timeline.get_gc_compaction_state();
254 0 : let l2_lsn = gc_compaction_state
255 0 : .map(|x| x.last_completed_lsn)
256 0 : .unwrap_or(Lsn::INVALID);
257 :
258 0 : let layers = {
259 0 : let guard = timeline.layers.read().await;
260 0 : let layer_map = guard.layer_map()?;
261 0 : layer_map.iter_historic_layers().collect_vec()
262 0 : };
263 0 : let mut l2_size: u64 = 0;
264 0 : let mut l1_size = 0;
265 0 : let gc_cutoff = *timeline.get_applied_gc_cutoff_lsn();
266 0 : for layer in layers {
267 0 : if layer.lsn_range.start <= l2_lsn {
268 0 : l2_size += layer.file_size();
269 0 : } else if layer.lsn_range.start <= gc_cutoff {
270 0 : l1_size += layer.file_size();
271 0 : }
272 : }
273 :
274 0 : fn trigger_compaction(
275 0 : l1_size: u64,
276 0 : l2_size: u64,
277 0 : gc_compaction_initial_threshold_kb: u64,
278 0 : gc_compaction_ratio_percent: u64,
279 0 : ) -> bool {
280 : const AUTO_TRIGGER_LIMIT: u64 = 150 * 1024 * 1024 * 1024; // 150GB
281 0 : if l1_size >= AUTO_TRIGGER_LIMIT || l2_size >= AUTO_TRIGGER_LIMIT {
282 : // Do not auto-trigger when physical size >= 150GB
283 0 : return false;
284 0 : }
285 0 : // initial trigger
286 0 : if l2_size == 0 && l1_size >= gc_compaction_initial_threshold_kb * 1024 {
287 0 : info!(
288 0 : "trigger auto-compaction because l1_size={} >= gc_compaction_initial_threshold_kb={}",
289 : l1_size, gc_compaction_initial_threshold_kb
290 : );
291 0 : return true;
292 0 : }
293 0 : // size ratio trigger
294 0 : if l2_size == 0 {
295 0 : return false;
296 0 : }
297 0 : if l1_size as f64 / l2_size as f64 >= (gc_compaction_ratio_percent as f64 / 100.0) {
298 0 : info!(
299 0 : "trigger auto-compaction because l1_size={} / l2_size={} > gc_compaction_ratio_percent={}",
300 : l1_size, l2_size, gc_compaction_ratio_percent
301 : );
302 0 : return true;
303 0 : }
304 0 : false
305 0 : }
306 :
307 0 : if trigger_compaction(
308 0 : l1_size,
309 0 : l2_size,
310 0 : gc_compaction_initial_threshold_kb,
311 0 : gc_compaction_ratio_percent,
312 0 : ) {
313 0 : self.schedule_auto_compaction(
314 0 : CompactOptions {
315 0 : flags: {
316 0 : let mut flags = EnumSet::new();
317 0 : flags |= CompactFlags::EnhancedGcBottomMostCompaction;
318 0 : flags
319 0 : },
320 0 : sub_compaction: true,
321 0 : compact_key_range: None,
322 0 : compact_lsn_range: None,
323 0 : sub_compaction_max_job_size_mb: None,
324 0 : },
325 0 : permit,
326 0 : );
327 0 : info!(
328 0 : "scheduled auto gc-compaction: l1_size={}, l2_size={}, l2_lsn={}, gc_cutoff={}",
329 : l1_size, l2_size, l2_lsn, gc_cutoff
330 : );
331 : } else {
332 0 : debug!(
333 0 : "did not trigger auto gc-compaction: l1_size={}, l2_size={}, l2_lsn={}, gc_cutoff={}",
334 : l1_size, l2_size, l2_lsn, gc_cutoff
335 : );
336 : }
337 0 : Ok(())
338 0 : }
339 :
340 : /// Notify the caller the job has finished and unblock GC.
341 0 : fn notify_and_unblock(&self, id: GcCompactionJobId) {
342 0 : info!("compaction job id={} finished", id);
343 0 : let mut guard = self.inner.lock().unwrap();
344 0 : if let Some(items) = guard.guards.remove(&id) {
345 0 : drop(items.gc_guard);
346 0 : if let Some(tx) = items.notify {
347 0 : let _ = tx.send(());
348 0 : }
349 0 : }
350 0 : }
351 :
352 0 : async fn handle_sub_compaction(
353 0 : &self,
354 0 : id: GcCompactionJobId,
355 0 : options: CompactOptions,
356 0 : timeline: &Arc<Timeline>,
357 0 : gc_block: &GcBlock,
358 0 : auto: bool,
359 0 : ) -> Result<(), CompactionError> {
360 0 : info!(
361 0 : "running scheduled enhanced gc bottom-most compaction with sub-compaction, splitting compaction jobs"
362 : );
363 0 : let jobs = timeline
364 0 : .gc_compaction_split_jobs(
365 0 : GcCompactJob::from_compact_options(options.clone()),
366 0 : options.sub_compaction_max_job_size_mb,
367 0 : )
368 0 : .await?;
369 0 : if jobs.is_empty() {
370 0 : info!("no jobs to run, skipping scheduled compaction task");
371 0 : self.notify_and_unblock(id);
372 : } else {
373 0 : let gc_guard = match gc_block.start().await {
374 0 : Ok(guard) => guard,
375 0 : Err(e) => {
376 0 : return Err(CompactionError::Other(anyhow!(
377 0 : "cannot run gc-compaction because gc is blocked: {}",
378 0 : e
379 0 : )));
380 : }
381 : };
382 :
383 0 : let jobs_len = jobs.len();
384 0 : let mut pending_tasks = Vec::new();
385 0 : // gc-compaction might pick more layers or fewer layers to compact. The L2 LSN does not need to be accurate.
386 0 : // And therefore, we simply assume the maximum LSN of all jobs is the expected L2 LSN.
387 0 : let expected_l2_lsn = jobs.iter().map(|job| job.compact_lsn_range.end).max();
388 0 : for job in jobs {
389 : // Unfortunately we need to convert the `GcCompactJob` back to `CompactionOptions`
390 : // until we do further refactors to allow directly call `compact_with_gc`.
391 0 : let mut flags: EnumSet<CompactFlags> = EnumSet::default();
392 0 : flags |= CompactFlags::EnhancedGcBottomMostCompaction;
393 0 : if job.dry_run {
394 0 : flags |= CompactFlags::DryRun;
395 0 : }
396 0 : if options.flags.contains(CompactFlags::NoYield) {
397 0 : flags |= CompactFlags::NoYield;
398 0 : }
399 0 : let options = CompactOptions {
400 0 : flags,
401 0 : sub_compaction: false,
402 0 : compact_key_range: Some(job.compact_key_range.into()),
403 0 : compact_lsn_range: Some(job.compact_lsn_range.into()),
404 0 : sub_compaction_max_job_size_mb: None,
405 0 : };
406 0 : pending_tasks.push(GcCompactionQueueItem::SubCompactionJob(options));
407 : }
408 :
409 0 : if !auto {
410 0 : pending_tasks.push(GcCompactionQueueItem::Notify(id, None));
411 0 : } else {
412 0 : pending_tasks.push(GcCompactionQueueItem::Notify(id, expected_l2_lsn));
413 0 : }
414 :
415 : {
416 0 : let mut guard = self.inner.lock().unwrap();
417 0 : guard.guards.entry(id).or_default().gc_guard = Some(gc_guard);
418 0 : let mut tasks = Vec::new();
419 0 : for task in pending_tasks {
420 0 : let id = guard.next_id();
421 0 : tasks.push((id, task));
422 0 : }
423 0 : tasks.reverse();
424 0 : for item in tasks {
425 0 : guard.queued.push_front(item);
426 0 : }
427 : }
428 0 : info!(
429 0 : "scheduled enhanced gc bottom-most compaction with sub-compaction, split into {} jobs",
430 : jobs_len
431 : );
432 : }
433 0 : Ok(())
434 0 : }
435 :
436 : /// Take a job from the queue and process it. Returns if there are still pending tasks.
437 0 : pub async fn iteration(
438 0 : &self,
439 0 : cancel: &CancellationToken,
440 0 : ctx: &RequestContext,
441 0 : gc_block: &GcBlock,
442 0 : timeline: &Arc<Timeline>,
443 0 : ) -> Result<CompactionOutcome, CompactionError> {
444 0 : let Ok(_one_op_at_a_time_guard) = self.consumer_lock.try_lock() else {
445 0 : return Err(CompactionError::AlreadyRunning(
446 0 : "cannot run gc-compaction because another gc-compaction is running. This should not happen because we only call this function from the gc-compaction queue.",
447 0 : ));
448 : };
449 : let has_pending_tasks;
450 0 : let mut yield_for_l0 = false;
451 0 : let Some((id, item)) = ({
452 0 : let mut guard = self.inner.lock().unwrap();
453 0 : if let Some((id, item)) = guard.queued.pop_front() {
454 0 : guard.running = Some((id, item.clone()));
455 0 : has_pending_tasks = !guard.queued.is_empty();
456 0 : Some((id, item))
457 : } else {
458 0 : has_pending_tasks = false;
459 0 : None
460 : }
461 : }) else {
462 0 : self.trigger_auto_compaction(timeline).await?;
463 : // Always yield after triggering auto-compaction. Gc-compaction is a low-priority task and we
464 : // have not implemented preemption mechanism yet. We always want to yield it to more important
465 : // tasks if there is one.
466 0 : return Ok(CompactionOutcome::Done);
467 : };
468 0 : match item {
469 0 : GcCompactionQueueItem::MetaJob { options, auto } => {
470 0 : if !options
471 0 : .flags
472 0 : .contains(CompactFlags::EnhancedGcBottomMostCompaction)
473 : {
474 0 : warn!(
475 0 : "ignoring scheduled compaction task: scheduled task must be gc compaction: {:?}",
476 : options
477 : );
478 0 : } else if options.sub_compaction {
479 0 : info!(
480 0 : "running scheduled enhanced gc bottom-most compaction with sub-compaction, splitting compaction jobs"
481 : );
482 0 : self.handle_sub_compaction(id, options, timeline, gc_block, auto)
483 0 : .await?;
484 : } else {
485 : // Auto compaction always enables sub-compaction so we don't need to handle update_l2_lsn
486 : // in this branch.
487 0 : let gc_guard = match gc_block.start().await {
488 0 : Ok(guard) => guard,
489 0 : Err(e) => {
490 0 : return Err(CompactionError::Other(anyhow!(
491 0 : "cannot run gc-compaction because gc is blocked: {}",
492 0 : e
493 0 : )));
494 : }
495 : };
496 0 : {
497 0 : let mut guard = self.inner.lock().unwrap();
498 0 : guard.guards.entry(id).or_default().gc_guard = Some(gc_guard);
499 0 : }
500 0 : let compaction_result =
501 0 : timeline.compact_with_options(cancel, options, ctx).await?;
502 0 : self.notify_and_unblock(id);
503 0 : if compaction_result == CompactionOutcome::YieldForL0 {
504 0 : yield_for_l0 = true;
505 0 : }
506 : }
507 : }
508 0 : GcCompactionQueueItem::SubCompactionJob(options) => {
509 : // TODO: error handling, clear the queue if any task fails?
510 0 : let compaction_result = timeline.compact_with_options(cancel, options, ctx).await?;
511 0 : if compaction_result == CompactionOutcome::YieldForL0 {
512 0 : // We will permenantly give up a task if we yield for L0 compaction: the preempted subcompaction job won't be running
513 0 : // again. This ensures that we don't keep doing duplicated work within gc-compaction. Not directly returning here because
514 0 : // we need to clean things up before returning from the function.
515 0 : yield_for_l0 = true;
516 0 : }
517 : }
518 0 : GcCompactionQueueItem::Notify(id, l2_lsn) => {
519 0 : self.notify_and_unblock(id);
520 0 : if let Some(l2_lsn) = l2_lsn {
521 0 : let current_l2_lsn = timeline
522 0 : .get_gc_compaction_state()
523 0 : .map(|x| x.last_completed_lsn)
524 0 : .unwrap_or(Lsn::INVALID);
525 0 : if l2_lsn >= current_l2_lsn {
526 0 : info!("l2_lsn updated to {}", l2_lsn);
527 0 : timeline
528 0 : .update_gc_compaction_state(GcCompactionState {
529 0 : last_completed_lsn: l2_lsn,
530 0 : })
531 0 : .map_err(CompactionError::Other)?;
532 : } else {
533 0 : warn!(
534 0 : "l2_lsn updated to {} but it is less than the current l2_lsn {}",
535 : l2_lsn, current_l2_lsn
536 : );
537 : }
538 0 : }
539 : }
540 : }
541 0 : {
542 0 : let mut guard = self.inner.lock().unwrap();
543 0 : guard.running = None;
544 0 : }
545 0 : Ok(if yield_for_l0 {
546 0 : tracing::info!("give up gc-compaction: yield for L0 compaction");
547 0 : CompactionOutcome::YieldForL0
548 0 : } else if has_pending_tasks {
549 0 : CompactionOutcome::Pending
550 : } else {
551 0 : CompactionOutcome::Done
552 : })
553 0 : }
554 :
555 : #[allow(clippy::type_complexity)]
556 0 : pub fn remaining_jobs(
557 0 : &self,
558 0 : ) -> (
559 0 : Option<(GcCompactionJobId, GcCompactionQueueItem)>,
560 0 : VecDeque<(GcCompactionJobId, GcCompactionQueueItem)>,
561 0 : ) {
562 0 : let guard = self.inner.lock().unwrap();
563 0 : (guard.running.clone(), guard.queued.clone())
564 0 : }
565 :
566 0 : pub fn remaining_jobs_num(&self) -> usize {
567 0 : let guard = self.inner.lock().unwrap();
568 0 : guard.queued.len() + if guard.running.is_some() { 1 } else { 0 }
569 0 : }
570 : }
571 :
572 : /// A job description for the gc-compaction job. This structure describes the rectangle range that the job will
573 : /// process. The exact layers that need to be compacted/rewritten will be generated when `compact_with_gc` gets
574 : /// called.
575 : #[derive(Debug, Clone)]
576 : pub(crate) struct GcCompactJob {
577 : pub dry_run: bool,
578 : /// The key range to be compacted. The compaction algorithm will only regenerate key-value pairs within this range
579 : /// [left inclusive, right exclusive), and other pairs will be rewritten into new files if necessary.
580 : pub compact_key_range: Range<Key>,
581 : /// The LSN range to be compacted. The compaction algorithm will use this range to determine the layers to be
582 : /// selected for the compaction, and it does not guarantee the generated layers will have exactly the same LSN range
583 : /// as specified here. The true range being compacted is `min_lsn/max_lsn` in [`GcCompactionJobDescription`].
584 : /// min_lsn will always <= the lower bound specified here, and max_lsn will always >= the upper bound specified here.
585 : pub compact_lsn_range: Range<Lsn>,
586 : }
587 :
588 : impl GcCompactJob {
589 108 : pub fn from_compact_options(options: CompactOptions) -> Self {
590 108 : GcCompactJob {
591 108 : dry_run: options.flags.contains(CompactFlags::DryRun),
592 108 : compact_key_range: options
593 108 : .compact_key_range
594 108 : .map(|x| x.into())
595 108 : .unwrap_or(Key::MIN..Key::MAX),
596 108 : compact_lsn_range: options
597 108 : .compact_lsn_range
598 108 : .map(|x| x.into())
599 108 : .unwrap_or(Lsn::INVALID..Lsn::MAX),
600 108 : }
601 108 : }
602 : }
603 :
604 : /// A job description for the gc-compaction job. This structure is generated when `compact_with_gc` is called
605 : /// and contains the exact layers we want to compact.
606 : pub struct GcCompactionJobDescription {
607 : /// All layers to read in the compaction job
608 : selected_layers: Vec<Layer>,
609 : /// GC cutoff of the job. This is the lowest LSN that will be accessed by the read/GC path and we need to
610 : /// keep all deltas <= this LSN or generate an image == this LSN.
611 : gc_cutoff: Lsn,
612 : /// LSNs to retain for the job. Read path will use this LSN so we need to keep deltas <= this LSN or
613 : /// generate an image == this LSN.
614 : retain_lsns_below_horizon: Vec<Lsn>,
615 : /// Maximum layer LSN processed in this compaction, that is max(end_lsn of layers). Exclusive. All data
616 : /// \>= this LSN will be kept and will not be rewritten.
617 : max_layer_lsn: Lsn,
618 : /// Minimum layer LSN processed in this compaction, that is min(start_lsn of layers). Inclusive.
619 : /// All access below (strict lower than `<`) this LSN will be routed through the normal read path instead of
620 : /// k-merge within gc-compaction.
621 : min_layer_lsn: Lsn,
622 : /// Only compact layers overlapping with this range.
623 : compaction_key_range: Range<Key>,
624 : /// When partial compaction is enabled, these layers need to be rewritten to ensure no overlap.
625 : /// This field is here solely for debugging. The field will not be read once the compaction
626 : /// description is generated.
627 : rewrite_layers: Vec<Arc<PersistentLayerDesc>>,
628 : }
629 :
630 : /// The result of bottom-most compaction for a single key at each LSN.
631 : #[derive(Debug)]
632 : #[cfg_attr(test, derive(PartialEq))]
633 : pub struct KeyLogAtLsn(pub Vec<(Lsn, Value)>);
634 :
635 : /// The result of bottom-most compaction.
636 : #[derive(Debug)]
637 : #[cfg_attr(test, derive(PartialEq))]
638 : pub(crate) struct KeyHistoryRetention {
639 : /// Stores logs to reconstruct the value at the given LSN, that is to say, logs <= LSN or image == LSN.
640 : pub(crate) below_horizon: Vec<(Lsn, KeyLogAtLsn)>,
641 : /// Stores logs to reconstruct the value at any LSN above the horizon, that is to say, log > LSN.
642 : pub(crate) above_horizon: KeyLogAtLsn,
643 : }
644 :
645 : impl KeyHistoryRetention {
646 : /// Hack: skip delta layer if we need to produce a layer of a same key-lsn.
647 : ///
648 : /// This can happen if we have removed some deltas in "the middle" of some existing layer's key-lsn-range.
649 : /// For example, consider the case where a single delta with range [0x10,0x50) exists.
650 : /// And we have branches at LSN 0x10, 0x20, 0x30.
651 : /// Then we delete branch @ 0x20.
652 : /// Bottom-most compaction may now delete the delta [0x20,0x30).
653 : /// And that wouldnt' change the shape of the layer.
654 : ///
655 : /// Note that bottom-most-gc-compaction never _adds_ new data in that case, only removes.
656 : ///
657 : /// `discard_key` will only be called when the writer reaches its target (instead of for every key), so it's fine to grab a lock inside.
658 148 : async fn discard_key(key: &PersistentLayerKey, tline: &Arc<Timeline>, dry_run: bool) -> bool {
659 148 : if dry_run {
660 0 : return true;
661 148 : }
662 148 : if LayerMap::is_l0(&key.key_range, key.is_delta) {
663 : // gc-compaction should not produce L0 deltas, otherwise it will break the layer order.
664 : // We should ignore such layers.
665 0 : return true;
666 148 : }
667 : let layer_generation;
668 : {
669 148 : let guard = tline.layers.read().await;
670 148 : if !guard.contains_key(key) {
671 104 : return false;
672 44 : }
673 44 : layer_generation = guard.get_from_key(key).metadata().generation;
674 44 : }
675 44 : if layer_generation == tline.generation {
676 44 : info!(
677 : key=%key,
678 : ?layer_generation,
679 0 : "discard layer due to duplicated layer key in the same generation",
680 : );
681 44 : true
682 : } else {
683 0 : false
684 : }
685 148 : }
686 :
687 : /// Pipe a history of a single key to the writers.
688 : ///
689 : /// If `image_writer` is none, the images will be placed into the delta layers.
690 : /// The delta writer will contain all images and deltas (below and above the horizon) except the bottom-most images.
691 : #[allow(clippy::too_many_arguments)]
692 1244 : async fn pipe_to(
693 1244 : self,
694 1244 : key: Key,
695 1244 : delta_writer: &mut SplitDeltaLayerWriter,
696 1244 : mut image_writer: Option<&mut SplitImageLayerWriter>,
697 1244 : stat: &mut CompactionStatistics,
698 1244 : ctx: &RequestContext,
699 1244 : ) -> anyhow::Result<()> {
700 1244 : let mut first_batch = true;
701 4024 : for (cutoff_lsn, KeyLogAtLsn(logs)) in self.below_horizon {
702 2780 : if first_batch {
703 1244 : if logs.len() == 1 && logs[0].1.is_image() {
704 1168 : let Value::Image(img) = &logs[0].1 else {
705 0 : unreachable!()
706 : };
707 1168 : stat.produce_image_key(img);
708 1168 : if let Some(image_writer) = image_writer.as_mut() {
709 1168 : image_writer.put_image(key, img.clone(), ctx).await?;
710 : } else {
711 0 : delta_writer
712 0 : .put_value(key, cutoff_lsn, Value::Image(img.clone()), ctx)
713 0 : .await?;
714 : }
715 : } else {
716 132 : for (lsn, val) in logs {
717 56 : stat.produce_key(&val);
718 56 : delta_writer.put_value(key, lsn, val, ctx).await?;
719 : }
720 : }
721 1244 : first_batch = false;
722 : } else {
723 1768 : for (lsn, val) in logs {
724 232 : stat.produce_key(&val);
725 232 : delta_writer.put_value(key, lsn, val, ctx).await?;
726 : }
727 : }
728 : }
729 1244 : let KeyLogAtLsn(above_horizon_logs) = self.above_horizon;
730 1360 : for (lsn, val) in above_horizon_logs {
731 116 : stat.produce_key(&val);
732 116 : delta_writer.put_value(key, lsn, val, ctx).await?;
733 : }
734 1244 : Ok(())
735 1244 : }
736 : }
737 :
738 : #[derive(Debug, Serialize, Default)]
739 : struct CompactionStatisticsNumSize {
740 : num: u64,
741 : size: u64,
742 : }
743 :
744 : #[derive(Debug, Serialize, Default)]
745 : pub struct CompactionStatistics {
746 : /// Delta layer visited (maybe compressed, physical size)
747 : delta_layer_visited: CompactionStatisticsNumSize,
748 : /// Image layer visited (maybe compressed, physical size)
749 : image_layer_visited: CompactionStatisticsNumSize,
750 : /// Delta layer produced (maybe compressed, physical size)
751 : delta_layer_produced: CompactionStatisticsNumSize,
752 : /// Image layer produced (maybe compressed, physical size)
753 : image_layer_produced: CompactionStatisticsNumSize,
754 : /// Delta layer discarded (maybe compressed, physical size of the layer being discarded instead of the original layer)
755 : delta_layer_discarded: CompactionStatisticsNumSize,
756 : /// Image layer discarded (maybe compressed, physical size of the layer being discarded instead of the original layer)
757 : image_layer_discarded: CompactionStatisticsNumSize,
758 : num_unique_keys_visited: usize,
759 : /// Delta visited (uncompressed, original size)
760 : wal_keys_visited: CompactionStatisticsNumSize,
761 : /// Image visited (uncompressed, original size)
762 : image_keys_visited: CompactionStatisticsNumSize,
763 : /// Delta produced (uncompressed, original size)
764 : wal_produced: CompactionStatisticsNumSize,
765 : /// Image produced (uncompressed, original size)
766 : image_produced: CompactionStatisticsNumSize,
767 :
768 : // Time spent in each phase
769 : time_acquire_lock_secs: f64,
770 : time_analyze_secs: f64,
771 : time_download_layer_secs: f64,
772 : time_main_loop_secs: f64,
773 : time_final_phase_secs: f64,
774 : time_total_secs: f64,
775 :
776 : // Summary
777 : /// Ratio of the key-value size before/after gc-compaction.
778 : uncompressed_size_ratio: f64,
779 : /// Ratio of the physical size before/after gc-compaction.
780 : physical_size_ratio: f64,
781 : }
782 :
783 : impl CompactionStatistics {
784 2084 : fn estimated_size_of_value(val: &Value) -> usize {
785 864 : match val {
786 1220 : Value::Image(img) => img.len(),
787 0 : Value::WalRecord(NeonWalRecord::Postgres { rec, .. }) => rec.len(),
788 864 : _ => std::mem::size_of::<NeonWalRecord>(),
789 : }
790 2084 : }
791 3272 : fn estimated_size_of_key() -> usize {
792 3272 : KEY_SIZE // TODO: distinguish image layer and delta layer (count LSN in delta layer)
793 3272 : }
794 172 : fn visit_delta_layer(&mut self, size: u64) {
795 172 : self.delta_layer_visited.num += 1;
796 172 : self.delta_layer_visited.size += size;
797 172 : }
798 132 : fn visit_image_layer(&mut self, size: u64) {
799 132 : self.image_layer_visited.num += 1;
800 132 : self.image_layer_visited.size += size;
801 132 : }
802 1244 : fn on_unique_key_visited(&mut self) {
803 1244 : self.num_unique_keys_visited += 1;
804 1244 : }
805 480 : fn visit_wal_key(&mut self, val: &Value) {
806 480 : self.wal_keys_visited.num += 1;
807 480 : self.wal_keys_visited.size +=
808 480 : Self::estimated_size_of_value(val) as u64 + Self::estimated_size_of_key() as u64;
809 480 : }
810 1220 : fn visit_image_key(&mut self, val: &Value) {
811 1220 : self.image_keys_visited.num += 1;
812 1220 : self.image_keys_visited.size +=
813 1220 : Self::estimated_size_of_value(val) as u64 + Self::estimated_size_of_key() as u64;
814 1220 : }
815 404 : fn produce_key(&mut self, val: &Value) {
816 404 : match val {
817 20 : Value::Image(img) => self.produce_image_key(img),
818 384 : Value::WalRecord(_) => self.produce_wal_key(val),
819 : }
820 404 : }
821 384 : fn produce_wal_key(&mut self, val: &Value) {
822 384 : self.wal_produced.num += 1;
823 384 : self.wal_produced.size +=
824 384 : Self::estimated_size_of_value(val) as u64 + Self::estimated_size_of_key() as u64;
825 384 : }
826 1188 : fn produce_image_key(&mut self, val: &Bytes) {
827 1188 : self.image_produced.num += 1;
828 1188 : self.image_produced.size += val.len() as u64 + Self::estimated_size_of_key() as u64;
829 1188 : }
830 28 : fn discard_delta_layer(&mut self, original_size: u64) {
831 28 : self.delta_layer_discarded.num += 1;
832 28 : self.delta_layer_discarded.size += original_size;
833 28 : }
834 16 : fn discard_image_layer(&mut self, original_size: u64) {
835 16 : self.image_layer_discarded.num += 1;
836 16 : self.image_layer_discarded.size += original_size;
837 16 : }
838 48 : fn produce_delta_layer(&mut self, size: u64) {
839 48 : self.delta_layer_produced.num += 1;
840 48 : self.delta_layer_produced.size += size;
841 48 : }
842 60 : fn produce_image_layer(&mut self, size: u64) {
843 60 : self.image_layer_produced.num += 1;
844 60 : self.image_layer_produced.size += size;
845 60 : }
846 104 : fn finalize(&mut self) {
847 104 : let original_key_value_size = self.image_keys_visited.size + self.wal_keys_visited.size;
848 104 : let produced_key_value_size = self.image_produced.size + self.wal_produced.size;
849 104 : self.uncompressed_size_ratio =
850 104 : original_key_value_size as f64 / (produced_key_value_size as f64 + 1.0); // avoid div by 0
851 104 : let original_physical_size = self.image_layer_visited.size + self.delta_layer_visited.size;
852 104 : let produced_physical_size = self.image_layer_produced.size
853 104 : + self.delta_layer_produced.size
854 104 : + self.image_layer_discarded.size
855 104 : + self.delta_layer_discarded.size; // Also include the discarded layers to make the ratio accurate
856 104 : self.physical_size_ratio =
857 104 : original_physical_size as f64 / (produced_physical_size as f64 + 1.0); // avoid div by 0
858 104 : }
859 : }
860 :
861 : #[derive(Default, Debug, Clone, Copy, PartialEq, Eq)]
862 : pub enum CompactionOutcome {
863 : #[default]
864 : /// No layers need to be compacted after this round. Compaction doesn't need
865 : /// to be immediately scheduled.
866 : Done,
867 : /// Still has pending layers to be compacted after this round. Ideally, the scheduler
868 : /// should immediately schedule another compaction.
869 : Pending,
870 : /// A timeline needs L0 compaction. Yield and schedule an immediate L0 compaction pass (only
871 : /// guaranteed when `compaction_l0_first` is enabled).
872 : YieldForL0,
873 : /// Compaction was skipped, because the timeline is ineligible for compaction.
874 : Skipped,
875 : }
876 :
877 : impl Timeline {
878 : /// TODO: cancellation
879 : ///
880 : /// Returns whether the compaction has pending tasks.
881 728 : pub(crate) async fn compact_legacy(
882 728 : self: &Arc<Self>,
883 728 : cancel: &CancellationToken,
884 728 : options: CompactOptions,
885 728 : ctx: &RequestContext,
886 728 : ) -> Result<CompactionOutcome, CompactionError> {
887 728 : if options
888 728 : .flags
889 728 : .contains(CompactFlags::EnhancedGcBottomMostCompaction)
890 : {
891 0 : self.compact_with_gc(cancel, options, ctx).await?;
892 0 : return Ok(CompactionOutcome::Done);
893 728 : }
894 728 :
895 728 : if options.flags.contains(CompactFlags::DryRun) {
896 0 : return Err(CompactionError::Other(anyhow!(
897 0 : "dry-run mode is not supported for legacy compaction for now"
898 0 : )));
899 728 : }
900 728 :
901 728 : if options.compact_key_range.is_some() || options.compact_lsn_range.is_some() {
902 : // maybe useful in the future? could implement this at some point
903 0 : return Err(CompactionError::Other(anyhow!(
904 0 : "compaction range is not supported for legacy compaction for now"
905 0 : )));
906 728 : }
907 728 :
908 728 : // High level strategy for compaction / image creation:
909 728 : //
910 728 : // 1. First, do a L0 compaction to ensure we move the L0
911 728 : // layers into the historic layer map get flat levels of
912 728 : // layers. If we did not compact all L0 layers, we will
913 728 : // prioritize compacting the timeline again and not do
914 728 : // any of the compactions below.
915 728 : //
916 728 : // 2. Then, calculate the desired "partitioning" of the
917 728 : // currently in-use key space. The goal is to partition the
918 728 : // key space into roughly fixed-size chunks, but also take into
919 728 : // account any existing image layers, and try to align the
920 728 : // chunk boundaries with the existing image layers to avoid
921 728 : // too much churn. Also try to align chunk boundaries with
922 728 : // relation boundaries. In principle, we don't know about
923 728 : // relation boundaries here, we just deal with key-value
924 728 : // pairs, and the code in pgdatadir_mapping.rs knows how to
925 728 : // map relations into key-value pairs. But in practice we know
926 728 : // that 'field6' is the block number, and the fields 1-5
927 728 : // identify a relation. This is just an optimization,
928 728 : // though.
929 728 : //
930 728 : // 3. Once we know the partitioning, for each partition,
931 728 : // decide if it's time to create a new image layer. The
932 728 : // criteria is: there has been too much "churn" since the last
933 728 : // image layer? The "churn" is fuzzy concept, it's a
934 728 : // combination of too many delta files, or too much WAL in
935 728 : // total in the delta file. Or perhaps: if creating an image
936 728 : // file would allow to delete some older files.
937 728 : //
938 728 : // 4. In the end, if the tenant gets auto-sharded, we will run
939 728 : // a shard-ancestor compaction.
940 728 :
941 728 : // Is the timeline being deleted?
942 728 : if self.is_stopping() {
943 0 : trace!("Dropping out of compaction on timeline shutdown");
944 0 : return Err(CompactionError::ShuttingDown);
945 728 : }
946 728 :
947 728 : let target_file_size = self.get_checkpoint_distance();
948 :
949 : // Define partitioning schema if needed
950 :
951 : // 1. L0 Compact
952 728 : let l0_outcome = {
953 728 : let timer = self.metrics.compact_time_histo.start_timer();
954 728 : let l0_outcome = self
955 728 : .compact_level0(
956 728 : target_file_size,
957 728 : options.flags.contains(CompactFlags::ForceL0Compaction),
958 728 : ctx,
959 728 : )
960 728 : .await?;
961 728 : timer.stop_and_record();
962 728 : l0_outcome
963 728 : };
964 728 :
965 728 : if options.flags.contains(CompactFlags::OnlyL0Compaction) {
966 0 : return Ok(l0_outcome);
967 728 : }
968 728 :
969 728 : // Yield if we have pending L0 compaction. The scheduler will do another pass.
970 728 : if (l0_outcome == CompactionOutcome::Pending || l0_outcome == CompactionOutcome::YieldForL0)
971 0 : && !options.flags.contains(CompactFlags::NoYield)
972 : {
973 0 : info!("image/ancestor compaction yielding for L0 compaction");
974 0 : return Ok(CompactionOutcome::YieldForL0);
975 728 : }
976 728 :
977 728 : // 2. Repartition and create image layers if necessary
978 728 : match self
979 728 : .repartition(
980 728 : self.get_last_record_lsn(),
981 728 : self.get_compaction_target_size(),
982 728 : options.flags,
983 728 : ctx,
984 728 : )
985 728 : .await
986 : {
987 728 : Ok(((dense_partitioning, sparse_partitioning), lsn)) => {
988 728 : // Disables access_stats updates, so that the files we read remain candidates for eviction after we're done with them
989 728 : let image_ctx = RequestContextBuilder::extend(ctx)
990 728 : .access_stats_behavior(AccessStatsBehavior::Skip)
991 728 : .build();
992 728 :
993 728 : let mut partitioning = dense_partitioning;
994 728 : partitioning
995 728 : .parts
996 728 : .extend(sparse_partitioning.into_dense().parts);
997 :
998 : // 3. Create new image layers for partitions that have been modified "enough".
999 728 : let (image_layers, outcome) = self
1000 728 : .create_image_layers(
1001 728 : &partitioning,
1002 728 : lsn,
1003 728 : if options
1004 728 : .flags
1005 728 : .contains(CompactFlags::ForceImageLayerCreation)
1006 : {
1007 28 : ImageLayerCreationMode::Force
1008 : } else {
1009 700 : ImageLayerCreationMode::Try
1010 : },
1011 728 : &image_ctx,
1012 728 : self.last_image_layer_creation_status
1013 728 : .load()
1014 728 : .as_ref()
1015 728 : .clone(),
1016 728 : !options.flags.contains(CompactFlags::NoYield),
1017 728 : )
1018 728 : .await
1019 728 : .inspect_err(|err| {
1020 : if let CreateImageLayersError::GetVectoredError(
1021 : GetVectoredError::MissingKey(_),
1022 0 : ) = err
1023 : {
1024 0 : critical!("missing key during compaction: {err:?}");
1025 0 : }
1026 728 : })?;
1027 :
1028 728 : self.last_image_layer_creation_status
1029 728 : .store(Arc::new(outcome.clone()));
1030 728 :
1031 728 : self.upload_new_image_layers(image_layers)?;
1032 728 : if let LastImageLayerCreationStatus::Incomplete { .. } = outcome {
1033 : // Yield and do not do any other kind of compaction.
1034 0 : info!(
1035 0 : "skipping shard ancestor compaction due to pending image layer generation tasks (preempted by L0 compaction)."
1036 : );
1037 0 : return Ok(CompactionOutcome::YieldForL0);
1038 728 : }
1039 : }
1040 :
1041 : // Suppress errors when cancelled.
1042 0 : Err(_) if self.cancel.is_cancelled() => {}
1043 0 : Err(err) if err.is_cancel() => {}
1044 :
1045 : // Alert on critical errors that indicate data corruption.
1046 0 : Err(err) if err.is_critical() => {
1047 0 : critical!("could not compact, repartitioning keyspace failed: {err:?}");
1048 : }
1049 :
1050 : // Log other errors. No partitioning? This is normal, if the timeline was just created
1051 : // as an empty timeline. Also in unit tests, when we use the timeline as a simple
1052 : // key-value store, ignoring the datadir layout. Log the error but continue.
1053 0 : Err(err) => error!("could not compact, repartitioning keyspace failed: {err:?}"),
1054 : };
1055 :
1056 728 : let partition_count = self.partitioning.read().0.0.parts.len();
1057 728 :
1058 728 : // 4. Shard ancestor compaction
1059 728 :
1060 728 : if self.shard_identity.count >= ShardCount::new(2) {
1061 : // Limit the number of layer rewrites to the number of partitions: this means its
1062 : // runtime should be comparable to a full round of image layer creations, rather than
1063 : // being potentially much longer.
1064 0 : let rewrite_max = partition_count;
1065 0 :
1066 0 : self.compact_shard_ancestors(rewrite_max, ctx).await?;
1067 728 : }
1068 :
1069 728 : Ok(CompactionOutcome::Done)
1070 728 : }
1071 :
1072 : /// Check for layers that are elegible to be rewritten:
1073 : /// - Shard splitting: After a shard split, ancestor layers beyond pitr_interval, so that
1074 : /// we don't indefinitely retain keys in this shard that aren't needed.
1075 : /// - For future use: layers beyond pitr_interval that are in formats we would
1076 : /// rather not maintain compatibility with indefinitely.
1077 : ///
1078 : /// Note: this phase may read and write many gigabytes of data: use rewrite_max to bound
1079 : /// how much work it will try to do in each compaction pass.
1080 0 : async fn compact_shard_ancestors(
1081 0 : self: &Arc<Self>,
1082 0 : rewrite_max: usize,
1083 0 : ctx: &RequestContext,
1084 0 : ) -> Result<(), CompactionError> {
1085 0 : let mut drop_layers = Vec::new();
1086 0 : let mut layers_to_rewrite: Vec<Layer> = Vec::new();
1087 0 :
1088 0 : // We will use the Lsn cutoff of the last GC as a threshold for rewriting layers: if a
1089 0 : // layer is behind this Lsn, it indicates that the layer is being retained beyond the
1090 0 : // pitr_interval, for example because a branchpoint references it.
1091 0 : //
1092 0 : // Holding this read guard also blocks [`Self::gc_timeline`] from entering while we
1093 0 : // are rewriting layers.
1094 0 : let latest_gc_cutoff = self.get_applied_gc_cutoff_lsn();
1095 0 :
1096 0 : tracing::info!(
1097 0 : "starting shard ancestor compaction, latest_gc_cutoff: {}, pitr cutoff {}",
1098 0 : *latest_gc_cutoff,
1099 0 : self.gc_info.read().unwrap().cutoffs.time
1100 : );
1101 :
1102 0 : let layers = self.layers.read().await;
1103 0 : for layer_desc in layers.layer_map()?.iter_historic_layers() {
1104 0 : let layer = layers.get_from_desc(&layer_desc);
1105 0 : if layer.metadata().shard.shard_count == self.shard_identity.count {
1106 : // This layer does not belong to a historic ancestor, no need to re-image it.
1107 0 : continue;
1108 0 : }
1109 0 :
1110 0 : // This layer was created on an ancestor shard: check if it contains any data for this shard.
1111 0 : let sharded_range = ShardedRange::new(layer_desc.get_key_range(), &self.shard_identity);
1112 0 : let layer_local_page_count = sharded_range.page_count();
1113 0 : let layer_raw_page_count = ShardedRange::raw_size(&layer_desc.get_key_range());
1114 0 : if layer_local_page_count == 0 {
1115 : // This ancestral layer only covers keys that belong to other shards.
1116 : // We include the full metadata in the log: if we had some critical bug that caused
1117 : // us to incorrectly drop layers, this would simplify manually debugging + reinstating those layers.
1118 0 : info!(%layer, old_metadata=?layer.metadata(),
1119 0 : "dropping layer after shard split, contains no keys for this shard.",
1120 : );
1121 :
1122 0 : if cfg!(debug_assertions) {
1123 : // Expensive, exhaustive check of keys in this layer: this guards against ShardedRange's calculations being
1124 : // wrong. If ShardedRange claims the local page count is zero, then no keys in this layer
1125 : // should be !is_key_disposable()
1126 : // TODO: exclude sparse keyspace from this check, otherwise it will infinitely loop.
1127 0 : let range = layer_desc.get_key_range();
1128 0 : let mut key = range.start;
1129 0 : while key < range.end {
1130 0 : debug_assert!(self.shard_identity.is_key_disposable(&key));
1131 0 : key = key.next();
1132 : }
1133 0 : }
1134 :
1135 0 : drop_layers.push(layer);
1136 0 : continue;
1137 0 : } else if layer_local_page_count != u32::MAX
1138 0 : && layer_local_page_count == layer_raw_page_count
1139 : {
1140 0 : debug!(%layer,
1141 0 : "layer is entirely shard local ({} keys), no need to filter it",
1142 : layer_local_page_count
1143 : );
1144 0 : continue;
1145 0 : }
1146 0 :
1147 0 : // Don't bother re-writing a layer unless it will at least halve its size
1148 0 : if layer_local_page_count != u32::MAX
1149 0 : && layer_local_page_count > layer_raw_page_count / 2
1150 : {
1151 0 : debug!(%layer,
1152 0 : "layer is already mostly local ({}/{}), not rewriting",
1153 : layer_local_page_count,
1154 : layer_raw_page_count
1155 : );
1156 0 : }
1157 :
1158 : // Don't bother re-writing a layer if it is within the PITR window: it will age-out eventually
1159 : // without incurring the I/O cost of a rewrite.
1160 0 : if layer_desc.get_lsn_range().end >= *latest_gc_cutoff {
1161 0 : debug!(%layer, "Skipping rewrite of layer still in GC window ({} >= {})",
1162 0 : layer_desc.get_lsn_range().end, *latest_gc_cutoff);
1163 0 : continue;
1164 0 : }
1165 0 :
1166 0 : if layer_desc.is_delta() {
1167 : // We do not yet implement rewrite of delta layers
1168 0 : debug!(%layer, "Skipping rewrite of delta layer");
1169 0 : continue;
1170 0 : }
1171 0 :
1172 0 : // Only rewrite layers if their generations differ. This guarantees:
1173 0 : // - that local rewrite is safe, as local layer paths will differ between existing layer and rewritten one
1174 0 : // - that the layer is persistent in remote storage, as we only see old-generation'd layer via loading from remote storage
1175 0 : if layer.metadata().generation == self.generation {
1176 0 : debug!(%layer, "Skipping rewrite, is not from old generation");
1177 0 : continue;
1178 0 : }
1179 0 :
1180 0 : if layers_to_rewrite.len() >= rewrite_max {
1181 0 : tracing::info!(%layer, "Will rewrite layer on a future compaction, already rewrote {}",
1182 0 : layers_to_rewrite.len()
1183 : );
1184 0 : continue;
1185 0 : }
1186 0 :
1187 0 : // Fall through: all our conditions for doing a rewrite passed.
1188 0 : layers_to_rewrite.push(layer);
1189 : }
1190 :
1191 : // Drop read lock on layer map before we start doing time-consuming I/O
1192 0 : drop(layers);
1193 0 :
1194 0 : let mut replace_image_layers = Vec::new();
1195 :
1196 0 : for layer in layers_to_rewrite {
1197 0 : tracing::info!(layer=%layer, "Rewriting layer after shard split...");
1198 0 : let mut image_layer_writer = ImageLayerWriter::new(
1199 0 : self.conf,
1200 0 : self.timeline_id,
1201 0 : self.tenant_shard_id,
1202 0 : &layer.layer_desc().key_range,
1203 0 : layer.layer_desc().image_layer_lsn(),
1204 0 : ctx,
1205 0 : )
1206 0 : .await
1207 0 : .map_err(CompactionError::Other)?;
1208 :
1209 : // Safety of layer rewrites:
1210 : // - We are writing to a different local file path than we are reading from, so the old Layer
1211 : // cannot interfere with the new one.
1212 : // - In the page cache, contents for a particular VirtualFile are stored with a file_id that
1213 : // is different for two layers with the same name (in `ImageLayerInner::new` we always
1214 : // acquire a fresh id from [`crate::page_cache::next_file_id`]. So readers do not risk
1215 : // reading the index from one layer file, and then data blocks from the rewritten layer file.
1216 : // - Any readers that have a reference to the old layer will keep it alive until they are done
1217 : // with it. If they are trying to promote from remote storage, that will fail, but this is the same
1218 : // as for compaction generally: compaction is allowed to delete layers that readers might be trying to use.
1219 : // - We do not run concurrently with other kinds of compaction, so the only layer map writes we race with are:
1220 : // - GC, which at worst witnesses us "undelete" a layer that they just deleted.
1221 : // - ingestion, which only inserts layers, therefore cannot collide with us.
1222 0 : let resident = layer.download_and_keep_resident(ctx).await?;
1223 :
1224 0 : let keys_written = resident
1225 0 : .filter(&self.shard_identity, &mut image_layer_writer, ctx)
1226 0 : .await?;
1227 :
1228 0 : if keys_written > 0 {
1229 0 : let (desc, path) = image_layer_writer
1230 0 : .finish(ctx)
1231 0 : .await
1232 0 : .map_err(CompactionError::Other)?;
1233 0 : let new_layer = Layer::finish_creating(self.conf, self, desc, &path)
1234 0 : .map_err(CompactionError::Other)?;
1235 0 : tracing::info!(layer=%new_layer, "Rewrote layer, {} -> {} bytes",
1236 0 : layer.metadata().file_size,
1237 0 : new_layer.metadata().file_size);
1238 :
1239 0 : replace_image_layers.push((layer, new_layer));
1240 0 : } else {
1241 0 : // Drop the old layer. Usually for this case we would already have noticed that
1242 0 : // the layer has no data for us with the ShardedRange check above, but
1243 0 : drop_layers.push(layer);
1244 0 : }
1245 : }
1246 :
1247 : // At this point, we have replaced local layer files with their rewritten form, but not yet uploaded
1248 : // metadata to reflect that. If we restart here, the replaced layer files will look invalid (size mismatch
1249 : // to remote index) and be removed. This is inefficient but safe.
1250 0 : fail::fail_point!("compact-shard-ancestors-localonly");
1251 0 :
1252 0 : // Update the LayerMap so that readers will use the new layers, and enqueue it for writing to remote storage
1253 0 : self.rewrite_layers(replace_image_layers, drop_layers)
1254 0 : .await?;
1255 :
1256 0 : fail::fail_point!("compact-shard-ancestors-enqueued");
1257 0 :
1258 0 : // We wait for all uploads to complete before finishing this compaction stage. This is not
1259 0 : // necessary for correctness, but it simplifies testing, and avoids proceeding with another
1260 0 : // Timeline's compaction while this timeline's uploads may be generating lots of disk I/O
1261 0 : // load.
1262 0 : match self.remote_client.wait_completion().await {
1263 0 : Ok(()) => (),
1264 0 : Err(WaitCompletionError::NotInitialized(ni)) => return Err(CompactionError::from(ni)),
1265 : Err(WaitCompletionError::UploadQueueShutDownOrStopped) => {
1266 0 : return Err(CompactionError::ShuttingDown);
1267 : }
1268 : }
1269 :
1270 0 : fail::fail_point!("compact-shard-ancestors-persistent");
1271 0 :
1272 0 : Ok(())
1273 0 : }
1274 :
1275 : /// Update the LayerVisibilityHint of layers covered by image layers, based on whether there is
1276 : /// an image layer between them and the most recent readable LSN (branch point or tip of timeline). The
1277 : /// purpose of the visibility hint is to record which layers need to be available to service reads.
1278 : ///
1279 : /// The result may be used as an input to eviction and secondary downloads to de-prioritize layers
1280 : /// that we know won't be needed for reads.
1281 468 : pub(crate) async fn update_layer_visibility(
1282 468 : &self,
1283 468 : ) -> Result<(), super::layer_manager::Shutdown> {
1284 468 : let head_lsn = self.get_last_record_lsn();
1285 :
1286 : // We will sweep through layers in reverse-LSN order. We only do historic layers. L0 deltas
1287 : // are implicitly left visible, because LayerVisibilityHint's default is Visible, and we never modify it here.
1288 : // Note that L0 deltas _can_ be covered by image layers, but we consider them 'visible' because we anticipate that
1289 : // they will be subject to L0->L1 compaction in the near future.
1290 468 : let layer_manager = self.layers.read().await;
1291 468 : let layer_map = layer_manager.layer_map()?;
1292 :
1293 468 : let readable_points = {
1294 468 : let children = self.gc_info.read().unwrap().retain_lsns.clone();
1295 468 :
1296 468 : let mut readable_points = Vec::with_capacity(children.len() + 1);
1297 468 : for (child_lsn, _child_timeline_id, is_offloaded) in &children {
1298 0 : if *is_offloaded == MaybeOffloaded::Yes {
1299 0 : continue;
1300 0 : }
1301 0 : readable_points.push(*child_lsn);
1302 : }
1303 468 : readable_points.push(head_lsn);
1304 468 : readable_points
1305 468 : };
1306 468 :
1307 468 : let (layer_visibility, covered) = layer_map.get_visibility(readable_points);
1308 1184 : for (layer_desc, visibility) in layer_visibility {
1309 716 : // FIXME: a more efficiency bulk zip() through the layers rather than NlogN getting each one
1310 716 : let layer = layer_manager.get_from_desc(&layer_desc);
1311 716 : layer.set_visibility(visibility);
1312 716 : }
1313 :
1314 : // TODO: publish our covered KeySpace to our parent, so that when they update their visibility, they can
1315 : // avoid assuming that everything at a branch point is visible.
1316 468 : drop(covered);
1317 468 : Ok(())
1318 468 : }
1319 :
1320 : /// Collect a bunch of Level 0 layer files, and compact and reshuffle them as
1321 : /// as Level 1 files. Returns whether the L0 layers are fully compacted.
1322 728 : async fn compact_level0(
1323 728 : self: &Arc<Self>,
1324 728 : target_file_size: u64,
1325 728 : force_compaction_ignore_threshold: bool,
1326 728 : ctx: &RequestContext,
1327 728 : ) -> Result<CompactionOutcome, CompactionError> {
1328 : let CompactLevel0Phase1Result {
1329 728 : new_layers,
1330 728 : deltas_to_compact,
1331 728 : outcome,
1332 : } = {
1333 728 : let phase1_span = info_span!("compact_level0_phase1");
1334 728 : let ctx = ctx.attached_child();
1335 728 : let mut stats = CompactLevel0Phase1StatsBuilder {
1336 728 : version: Some(2),
1337 728 : tenant_id: Some(self.tenant_shard_id),
1338 728 : timeline_id: Some(self.timeline_id),
1339 728 : ..Default::default()
1340 728 : };
1341 728 :
1342 728 : let begin = tokio::time::Instant::now();
1343 728 : let phase1_layers_locked = self.layers.read().await;
1344 728 : let now = tokio::time::Instant::now();
1345 728 : stats.read_lock_acquisition_micros =
1346 728 : DurationRecorder::Recorded(RecordedDuration(now - begin), now);
1347 728 : self.compact_level0_phase1(
1348 728 : phase1_layers_locked,
1349 728 : stats,
1350 728 : target_file_size,
1351 728 : force_compaction_ignore_threshold,
1352 728 : &ctx,
1353 728 : )
1354 728 : .instrument(phase1_span)
1355 728 : .await?
1356 : };
1357 :
1358 728 : if new_layers.is_empty() && deltas_to_compact.is_empty() {
1359 : // nothing to do
1360 672 : return Ok(CompactionOutcome::Done);
1361 56 : }
1362 56 :
1363 56 : self.finish_compact_batch(&new_layers, &Vec::new(), &deltas_to_compact)
1364 56 : .await?;
1365 56 : Ok(outcome)
1366 728 : }
1367 :
1368 : /// Level0 files first phase of compaction, explained in the [`Self::compact_legacy`] comment.
1369 728 : async fn compact_level0_phase1<'a>(
1370 728 : self: &'a Arc<Self>,
1371 728 : guard: tokio::sync::RwLockReadGuard<'a, LayerManager>,
1372 728 : mut stats: CompactLevel0Phase1StatsBuilder,
1373 728 : target_file_size: u64,
1374 728 : force_compaction_ignore_threshold: bool,
1375 728 : ctx: &RequestContext,
1376 728 : ) -> Result<CompactLevel0Phase1Result, CompactionError> {
1377 728 : stats.read_lock_held_spawn_blocking_startup_micros =
1378 728 : stats.read_lock_acquisition_micros.till_now(); // set by caller
1379 728 : let layers = guard.layer_map()?;
1380 728 : let level0_deltas = layers.level0_deltas();
1381 728 : stats.level0_deltas_count = Some(level0_deltas.len());
1382 728 :
1383 728 : // Only compact if enough layers have accumulated.
1384 728 : let threshold = self.get_compaction_threshold();
1385 728 : if level0_deltas.is_empty() || level0_deltas.len() < threshold {
1386 672 : if force_compaction_ignore_threshold {
1387 0 : if !level0_deltas.is_empty() {
1388 0 : info!(
1389 0 : level0_deltas = level0_deltas.len(),
1390 0 : threshold, "too few deltas to compact, but forcing compaction"
1391 : );
1392 : } else {
1393 0 : info!(
1394 0 : level0_deltas = level0_deltas.len(),
1395 0 : threshold, "too few deltas to compact, cannot force compaction"
1396 : );
1397 0 : return Ok(CompactLevel0Phase1Result::default());
1398 : }
1399 : } else {
1400 672 : debug!(
1401 0 : level0_deltas = level0_deltas.len(),
1402 0 : threshold, "too few deltas to compact"
1403 : );
1404 672 : return Ok(CompactLevel0Phase1Result::default());
1405 : }
1406 56 : }
1407 :
1408 56 : let mut level0_deltas = level0_deltas
1409 56 : .iter()
1410 804 : .map(|x| guard.get_from_desc(x))
1411 56 : .collect::<Vec<_>>();
1412 56 :
1413 56 : // Gather the files to compact in this iteration.
1414 56 : //
1415 56 : // Start with the oldest Level 0 delta file, and collect any other
1416 56 : // level 0 files that form a contiguous sequence, such that the end
1417 56 : // LSN of previous file matches the start LSN of the next file.
1418 56 : //
1419 56 : // Note that if the files don't form such a sequence, we might
1420 56 : // "compact" just a single file. That's a bit pointless, but it allows
1421 56 : // us to get rid of the level 0 file, and compact the other files on
1422 56 : // the next iteration. This could probably made smarter, but such
1423 56 : // "gaps" in the sequence of level 0 files should only happen in case
1424 56 : // of a crash, partial download from cloud storage, or something like
1425 56 : // that, so it's not a big deal in practice.
1426 1496 : level0_deltas.sort_by_key(|l| l.layer_desc().lsn_range.start);
1427 56 : let mut level0_deltas_iter = level0_deltas.iter();
1428 56 :
1429 56 : let first_level0_delta = level0_deltas_iter.next().unwrap();
1430 56 : let mut prev_lsn_end = first_level0_delta.layer_desc().lsn_range.end;
1431 56 : let mut deltas_to_compact = Vec::with_capacity(level0_deltas.len());
1432 56 :
1433 56 : // Accumulate the size of layers in `deltas_to_compact`
1434 56 : let mut deltas_to_compact_bytes = 0;
1435 56 :
1436 56 : // Under normal circumstances, we will accumulate up to compaction_upper_limit L0s of size
1437 56 : // checkpoint_distance each. To avoid edge cases using extra system resources, bound our
1438 56 : // work in this function to only operate on this much delta data at once.
1439 56 : //
1440 56 : // In general, compaction_threshold should be <= compaction_upper_limit, but in case that
1441 56 : // the constraint is not respected, we use the larger of the two.
1442 56 : let delta_size_limit = std::cmp::max(
1443 56 : self.get_compaction_upper_limit(),
1444 56 : self.get_compaction_threshold(),
1445 56 : ) as u64
1446 56 : * std::cmp::max(self.get_checkpoint_distance(), DEFAULT_CHECKPOINT_DISTANCE);
1447 56 :
1448 56 : let mut fully_compacted = true;
1449 56 :
1450 56 : deltas_to_compact.push(first_level0_delta.download_and_keep_resident(ctx).await?);
1451 804 : for l in level0_deltas_iter {
1452 748 : let lsn_range = &l.layer_desc().lsn_range;
1453 748 :
1454 748 : if lsn_range.start != prev_lsn_end {
1455 0 : break;
1456 748 : }
1457 748 : deltas_to_compact.push(l.download_and_keep_resident(ctx).await?);
1458 748 : deltas_to_compact_bytes += l.metadata().file_size;
1459 748 : prev_lsn_end = lsn_range.end;
1460 748 :
1461 748 : if deltas_to_compact_bytes >= delta_size_limit {
1462 0 : info!(
1463 0 : l0_deltas_selected = deltas_to_compact.len(),
1464 0 : l0_deltas_total = level0_deltas.len(),
1465 0 : "L0 compaction picker hit max delta layer size limit: {}",
1466 : delta_size_limit
1467 : );
1468 0 : fully_compacted = false;
1469 0 :
1470 0 : // Proceed with compaction, but only a subset of L0s
1471 0 : break;
1472 748 : }
1473 : }
1474 56 : let lsn_range = Range {
1475 56 : start: deltas_to_compact
1476 56 : .first()
1477 56 : .unwrap()
1478 56 : .layer_desc()
1479 56 : .lsn_range
1480 56 : .start,
1481 56 : end: deltas_to_compact.last().unwrap().layer_desc().lsn_range.end,
1482 56 : };
1483 56 :
1484 56 : info!(
1485 0 : "Starting Level0 compaction in LSN range {}-{} for {} layers ({} deltas in total)",
1486 0 : lsn_range.start,
1487 0 : lsn_range.end,
1488 0 : deltas_to_compact.len(),
1489 0 : level0_deltas.len()
1490 : );
1491 :
1492 804 : for l in deltas_to_compact.iter() {
1493 804 : info!("compact includes {l}");
1494 : }
1495 :
1496 : // We don't need the original list of layers anymore. Drop it so that
1497 : // we don't accidentally use it later in the function.
1498 56 : drop(level0_deltas);
1499 56 :
1500 56 : stats.read_lock_held_prerequisites_micros = stats
1501 56 : .read_lock_held_spawn_blocking_startup_micros
1502 56 : .till_now();
1503 :
1504 : // TODO: replace with streaming k-merge
1505 56 : let all_keys = {
1506 56 : let mut all_keys = Vec::new();
1507 804 : for l in deltas_to_compact.iter() {
1508 804 : if self.cancel.is_cancelled() {
1509 0 : return Err(CompactionError::ShuttingDown);
1510 804 : }
1511 804 : let delta = l.get_as_delta(ctx).await.map_err(CompactionError::Other)?;
1512 804 : let keys = delta
1513 804 : .index_entries(ctx)
1514 804 : .await
1515 804 : .map_err(CompactionError::Other)?;
1516 804 : all_keys.extend(keys);
1517 : }
1518 : // The current stdlib sorting implementation is designed in a way where it is
1519 : // particularly fast where the slice is made up of sorted sub-ranges.
1520 8847552 : all_keys.sort_by_key(|DeltaEntry { key, lsn, .. }| (*key, *lsn));
1521 56 : all_keys
1522 56 : };
1523 56 :
1524 56 : stats.read_lock_held_key_sort_micros = stats.read_lock_held_prerequisites_micros.till_now();
1525 :
1526 : // Determine N largest holes where N is number of compacted layers. The vec is sorted by key range start.
1527 : //
1528 : // A hole is a key range for which this compaction doesn't have any WAL records.
1529 : // Our goal in this compaction iteration is to avoid creating L1s that, in terms of their key range,
1530 : // cover the hole, but actually don't contain any WAL records for that key range.
1531 : // The reason is that the mere stack of L1s (`count_deltas`) triggers image layer creation (`create_image_layers`).
1532 : // That image layer creation would be useless for a hole range covered by L1s that don't contain any WAL records.
1533 : //
1534 : // The algorithm chooses holes as follows.
1535 : // - Slide a 2-window over the keys in key orde to get the hole range (=distance between two keys).
1536 : // - Filter: min threshold on range length
1537 : // - Rank: by coverage size (=number of image layers required to reconstruct each key in the range for which we have any data)
1538 : //
1539 : // For more details, intuition, and some ASCII art see https://github.com/neondatabase/neon/pull/3597#discussion_r1112704451
1540 : #[derive(PartialEq, Eq)]
1541 : struct Hole {
1542 : key_range: Range<Key>,
1543 : coverage_size: usize,
1544 : }
1545 56 : let holes: Vec<Hole> = {
1546 : use std::cmp::Ordering;
1547 : impl Ord for Hole {
1548 0 : fn cmp(&self, other: &Self) -> Ordering {
1549 0 : self.coverage_size.cmp(&other.coverage_size).reverse()
1550 0 : }
1551 : }
1552 : impl PartialOrd for Hole {
1553 0 : fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
1554 0 : Some(self.cmp(other))
1555 0 : }
1556 : }
1557 56 : let max_holes = deltas_to_compact.len();
1558 56 : let last_record_lsn = self.get_last_record_lsn();
1559 56 : let min_hole_range = (target_file_size / page_cache::PAGE_SZ as u64) as i128;
1560 56 : let min_hole_coverage_size = 3; // TODO: something more flexible?
1561 56 : // min-heap (reserve space for one more element added before eviction)
1562 56 : let mut heap: BinaryHeap<Hole> = BinaryHeap::with_capacity(max_holes + 1);
1563 56 : let mut prev: Option<Key> = None;
1564 :
1565 4128076 : for &DeltaEntry { key: next_key, .. } in all_keys.iter() {
1566 4128076 : if let Some(prev_key) = prev {
1567 : // just first fast filter, do not create hole entries for metadata keys. The last hole in the
1568 : // compaction is the gap between data key and metadata keys.
1569 4128020 : if next_key.to_i128() - prev_key.to_i128() >= min_hole_range
1570 0 : && !Key::is_metadata_key(&prev_key)
1571 : {
1572 0 : let key_range = prev_key..next_key;
1573 0 : // Measuring hole by just subtraction of i128 representation of key range boundaries
1574 0 : // has not so much sense, because largest holes will corresponds field1/field2 changes.
1575 0 : // But we are mostly interested to eliminate holes which cause generation of excessive image layers.
1576 0 : // That is why it is better to measure size of hole as number of covering image layers.
1577 0 : let coverage_size =
1578 0 : layers.image_coverage(&key_range, last_record_lsn).len();
1579 0 : if coverage_size >= min_hole_coverage_size {
1580 0 : heap.push(Hole {
1581 0 : key_range,
1582 0 : coverage_size,
1583 0 : });
1584 0 : if heap.len() > max_holes {
1585 0 : heap.pop(); // remove smallest hole
1586 0 : }
1587 0 : }
1588 4128020 : }
1589 56 : }
1590 4128076 : prev = Some(next_key.next());
1591 : }
1592 56 : let mut holes = heap.into_vec();
1593 56 : holes.sort_unstable_by_key(|hole| hole.key_range.start);
1594 56 : holes
1595 56 : };
1596 56 : stats.read_lock_held_compute_holes_micros = stats.read_lock_held_key_sort_micros.till_now();
1597 56 : drop_rlock(guard);
1598 56 :
1599 56 : if self.cancel.is_cancelled() {
1600 0 : return Err(CompactionError::ShuttingDown);
1601 56 : }
1602 56 :
1603 56 : stats.read_lock_drop_micros = stats.read_lock_held_compute_holes_micros.till_now();
1604 :
1605 : // This iterator walks through all key-value pairs from all the layers
1606 : // we're compacting, in key, LSN order.
1607 : // If there's both a Value::Image and Value::WalRecord for the same (key,lsn),
1608 : // then the Value::Image is ordered before Value::WalRecord.
1609 56 : let mut all_values_iter = {
1610 56 : let mut deltas = Vec::with_capacity(deltas_to_compact.len());
1611 804 : for l in deltas_to_compact.iter() {
1612 804 : let l = l.get_as_delta(ctx).await.map_err(CompactionError::Other)?;
1613 804 : deltas.push(l);
1614 : }
1615 56 : MergeIterator::create(&deltas, &[], ctx)
1616 56 : };
1617 56 :
1618 56 : // This iterator walks through all keys and is needed to calculate size used by each key
1619 56 : let mut all_keys_iter = all_keys
1620 56 : .iter()
1621 4128076 : .map(|DeltaEntry { key, lsn, size, .. }| (*key, *lsn, *size))
1622 4128020 : .coalesce(|mut prev, cur| {
1623 4128020 : // Coalesce keys that belong to the same key pair.
1624 4128020 : // This ensures that compaction doesn't put them
1625 4128020 : // into different layer files.
1626 4128020 : // Still limit this by the target file size,
1627 4128020 : // so that we keep the size of the files in
1628 4128020 : // check.
1629 4128020 : if prev.0 == cur.0 && prev.2 < target_file_size {
1630 80076 : prev.2 += cur.2;
1631 80076 : Ok(prev)
1632 : } else {
1633 4047944 : Err((prev, cur))
1634 : }
1635 4128020 : });
1636 56 :
1637 56 : // Merge the contents of all the input delta layers into a new set
1638 56 : // of delta layers, based on the current partitioning.
1639 56 : //
1640 56 : // We split the new delta layers on the key dimension. We iterate through the key space, and for each key, check if including the next key to the current output layer we're building would cause the layer to become too large. If so, dump the current output layer and start new one.
1641 56 : // It's possible that there is a single key with so many page versions that storing all of them in a single layer file
1642 56 : // would be too large. In that case, we also split on the LSN dimension.
1643 56 : //
1644 56 : // LSN
1645 56 : // ^
1646 56 : // |
1647 56 : // | +-----------+ +--+--+--+--+
1648 56 : // | | | | | | | |
1649 56 : // | +-----------+ | | | | |
1650 56 : // | | | | | | | |
1651 56 : // | +-----------+ ==> | | | | |
1652 56 : // | | | | | | | |
1653 56 : // | +-----------+ | | | | |
1654 56 : // | | | | | | | |
1655 56 : // | +-----------+ +--+--+--+--+
1656 56 : // |
1657 56 : // +--------------> key
1658 56 : //
1659 56 : //
1660 56 : // If one key (X) has a lot of page versions:
1661 56 : //
1662 56 : // LSN
1663 56 : // ^
1664 56 : // | (X)
1665 56 : // | +-----------+ +--+--+--+--+
1666 56 : // | | | | | | | |
1667 56 : // | +-----------+ | | +--+ |
1668 56 : // | | | | | | | |
1669 56 : // | +-----------+ ==> | | | | |
1670 56 : // | | | | | +--+ |
1671 56 : // | +-----------+ | | | | |
1672 56 : // | | | | | | | |
1673 56 : // | +-----------+ +--+--+--+--+
1674 56 : // |
1675 56 : // +--------------> key
1676 56 : // TODO: this actually divides the layers into fixed-size chunks, not
1677 56 : // based on the partitioning.
1678 56 : //
1679 56 : // TODO: we should also opportunistically materialize and
1680 56 : // garbage collect what we can.
1681 56 : let mut new_layers = Vec::new();
1682 56 : let mut prev_key: Option<Key> = None;
1683 56 : let mut writer: Option<DeltaLayerWriter> = None;
1684 56 : let mut key_values_total_size = 0u64;
1685 56 : let mut dup_start_lsn: Lsn = Lsn::INVALID; // start LSN of layer containing values of the single key
1686 56 : let mut dup_end_lsn: Lsn = Lsn::INVALID; // end LSN of layer containing values of the single key
1687 56 : let mut next_hole = 0; // index of next hole in holes vector
1688 56 :
1689 56 : let mut keys = 0;
1690 :
1691 4128132 : while let Some((key, lsn, value)) = all_values_iter
1692 4128132 : .next()
1693 4128132 : .await
1694 4128132 : .map_err(CompactionError::Other)?
1695 : {
1696 4128076 : keys += 1;
1697 4128076 :
1698 4128076 : if keys % 32_768 == 0 && self.cancel.is_cancelled() {
1699 : // avoid hitting the cancellation token on every key. in benches, we end up
1700 : // shuffling an order of million keys per layer, this means we'll check it
1701 : // around tens of times per layer.
1702 0 : return Err(CompactionError::ShuttingDown);
1703 4128076 : }
1704 4128076 :
1705 4128076 : let same_key = prev_key == Some(key);
1706 4128076 : // We need to check key boundaries once we reach next key or end of layer with the same key
1707 4128076 : if !same_key || lsn == dup_end_lsn {
1708 4048000 : let mut next_key_size = 0u64;
1709 4048000 : let is_dup_layer = dup_end_lsn.is_valid();
1710 4048000 : dup_start_lsn = Lsn::INVALID;
1711 4048000 : if !same_key {
1712 4048000 : dup_end_lsn = Lsn::INVALID;
1713 4048000 : }
1714 : // Determine size occupied by this key. We stop at next key or when size becomes larger than target_file_size
1715 4048000 : for (next_key, next_lsn, next_size) in all_keys_iter.by_ref() {
1716 4048000 : next_key_size = next_size;
1717 4048000 : if key != next_key {
1718 4047944 : if dup_end_lsn.is_valid() {
1719 0 : // We are writting segment with duplicates:
1720 0 : // place all remaining values of this key in separate segment
1721 0 : dup_start_lsn = dup_end_lsn; // new segments starts where old stops
1722 0 : dup_end_lsn = lsn_range.end; // there are no more values of this key till end of LSN range
1723 4047944 : }
1724 4047944 : break;
1725 56 : }
1726 56 : key_values_total_size += next_size;
1727 56 : // Check if it is time to split segment: if total keys size is larger than target file size.
1728 56 : // We need to avoid generation of empty segments if next_size > target_file_size.
1729 56 : if key_values_total_size > target_file_size && lsn != next_lsn {
1730 : // Split key between multiple layers: such layer can contain only single key
1731 0 : dup_start_lsn = if dup_end_lsn.is_valid() {
1732 0 : dup_end_lsn // new segment with duplicates starts where old one stops
1733 : } else {
1734 0 : lsn // start with the first LSN for this key
1735 : };
1736 0 : dup_end_lsn = next_lsn; // upper LSN boundary is exclusive
1737 0 : break;
1738 56 : }
1739 : }
1740 : // handle case when loop reaches last key: in this case dup_end is non-zero but dup_start is not set.
1741 4048000 : if dup_end_lsn.is_valid() && !dup_start_lsn.is_valid() {
1742 0 : dup_start_lsn = dup_end_lsn;
1743 0 : dup_end_lsn = lsn_range.end;
1744 4048000 : }
1745 4048000 : if writer.is_some() {
1746 4047944 : let written_size = writer.as_mut().unwrap().size();
1747 4047944 : let contains_hole =
1748 4047944 : next_hole < holes.len() && key >= holes[next_hole].key_range.end;
1749 : // check if key cause layer overflow or contains hole...
1750 4047944 : if is_dup_layer
1751 4047944 : || dup_end_lsn.is_valid()
1752 4047944 : || written_size + key_values_total_size > target_file_size
1753 4047384 : || contains_hole
1754 : {
1755 : // ... if so, flush previous layer and prepare to write new one
1756 560 : let (desc, path) = writer
1757 560 : .take()
1758 560 : .unwrap()
1759 560 : .finish(prev_key.unwrap().next(), ctx)
1760 560 : .await
1761 560 : .map_err(CompactionError::Other)?;
1762 560 : let new_delta = Layer::finish_creating(self.conf, self, desc, &path)
1763 560 : .map_err(CompactionError::Other)?;
1764 :
1765 560 : new_layers.push(new_delta);
1766 560 : writer = None;
1767 560 :
1768 560 : if contains_hole {
1769 0 : // skip hole
1770 0 : next_hole += 1;
1771 560 : }
1772 4047384 : }
1773 56 : }
1774 : // Remember size of key value because at next iteration we will access next item
1775 4048000 : key_values_total_size = next_key_size;
1776 80076 : }
1777 4128076 : fail_point!("delta-layer-writer-fail-before-finish", |_| {
1778 0 : Err(CompactionError::Other(anyhow::anyhow!(
1779 0 : "failpoint delta-layer-writer-fail-before-finish"
1780 0 : )))
1781 4128076 : });
1782 :
1783 4128076 : if !self.shard_identity.is_key_disposable(&key) {
1784 4128076 : if writer.is_none() {
1785 616 : if self.cancel.is_cancelled() {
1786 : // to be somewhat responsive to cancellation, check for each new layer
1787 0 : return Err(CompactionError::ShuttingDown);
1788 616 : }
1789 : // Create writer if not initiaized yet
1790 616 : writer = Some(
1791 : DeltaLayerWriter::new(
1792 616 : self.conf,
1793 616 : self.timeline_id,
1794 616 : self.tenant_shard_id,
1795 616 : key,
1796 616 : if dup_end_lsn.is_valid() {
1797 : // this is a layer containing slice of values of the same key
1798 0 : debug!("Create new dup layer {}..{}", dup_start_lsn, dup_end_lsn);
1799 0 : dup_start_lsn..dup_end_lsn
1800 : } else {
1801 616 : debug!("Create new layer {}..{}", lsn_range.start, lsn_range.end);
1802 616 : lsn_range.clone()
1803 : },
1804 616 : ctx,
1805 616 : )
1806 616 : .await
1807 616 : .map_err(CompactionError::Other)?,
1808 : );
1809 :
1810 616 : keys = 0;
1811 4127460 : }
1812 :
1813 4128076 : writer
1814 4128076 : .as_mut()
1815 4128076 : .unwrap()
1816 4128076 : .put_value(key, lsn, value, ctx)
1817 4128076 : .await
1818 4128076 : .map_err(CompactionError::Other)?;
1819 : } else {
1820 0 : let owner = self.shard_identity.get_shard_number(&key);
1821 0 :
1822 0 : // This happens after a shard split, when we're compacting an L0 created by our parent shard
1823 0 : debug!("dropping key {key} during compaction (it belongs on shard {owner})");
1824 : }
1825 :
1826 4128076 : if !new_layers.is_empty() {
1827 39572 : fail_point!("after-timeline-compacted-first-L1");
1828 4088504 : }
1829 :
1830 4128076 : prev_key = Some(key);
1831 : }
1832 56 : if let Some(writer) = writer {
1833 56 : let (desc, path) = writer
1834 56 : .finish(prev_key.unwrap().next(), ctx)
1835 56 : .await
1836 56 : .map_err(CompactionError::Other)?;
1837 56 : let new_delta = Layer::finish_creating(self.conf, self, desc, &path)
1838 56 : .map_err(CompactionError::Other)?;
1839 56 : new_layers.push(new_delta);
1840 0 : }
1841 :
1842 : // Sync layers
1843 56 : if !new_layers.is_empty() {
1844 : // Print a warning if the created layer is larger than double the target size
1845 : // Add two pages for potential overhead. This should in theory be already
1846 : // accounted for in the target calculation, but for very small targets,
1847 : // we still might easily hit the limit otherwise.
1848 56 : let warn_limit = target_file_size * 2 + page_cache::PAGE_SZ as u64 * 2;
1849 616 : for layer in new_layers.iter() {
1850 616 : if layer.layer_desc().file_size > warn_limit {
1851 0 : warn!(
1852 : %layer,
1853 0 : "created delta file of size {} larger than double of target of {target_file_size}", layer.layer_desc().file_size
1854 : );
1855 616 : }
1856 : }
1857 :
1858 : // The writer.finish() above already did the fsync of the inodes.
1859 : // We just need to fsync the directory in which these inodes are linked,
1860 : // which we know to be the timeline directory.
1861 : //
1862 : // We use fatal_err() below because the after writer.finish() returns with success,
1863 : // the in-memory state of the filesystem already has the layer file in its final place,
1864 : // and subsequent pageserver code could think it's durable while it really isn't.
1865 56 : let timeline_dir = VirtualFile::open(
1866 56 : &self
1867 56 : .conf
1868 56 : .timeline_path(&self.tenant_shard_id, &self.timeline_id),
1869 56 : ctx,
1870 56 : )
1871 56 : .await
1872 56 : .fatal_err("VirtualFile::open for timeline dir fsync");
1873 56 : timeline_dir
1874 56 : .sync_all()
1875 56 : .await
1876 56 : .fatal_err("VirtualFile::sync_all timeline dir");
1877 0 : }
1878 :
1879 56 : stats.write_layer_files_micros = stats.read_lock_drop_micros.till_now();
1880 56 : stats.new_deltas_count = Some(new_layers.len());
1881 616 : stats.new_deltas_size = Some(new_layers.iter().map(|l| l.layer_desc().file_size).sum());
1882 56 :
1883 56 : match TryInto::<CompactLevel0Phase1Stats>::try_into(stats)
1884 56 : .and_then(|stats| serde_json::to_string(&stats).context("serde_json::to_string"))
1885 : {
1886 56 : Ok(stats_json) => {
1887 56 : info!(
1888 0 : stats_json = stats_json.as_str(),
1889 0 : "compact_level0_phase1 stats available"
1890 : )
1891 : }
1892 0 : Err(e) => {
1893 0 : warn!("compact_level0_phase1 stats failed to serialize: {:#}", e);
1894 : }
1895 : }
1896 :
1897 : // Without this, rustc complains about deltas_to_compact still
1898 : // being borrowed when we `.into_iter()` below.
1899 56 : drop(all_values_iter);
1900 56 :
1901 56 : Ok(CompactLevel0Phase1Result {
1902 56 : new_layers,
1903 56 : deltas_to_compact: deltas_to_compact
1904 56 : .into_iter()
1905 804 : .map(|x| x.drop_eviction_guard())
1906 56 : .collect::<Vec<_>>(),
1907 56 : outcome: if fully_compacted {
1908 56 : CompactionOutcome::Done
1909 : } else {
1910 0 : CompactionOutcome::Pending
1911 : },
1912 : })
1913 728 : }
1914 : }
1915 :
1916 : #[derive(Default)]
1917 : struct CompactLevel0Phase1Result {
1918 : new_layers: Vec<ResidentLayer>,
1919 : deltas_to_compact: Vec<Layer>,
1920 : // Whether we have included all L0 layers, or selected only part of them due to the
1921 : // L0 compaction size limit.
1922 : outcome: CompactionOutcome,
1923 : }
1924 :
1925 : #[derive(Default)]
1926 : struct CompactLevel0Phase1StatsBuilder {
1927 : version: Option<u64>,
1928 : tenant_id: Option<TenantShardId>,
1929 : timeline_id: Option<TimelineId>,
1930 : read_lock_acquisition_micros: DurationRecorder,
1931 : read_lock_held_spawn_blocking_startup_micros: DurationRecorder,
1932 : read_lock_held_key_sort_micros: DurationRecorder,
1933 : read_lock_held_prerequisites_micros: DurationRecorder,
1934 : read_lock_held_compute_holes_micros: DurationRecorder,
1935 : read_lock_drop_micros: DurationRecorder,
1936 : write_layer_files_micros: DurationRecorder,
1937 : level0_deltas_count: Option<usize>,
1938 : new_deltas_count: Option<usize>,
1939 : new_deltas_size: Option<u64>,
1940 : }
1941 :
1942 : #[derive(serde::Serialize)]
1943 : struct CompactLevel0Phase1Stats {
1944 : version: u64,
1945 : tenant_id: TenantShardId,
1946 : timeline_id: TimelineId,
1947 : read_lock_acquisition_micros: RecordedDuration,
1948 : read_lock_held_spawn_blocking_startup_micros: RecordedDuration,
1949 : read_lock_held_key_sort_micros: RecordedDuration,
1950 : read_lock_held_prerequisites_micros: RecordedDuration,
1951 : read_lock_held_compute_holes_micros: RecordedDuration,
1952 : read_lock_drop_micros: RecordedDuration,
1953 : write_layer_files_micros: RecordedDuration,
1954 : level0_deltas_count: usize,
1955 : new_deltas_count: usize,
1956 : new_deltas_size: u64,
1957 : }
1958 :
1959 : impl TryFrom<CompactLevel0Phase1StatsBuilder> for CompactLevel0Phase1Stats {
1960 : type Error = anyhow::Error;
1961 :
1962 56 : fn try_from(value: CompactLevel0Phase1StatsBuilder) -> Result<Self, Self::Error> {
1963 56 : Ok(Self {
1964 56 : version: value.version.ok_or_else(|| anyhow!("version not set"))?,
1965 56 : tenant_id: value
1966 56 : .tenant_id
1967 56 : .ok_or_else(|| anyhow!("tenant_id not set"))?,
1968 56 : timeline_id: value
1969 56 : .timeline_id
1970 56 : .ok_or_else(|| anyhow!("timeline_id not set"))?,
1971 56 : read_lock_acquisition_micros: value
1972 56 : .read_lock_acquisition_micros
1973 56 : .into_recorded()
1974 56 : .ok_or_else(|| anyhow!("read_lock_acquisition_micros not set"))?,
1975 56 : read_lock_held_spawn_blocking_startup_micros: value
1976 56 : .read_lock_held_spawn_blocking_startup_micros
1977 56 : .into_recorded()
1978 56 : .ok_or_else(|| anyhow!("read_lock_held_spawn_blocking_startup_micros not set"))?,
1979 56 : read_lock_held_key_sort_micros: value
1980 56 : .read_lock_held_key_sort_micros
1981 56 : .into_recorded()
1982 56 : .ok_or_else(|| anyhow!("read_lock_held_key_sort_micros not set"))?,
1983 56 : read_lock_held_prerequisites_micros: value
1984 56 : .read_lock_held_prerequisites_micros
1985 56 : .into_recorded()
1986 56 : .ok_or_else(|| anyhow!("read_lock_held_prerequisites_micros not set"))?,
1987 56 : read_lock_held_compute_holes_micros: value
1988 56 : .read_lock_held_compute_holes_micros
1989 56 : .into_recorded()
1990 56 : .ok_or_else(|| anyhow!("read_lock_held_compute_holes_micros not set"))?,
1991 56 : read_lock_drop_micros: value
1992 56 : .read_lock_drop_micros
1993 56 : .into_recorded()
1994 56 : .ok_or_else(|| anyhow!("read_lock_drop_micros not set"))?,
1995 56 : write_layer_files_micros: value
1996 56 : .write_layer_files_micros
1997 56 : .into_recorded()
1998 56 : .ok_or_else(|| anyhow!("write_layer_files_micros not set"))?,
1999 56 : level0_deltas_count: value
2000 56 : .level0_deltas_count
2001 56 : .ok_or_else(|| anyhow!("level0_deltas_count not set"))?,
2002 56 : new_deltas_count: value
2003 56 : .new_deltas_count
2004 56 : .ok_or_else(|| anyhow!("new_deltas_count not set"))?,
2005 56 : new_deltas_size: value
2006 56 : .new_deltas_size
2007 56 : .ok_or_else(|| anyhow!("new_deltas_size not set"))?,
2008 : })
2009 56 : }
2010 : }
2011 :
2012 : impl Timeline {
2013 : /// Entry point for new tiered compaction algorithm.
2014 : ///
2015 : /// All the real work is in the implementation in the pageserver_compaction
2016 : /// crate. The code here would apply to any algorithm implemented by the
2017 : /// same interface, but tiered is the only one at the moment.
2018 : ///
2019 : /// TODO: cancellation
2020 0 : pub(crate) async fn compact_tiered(
2021 0 : self: &Arc<Self>,
2022 0 : _cancel: &CancellationToken,
2023 0 : ctx: &RequestContext,
2024 0 : ) -> Result<(), CompactionError> {
2025 0 : let fanout = self.get_compaction_threshold() as u64;
2026 0 : let target_file_size = self.get_checkpoint_distance();
2027 :
2028 : // Find the top of the historical layers
2029 0 : let end_lsn = {
2030 0 : let guard = self.layers.read().await;
2031 0 : let layers = guard.layer_map()?;
2032 :
2033 0 : let l0_deltas = layers.level0_deltas();
2034 0 :
2035 0 : // As an optimization, if we find that there are too few L0 layers,
2036 0 : // bail out early. We know that the compaction algorithm would do
2037 0 : // nothing in that case.
2038 0 : if l0_deltas.len() < fanout as usize {
2039 : // doesn't need compacting
2040 0 : return Ok(());
2041 0 : }
2042 0 : l0_deltas.iter().map(|l| l.lsn_range.end).max().unwrap()
2043 0 : };
2044 0 :
2045 0 : // Is the timeline being deleted?
2046 0 : if self.is_stopping() {
2047 0 : trace!("Dropping out of compaction on timeline shutdown");
2048 0 : return Err(CompactionError::ShuttingDown);
2049 0 : }
2050 :
2051 0 : let (dense_ks, _sparse_ks) = self.collect_keyspace(end_lsn, ctx).await?;
2052 : // TODO(chi): ignore sparse_keyspace for now, compact it in the future.
2053 0 : let mut adaptor = TimelineAdaptor::new(self, (end_lsn, dense_ks));
2054 0 :
2055 0 : pageserver_compaction::compact_tiered::compact_tiered(
2056 0 : &mut adaptor,
2057 0 : end_lsn,
2058 0 : target_file_size,
2059 0 : fanout,
2060 0 : ctx,
2061 0 : )
2062 0 : .await
2063 : // TODO: compact_tiered needs to return CompactionError
2064 0 : .map_err(CompactionError::Other)?;
2065 :
2066 0 : adaptor.flush_updates().await?;
2067 0 : Ok(())
2068 0 : }
2069 :
2070 : /// Take a list of images and deltas, produce images and deltas according to GC horizon and retain_lsns.
2071 : ///
2072 : /// It takes a key, the values of the key within the compaction process, a GC horizon, and all retain_lsns below the horizon.
2073 : /// For now, it requires the `accumulated_values` contains the full history of the key (i.e., the key with the lowest LSN is
2074 : /// an image or a WAL not requiring a base image). This restriction will be removed once we implement gc-compaction on branch.
2075 : ///
2076 : /// The function returns the deltas and the base image that need to be placed at each of the retain LSN. For example, we have:
2077 : ///
2078 : /// A@0x10, +B@0x20, +C@0x30, +D@0x40, +E@0x50, +F@0x60
2079 : /// horizon = 0x50, retain_lsn = 0x20, 0x40, delta_threshold=3
2080 : ///
2081 : /// The function will produce:
2082 : ///
2083 : /// ```plain
2084 : /// 0x20(retain_lsn) -> img=AB@0x20 always produce a single image below the lowest retain LSN
2085 : /// 0x40(retain_lsn) -> deltas=[+C@0x30, +D@0x40] two deltas since the last base image, keeping the deltas
2086 : /// 0x50(horizon) -> deltas=[ABCDE@0x50] three deltas since the last base image, generate an image but put it in the delta
2087 : /// above_horizon -> deltas=[+F@0x60] full history above the horizon
2088 : /// ```
2089 : ///
2090 : /// Note that `accumulated_values` must be sorted by LSN and should belong to a single key.
2091 1260 : pub(crate) async fn generate_key_retention(
2092 1260 : self: &Arc<Timeline>,
2093 1260 : key: Key,
2094 1260 : full_history: &[(Key, Lsn, Value)],
2095 1260 : horizon: Lsn,
2096 1260 : retain_lsn_below_horizon: &[Lsn],
2097 1260 : delta_threshold_cnt: usize,
2098 1260 : base_img_from_ancestor: Option<(Key, Lsn, Bytes)>,
2099 1260 : ) -> anyhow::Result<KeyHistoryRetention> {
2100 : // Pre-checks for the invariants
2101 :
2102 1260 : let debug_mode = cfg!(debug_assertions) || cfg!(feature = "testing");
2103 :
2104 1260 : if debug_mode {
2105 3060 : for (log_key, _, _) in full_history {
2106 1800 : assert_eq!(log_key, &key, "mismatched key");
2107 : }
2108 1260 : for i in 1..full_history.len() {
2109 540 : assert!(full_history[i - 1].1 <= full_history[i].1, "unordered LSN");
2110 540 : if full_history[i - 1].1 == full_history[i].1 {
2111 0 : assert!(
2112 0 : matches!(full_history[i - 1].2, Value::Image(_)),
2113 0 : "unordered delta/image, or duplicated delta"
2114 : );
2115 540 : }
2116 : }
2117 : // There was an assertion for no base image that checks if the first
2118 : // record in the history is `will_init` before, but it was removed.
2119 : // This is explained in the test cases for generate_key_retention.
2120 : // Search "incomplete history" for more information.
2121 2820 : for lsn in retain_lsn_below_horizon {
2122 1560 : assert!(lsn < &horizon, "retain lsn must be below horizon")
2123 : }
2124 1260 : for i in 1..retain_lsn_below_horizon.len() {
2125 712 : assert!(
2126 712 : retain_lsn_below_horizon[i - 1] <= retain_lsn_below_horizon[i],
2127 0 : "unordered LSN"
2128 : );
2129 : }
2130 0 : }
2131 1260 : let has_ancestor = base_img_from_ancestor.is_some();
2132 : // Step 1: split history into len(retain_lsn_below_horizon) + 2 buckets, where the last bucket is for all deltas above the horizon,
2133 : // and the second-to-last bucket is for the horizon. Each bucket contains lsn_last_bucket < deltas <= lsn_this_bucket.
2134 1260 : let (mut split_history, lsn_split_points) = {
2135 1260 : let mut split_history = Vec::new();
2136 1260 : split_history.resize_with(retain_lsn_below_horizon.len() + 2, Vec::new);
2137 1260 : let mut lsn_split_points = Vec::with_capacity(retain_lsn_below_horizon.len() + 1);
2138 2820 : for lsn in retain_lsn_below_horizon {
2139 1560 : lsn_split_points.push(*lsn);
2140 1560 : }
2141 1260 : lsn_split_points.push(horizon);
2142 1260 : let mut current_idx = 0;
2143 3060 : for item @ (_, lsn, _) in full_history {
2144 2288 : while current_idx < lsn_split_points.len() && *lsn > lsn_split_points[current_idx] {
2145 488 : current_idx += 1;
2146 488 : }
2147 1800 : split_history[current_idx].push(item);
2148 : }
2149 1260 : (split_history, lsn_split_points)
2150 : };
2151 : // Step 2: filter out duplicated records due to the k-merge of image/delta layers
2152 5340 : for split_for_lsn in &mut split_history {
2153 4080 : let mut prev_lsn = None;
2154 4080 : let mut new_split_for_lsn = Vec::with_capacity(split_for_lsn.len());
2155 4080 : for record @ (_, lsn, _) in std::mem::take(split_for_lsn) {
2156 1800 : if let Some(prev_lsn) = &prev_lsn {
2157 236 : if *prev_lsn == lsn {
2158 : // The case that we have an LSN with both data from the delta layer and the image layer. As
2159 : // `ValueWrapper` ensures that an image is ordered before a delta at the same LSN, we simply
2160 : // drop this delta and keep the image.
2161 : //
2162 : // For example, we have delta layer key1@0x10, key1@0x20, and image layer key1@0x10, we will
2163 : // keep the image for key1@0x10 and the delta for key1@0x20. key1@0x10 delta will be simply
2164 : // dropped.
2165 : //
2166 : // TODO: in case we have both delta + images for a given LSN and it does not exceed the delta
2167 : // threshold, we could have kept delta instead to save space. This is an optimization for the future.
2168 0 : continue;
2169 236 : }
2170 1564 : }
2171 1800 : prev_lsn = Some(lsn);
2172 1800 : new_split_for_lsn.push(record);
2173 : }
2174 4080 : *split_for_lsn = new_split_for_lsn;
2175 : }
2176 : // Step 3: generate images when necessary
2177 1260 : let mut retention = Vec::with_capacity(split_history.len());
2178 1260 : let mut records_since_last_image = 0;
2179 1260 : let batch_cnt = split_history.len();
2180 1260 : assert!(
2181 1260 : batch_cnt >= 2,
2182 0 : "should have at least below + above horizon batches"
2183 : );
2184 1260 : let mut replay_history: Vec<(Key, Lsn, Value)> = Vec::new();
2185 1260 : if let Some((key, lsn, img)) = base_img_from_ancestor {
2186 84 : replay_history.push((key, lsn, Value::Image(img)));
2187 1176 : }
2188 :
2189 : /// Generate debug information for the replay history
2190 0 : fn generate_history_trace(replay_history: &[(Key, Lsn, Value)]) -> String {
2191 : use std::fmt::Write;
2192 0 : let mut output = String::new();
2193 0 : if let Some((key, _, _)) = replay_history.first() {
2194 0 : write!(output, "key={} ", key).unwrap();
2195 0 : let mut cnt = 0;
2196 0 : for (_, lsn, val) in replay_history {
2197 0 : if val.is_image() {
2198 0 : write!(output, "i@{} ", lsn).unwrap();
2199 0 : } else if val.will_init() {
2200 0 : write!(output, "di@{} ", lsn).unwrap();
2201 0 : } else {
2202 0 : write!(output, "d@{} ", lsn).unwrap();
2203 0 : }
2204 0 : cnt += 1;
2205 0 : if cnt >= 128 {
2206 0 : write!(output, "... and more").unwrap();
2207 0 : break;
2208 0 : }
2209 : }
2210 0 : } else {
2211 0 : write!(output, "<no history>").unwrap();
2212 0 : }
2213 0 : output
2214 0 : }
2215 :
2216 0 : fn generate_debug_trace(
2217 0 : replay_history: Option<&[(Key, Lsn, Value)]>,
2218 0 : full_history: &[(Key, Lsn, Value)],
2219 0 : lsns: &[Lsn],
2220 0 : horizon: Lsn,
2221 0 : ) -> String {
2222 : use std::fmt::Write;
2223 0 : let mut output = String::new();
2224 0 : if let Some(replay_history) = replay_history {
2225 0 : writeln!(
2226 0 : output,
2227 0 : "replay_history: {}",
2228 0 : generate_history_trace(replay_history)
2229 0 : )
2230 0 : .unwrap();
2231 0 : } else {
2232 0 : writeln!(output, "replay_history: <disabled>",).unwrap();
2233 0 : }
2234 0 : writeln!(
2235 0 : output,
2236 0 : "full_history: {}",
2237 0 : generate_history_trace(full_history)
2238 0 : )
2239 0 : .unwrap();
2240 0 : writeln!(
2241 0 : output,
2242 0 : "when processing: [{}] horizon={}",
2243 0 : lsns.iter().map(|l| format!("{l}")).join(","),
2244 0 : horizon
2245 0 : )
2246 0 : .unwrap();
2247 0 : output
2248 0 : }
2249 :
2250 1260 : let mut key_exists = false;
2251 4080 : for (i, split_for_lsn) in split_history.into_iter().enumerate() {
2252 : // TODO: there could be image keys inside the splits, and we can compute records_since_last_image accordingly.
2253 4080 : records_since_last_image += split_for_lsn.len();
2254 : // Whether to produce an image into the final layer files
2255 4080 : let produce_image = if i == 0 && !has_ancestor {
2256 : // We always generate images for the first batch (below horizon / lowest retain_lsn)
2257 1176 : true
2258 2904 : } else if i == batch_cnt - 1 {
2259 : // Do not generate images for the last batch (above horizon)
2260 1260 : false
2261 1644 : } else if records_since_last_image == 0 {
2262 1288 : false
2263 356 : } else if records_since_last_image >= delta_threshold_cnt {
2264 : // Generate images when there are too many records
2265 12 : true
2266 : } else {
2267 344 : false
2268 : };
2269 4080 : replay_history.extend(split_for_lsn.iter().map(|x| (*x).clone()));
2270 : // Only retain the items after the last image record
2271 5028 : for idx in (0..replay_history.len()).rev() {
2272 5028 : if replay_history[idx].2.will_init() {
2273 4080 : replay_history = replay_history[idx..].to_vec();
2274 4080 : break;
2275 948 : }
2276 : }
2277 4080 : if replay_history.is_empty() && !key_exists {
2278 : // The key does not exist at earlier LSN, we can skip this iteration.
2279 0 : retention.push(Vec::new());
2280 0 : continue;
2281 4080 : } else {
2282 4080 : key_exists = true;
2283 4080 : }
2284 4080 : let Some((_, _, val)) = replay_history.first() else {
2285 0 : unreachable!("replay history should not be empty once it exists")
2286 : };
2287 4080 : if !val.will_init() {
2288 0 : return Err(anyhow::anyhow!("invalid history, no base image")).with_context(|| {
2289 0 : generate_debug_trace(
2290 0 : Some(&replay_history),
2291 0 : full_history,
2292 0 : retain_lsn_below_horizon,
2293 0 : horizon,
2294 0 : )
2295 0 : });
2296 4080 : }
2297 : // Whether to reconstruct the image. In debug mode, we will generate an image
2298 : // at every retain_lsn to ensure data is not corrupted, but we won't put the
2299 : // image into the final layer.
2300 4080 : let generate_image = produce_image || debug_mode;
2301 4080 : if produce_image {
2302 1188 : records_since_last_image = 0;
2303 2892 : }
2304 4080 : let img_and_lsn = if generate_image {
2305 4080 : let replay_history_for_debug = if debug_mode {
2306 4080 : Some(replay_history.clone())
2307 : } else {
2308 0 : None
2309 : };
2310 4080 : let replay_history_for_debug_ref = replay_history_for_debug.as_deref();
2311 4080 : let history = if produce_image {
2312 1188 : std::mem::take(&mut replay_history)
2313 : } else {
2314 2892 : replay_history.clone()
2315 : };
2316 4080 : let mut img = None;
2317 4080 : let mut records = Vec::with_capacity(history.len());
2318 4080 : if let (_, lsn, Value::Image(val)) = history.first().as_ref().unwrap() {
2319 4036 : img = Some((*lsn, val.clone()));
2320 4036 : for (_, lsn, val) in history.into_iter().skip(1) {
2321 920 : let Value::WalRecord(rec) = val else {
2322 0 : return Err(anyhow::anyhow!(
2323 0 : "invalid record, first record is image, expect walrecords"
2324 0 : ))
2325 0 : .with_context(|| {
2326 0 : generate_debug_trace(
2327 0 : replay_history_for_debug_ref,
2328 0 : full_history,
2329 0 : retain_lsn_below_horizon,
2330 0 : horizon,
2331 0 : )
2332 0 : });
2333 : };
2334 920 : records.push((lsn, rec));
2335 : }
2336 : } else {
2337 72 : for (_, lsn, val) in history.into_iter() {
2338 72 : let Value::WalRecord(rec) = val else {
2339 0 : return Err(anyhow::anyhow!("invalid record, first record is walrecord, expect rest are walrecord"))
2340 0 : .with_context(|| generate_debug_trace(
2341 0 : replay_history_for_debug_ref,
2342 0 : full_history,
2343 0 : retain_lsn_below_horizon,
2344 0 : horizon,
2345 0 : ));
2346 : };
2347 72 : records.push((lsn, rec));
2348 : }
2349 : }
2350 4080 : records.reverse();
2351 4080 : let state = ValueReconstructState { img, records };
2352 : // last batch does not generate image so i is always in range, unless we force generate
2353 : // an image during testing
2354 4080 : let request_lsn = if i >= lsn_split_points.len() {
2355 1260 : Lsn::MAX
2356 : } else {
2357 2820 : lsn_split_points[i]
2358 : };
2359 4080 : let img = self.reconstruct_value(key, request_lsn, state).await?;
2360 4080 : Some((request_lsn, img))
2361 : } else {
2362 0 : None
2363 : };
2364 4080 : if produce_image {
2365 1188 : let (request_lsn, img) = img_and_lsn.unwrap();
2366 1188 : replay_history.push((key, request_lsn, Value::Image(img.clone())));
2367 1188 : retention.push(vec![(request_lsn, Value::Image(img))]);
2368 2892 : } else {
2369 2892 : let deltas = split_for_lsn
2370 2892 : .iter()
2371 2892 : .map(|(_, lsn, value)| (*lsn, value.clone()))
2372 2892 : .collect_vec();
2373 2892 : retention.push(deltas);
2374 2892 : }
2375 : }
2376 1260 : let mut result = Vec::with_capacity(retention.len());
2377 1260 : assert_eq!(retention.len(), lsn_split_points.len() + 1);
2378 4080 : for (idx, logs) in retention.into_iter().enumerate() {
2379 4080 : if idx == lsn_split_points.len() {
2380 1260 : return Ok(KeyHistoryRetention {
2381 1260 : below_horizon: result,
2382 1260 : above_horizon: KeyLogAtLsn(logs),
2383 1260 : });
2384 2820 : } else {
2385 2820 : result.push((lsn_split_points[idx], KeyLogAtLsn(logs)));
2386 2820 : }
2387 : }
2388 0 : unreachable!("key retention is empty")
2389 1260 : }
2390 :
2391 : /// Check how much space is left on the disk
2392 104 : async fn check_available_space(self: &Arc<Self>) -> anyhow::Result<u64> {
2393 104 : let tenants_dir = self.conf.tenants_path();
2394 :
2395 104 : let stat = Statvfs::get(&tenants_dir, None)
2396 104 : .context("statvfs failed, presumably directory got unlinked")?;
2397 :
2398 104 : let (avail_bytes, _) = stat.get_avail_total_bytes();
2399 104 :
2400 104 : Ok(avail_bytes)
2401 104 : }
2402 :
2403 : /// Check if the compaction can proceed safely without running out of space. We assume the size
2404 : /// upper bound of the produced files of a compaction job is the same as all layers involved in
2405 : /// the compaction. Therefore, we need `2 * layers_to_be_compacted_size` at least to do a
2406 : /// compaction.
2407 104 : async fn check_compaction_space(
2408 104 : self: &Arc<Self>,
2409 104 : layer_selection: &[Layer],
2410 104 : ) -> Result<(), CompactionError> {
2411 104 : let available_space = self
2412 104 : .check_available_space()
2413 104 : .await
2414 104 : .map_err(CompactionError::Other)?;
2415 104 : let mut remote_layer_size = 0;
2416 104 : let mut all_layer_size = 0;
2417 408 : for layer in layer_selection {
2418 304 : let needs_download = layer
2419 304 : .needs_download()
2420 304 : .await
2421 304 : .context("failed to check if layer needs download")
2422 304 : .map_err(CompactionError::Other)?;
2423 304 : if needs_download.is_some() {
2424 0 : remote_layer_size += layer.layer_desc().file_size;
2425 304 : }
2426 304 : all_layer_size += layer.layer_desc().file_size;
2427 : }
2428 104 : let allocated_space = (available_space as f64 * 0.8) as u64; /* reserve 20% space for other tasks */
2429 104 : if all_layer_size /* space needed for newly-generated file */ + remote_layer_size /* space for downloading layers */ > allocated_space
2430 : {
2431 0 : return Err(CompactionError::Other(anyhow!(
2432 0 : "not enough space for compaction: available_space={}, allocated_space={}, all_layer_size={}, remote_layer_size={}, required_space={}",
2433 0 : available_space,
2434 0 : allocated_space,
2435 0 : all_layer_size,
2436 0 : remote_layer_size,
2437 0 : all_layer_size + remote_layer_size
2438 0 : )));
2439 104 : }
2440 104 : Ok(())
2441 104 : }
2442 :
2443 : /// Get a watermark for gc-compaction, that is the lowest LSN that we can use as the `gc_horizon` for
2444 : /// the compaction algorithm. It is min(space_cutoff, time_cutoff, latest_gc_cutoff, standby_horizon).
2445 : /// Leases and retain_lsns are considered in the gc-compaction job itself so we don't need to account for them
2446 : /// here.
2447 108 : pub(crate) fn get_gc_compaction_watermark(self: &Arc<Self>) -> Lsn {
2448 108 : let gc_cutoff_lsn = {
2449 108 : let gc_info = self.gc_info.read().unwrap();
2450 108 : gc_info.min_cutoff()
2451 108 : };
2452 108 :
2453 108 : // TODO: standby horizon should use leases so we don't really need to consider it here.
2454 108 : // let watermark = watermark.min(self.standby_horizon.load());
2455 108 :
2456 108 : // TODO: ensure the child branches will not use anything below the watermark, or consider
2457 108 : // them when computing the watermark.
2458 108 : gc_cutoff_lsn.min(*self.get_applied_gc_cutoff_lsn())
2459 108 : }
2460 :
2461 : /// Split a gc-compaction job into multiple compaction jobs. The split is based on the key range and the estimated size of the compaction job.
2462 : /// The function returns a list of compaction jobs that can be executed separately. If the upper bound of the compact LSN
2463 : /// range is not specified, we will use the latest gc_cutoff as the upper bound, so that all jobs in the jobset acts
2464 : /// like a full compaction of the specified keyspace.
2465 0 : pub(crate) async fn gc_compaction_split_jobs(
2466 0 : self: &Arc<Self>,
2467 0 : job: GcCompactJob,
2468 0 : sub_compaction_max_job_size_mb: Option<u64>,
2469 0 : ) -> Result<Vec<GcCompactJob>, CompactionError> {
2470 0 : let compact_below_lsn = if job.compact_lsn_range.end != Lsn::MAX {
2471 0 : job.compact_lsn_range.end
2472 : } else {
2473 0 : self.get_gc_compaction_watermark()
2474 : };
2475 :
2476 0 : if compact_below_lsn == Lsn::INVALID {
2477 0 : tracing::warn!(
2478 0 : "no layers to compact with gc: gc_cutoff not generated yet, skipping gc bottom-most compaction"
2479 : );
2480 0 : return Ok(vec![]);
2481 0 : }
2482 :
2483 : // Split compaction job to about 4GB each
2484 : const GC_COMPACT_MAX_SIZE_MB: u64 = 4 * 1024;
2485 0 : let sub_compaction_max_job_size_mb =
2486 0 : sub_compaction_max_job_size_mb.unwrap_or(GC_COMPACT_MAX_SIZE_MB);
2487 0 :
2488 0 : let mut compact_jobs = Vec::<GcCompactJob>::new();
2489 0 : // For now, we simply use the key partitioning information; we should do a more fine-grained partitioning
2490 0 : // by estimating the amount of files read for a compaction job. We should also partition on LSN.
2491 0 : let ((dense_ks, sparse_ks), _) = self.partitioning.read().as_ref().clone();
2492 : // Truncate the key range to be within user specified compaction range.
2493 0 : fn truncate_to(
2494 0 : source_start: &Key,
2495 0 : source_end: &Key,
2496 0 : target_start: &Key,
2497 0 : target_end: &Key,
2498 0 : ) -> Option<(Key, Key)> {
2499 0 : let start = source_start.max(target_start);
2500 0 : let end = source_end.min(target_end);
2501 0 : if start < end {
2502 0 : Some((*start, *end))
2503 : } else {
2504 0 : None
2505 : }
2506 0 : }
2507 0 : let mut split_key_ranges = Vec::new();
2508 0 : let ranges = dense_ks
2509 0 : .parts
2510 0 : .iter()
2511 0 : .map(|partition| partition.ranges.iter())
2512 0 : .chain(sparse_ks.parts.iter().map(|x| x.0.ranges.iter()))
2513 0 : .flatten()
2514 0 : .cloned()
2515 0 : .collect_vec();
2516 0 : for range in ranges.iter() {
2517 0 : let Some((start, end)) = truncate_to(
2518 0 : &range.start,
2519 0 : &range.end,
2520 0 : &job.compact_key_range.start,
2521 0 : &job.compact_key_range.end,
2522 0 : ) else {
2523 0 : continue;
2524 : };
2525 0 : split_key_ranges.push((start, end));
2526 : }
2527 0 : split_key_ranges.sort();
2528 0 : let all_layers = {
2529 0 : let guard = self.layers.read().await;
2530 0 : let layer_map = guard.layer_map()?;
2531 0 : layer_map.iter_historic_layers().collect_vec()
2532 0 : };
2533 0 : let mut current_start = None;
2534 0 : let ranges_num = split_key_ranges.len();
2535 0 : for (idx, (start, end)) in split_key_ranges.into_iter().enumerate() {
2536 0 : if current_start.is_none() {
2537 0 : current_start = Some(start);
2538 0 : }
2539 0 : let start = current_start.unwrap();
2540 0 : if start >= end {
2541 : // We have already processed this partition.
2542 0 : continue;
2543 0 : }
2544 0 : let overlapping_layers = {
2545 0 : let mut desc = Vec::new();
2546 0 : for layer in all_layers.iter() {
2547 0 : if overlaps_with(&layer.get_key_range(), &(start..end))
2548 0 : && layer.get_lsn_range().start <= compact_below_lsn
2549 0 : {
2550 0 : desc.push(layer.clone());
2551 0 : }
2552 : }
2553 0 : desc
2554 0 : };
2555 0 : let total_size = overlapping_layers.iter().map(|x| x.file_size).sum::<u64>();
2556 0 : if total_size > sub_compaction_max_job_size_mb * 1024 * 1024 || ranges_num == idx + 1 {
2557 : // Try to extend the compaction range so that we include at least one full layer file.
2558 0 : let extended_end = overlapping_layers
2559 0 : .iter()
2560 0 : .map(|layer| layer.key_range.end)
2561 0 : .min();
2562 : // It is possible that the search range does not contain any layer files when we reach the end of the loop.
2563 : // In this case, we simply use the specified key range end.
2564 0 : let end = if let Some(extended_end) = extended_end {
2565 0 : extended_end.max(end)
2566 : } else {
2567 0 : end
2568 : };
2569 0 : let end = if ranges_num == idx + 1 {
2570 : // extend the compaction range to the end of the key range if it's the last partition
2571 0 : end.max(job.compact_key_range.end)
2572 : } else {
2573 0 : end
2574 : };
2575 0 : if total_size == 0 && !compact_jobs.is_empty() {
2576 0 : info!(
2577 0 : "splitting compaction job: {}..{}, estimated_size={}, extending the previous job",
2578 : start, end, total_size
2579 : );
2580 0 : compact_jobs.last_mut().unwrap().compact_key_range.end = end;
2581 0 : current_start = Some(end);
2582 : } else {
2583 0 : info!(
2584 0 : "splitting compaction job: {}..{}, estimated_size={}",
2585 : start, end, total_size
2586 : );
2587 0 : compact_jobs.push(GcCompactJob {
2588 0 : dry_run: job.dry_run,
2589 0 : compact_key_range: start..end,
2590 0 : compact_lsn_range: job.compact_lsn_range.start..compact_below_lsn,
2591 0 : });
2592 0 : current_start = Some(end);
2593 : }
2594 0 : }
2595 : }
2596 0 : Ok(compact_jobs)
2597 0 : }
2598 :
2599 : /// An experimental compaction building block that combines compaction with garbage collection.
2600 : ///
2601 : /// The current implementation picks all delta + image layers that are below or intersecting with
2602 : /// the GC horizon without considering retain_lsns. Then, it does a full compaction over all these delta
2603 : /// layers and image layers, which generates image layers on the gc horizon, drop deltas below gc horizon,
2604 : /// and create delta layers with all deltas >= gc horizon.
2605 : ///
2606 : /// If `options.compact_range` is provided, it will only compact the keys within the range, aka partial compaction.
2607 : /// Partial compaction will read and process all layers overlapping with the key range, even if it might
2608 : /// contain extra keys. After the gc-compaction phase completes, delta layers that are not fully contained
2609 : /// within the key range will be rewritten to ensure they do not overlap with the delta layers. Providing
2610 : /// Key::MIN..Key..MAX to the function indicates a full compaction, though technically, `Key::MAX` is not
2611 : /// part of the range.
2612 : ///
2613 : /// If `options.compact_lsn_range.end` is provided, the compaction will only compact layers below or intersect with
2614 : /// the LSN. Otherwise, it will use the gc cutoff by default.
2615 108 : pub(crate) async fn compact_with_gc(
2616 108 : self: &Arc<Self>,
2617 108 : cancel: &CancellationToken,
2618 108 : options: CompactOptions,
2619 108 : ctx: &RequestContext,
2620 108 : ) -> Result<CompactionOutcome, CompactionError> {
2621 108 : let sub_compaction = options.sub_compaction;
2622 108 : let job = GcCompactJob::from_compact_options(options.clone());
2623 108 : let no_yield = options.flags.contains(CompactFlags::NoYield);
2624 108 : if sub_compaction {
2625 0 : info!(
2626 0 : "running enhanced gc bottom-most compaction with sub-compaction, splitting compaction jobs"
2627 : );
2628 0 : let jobs = self
2629 0 : .gc_compaction_split_jobs(job, options.sub_compaction_max_job_size_mb)
2630 0 : .await?;
2631 0 : let jobs_len = jobs.len();
2632 0 : for (idx, job) in jobs.into_iter().enumerate() {
2633 0 : info!(
2634 0 : "running enhanced gc bottom-most compaction, sub-compaction {}/{}",
2635 0 : idx + 1,
2636 : jobs_len
2637 : );
2638 0 : self.compact_with_gc_inner(cancel, job, ctx, no_yield)
2639 0 : .await?;
2640 : }
2641 0 : if jobs_len == 0 {
2642 0 : info!("no jobs to run, skipping gc bottom-most compaction");
2643 0 : }
2644 0 : return Ok(CompactionOutcome::Done);
2645 108 : }
2646 108 : self.compact_with_gc_inner(cancel, job, ctx, no_yield).await
2647 108 : }
2648 :
2649 108 : async fn compact_with_gc_inner(
2650 108 : self: &Arc<Self>,
2651 108 : cancel: &CancellationToken,
2652 108 : job: GcCompactJob,
2653 108 : ctx: &RequestContext,
2654 108 : no_yield: bool,
2655 108 : ) -> Result<CompactionOutcome, CompactionError> {
2656 108 : // Block other compaction/GC tasks from running for now. GC-compaction could run along
2657 108 : // with legacy compaction tasks in the future. Always ensure the lock order is compaction -> gc.
2658 108 : // Note that we already acquired the compaction lock when the outer `compact` function gets called.
2659 108 :
2660 108 : let timer = Instant::now();
2661 108 : let begin_timer = timer;
2662 108 :
2663 108 : let gc_lock = async {
2664 108 : tokio::select! {
2665 108 : guard = self.gc_lock.lock() => Ok(guard),
2666 108 : _ = cancel.cancelled() => Err(CompactionError::ShuttingDown),
2667 : }
2668 108 : };
2669 :
2670 108 : let time_acquire_lock = timer.elapsed();
2671 108 : let timer = Instant::now();
2672 :
2673 108 : let gc_lock = crate::timed(
2674 108 : gc_lock,
2675 108 : "acquires gc lock",
2676 108 : std::time::Duration::from_secs(5),
2677 108 : )
2678 108 : .await?;
2679 :
2680 108 : let dry_run = job.dry_run;
2681 108 : let compact_key_range = job.compact_key_range;
2682 108 : let compact_lsn_range = job.compact_lsn_range;
2683 :
2684 108 : let debug_mode = cfg!(debug_assertions) || cfg!(feature = "testing");
2685 :
2686 108 : info!(
2687 0 : "running enhanced gc bottom-most compaction, dry_run={dry_run}, compact_key_range={}..{}, compact_lsn_range={}..{}",
2688 : compact_key_range.start,
2689 : compact_key_range.end,
2690 : compact_lsn_range.start,
2691 : compact_lsn_range.end
2692 : );
2693 :
2694 108 : scopeguard::defer! {
2695 108 : info!("done enhanced gc bottom-most compaction");
2696 108 : };
2697 108 :
2698 108 : let mut stat = CompactionStatistics::default();
2699 :
2700 : // Step 0: pick all delta layers + image layers below/intersect with the GC horizon.
2701 : // The layer selection has the following properties:
2702 : // 1. If a layer is in the selection, all layers below it are in the selection.
2703 : // 2. Inferred from (1), for each key in the layer selection, the value can be reconstructed only with the layers in the layer selection.
2704 104 : let job_desc = {
2705 108 : let guard = self.layers.read().await;
2706 108 : let layers = guard.layer_map()?;
2707 108 : let gc_info = self.gc_info.read().unwrap();
2708 108 : let mut retain_lsns_below_horizon = Vec::new();
2709 108 : let gc_cutoff = {
2710 : // Currently, gc-compaction only kicks in after the legacy gc has updated the gc_cutoff.
2711 : // Therefore, it can only clean up data that cannot be cleaned up with legacy gc, instead of
2712 : // cleaning everything that theoritically it could. In the future, it should use `self.gc_info`
2713 : // to get the truth data.
2714 108 : let real_gc_cutoff = self.get_gc_compaction_watermark();
2715 : // The compaction algorithm will keep all keys above the gc_cutoff while keeping only necessary keys below the gc_cutoff for
2716 : // each of the retain_lsn. Therefore, if the user-provided `compact_lsn_range.end` is larger than the real gc cutoff, we will use
2717 : // the real cutoff.
2718 108 : let mut gc_cutoff = if compact_lsn_range.end == Lsn::MAX {
2719 96 : if real_gc_cutoff == Lsn::INVALID {
2720 : // If the gc_cutoff is not generated yet, we should not compact anything.
2721 0 : tracing::warn!(
2722 0 : "no layers to compact with gc: gc_cutoff not generated yet, skipping gc bottom-most compaction"
2723 : );
2724 0 : return Ok(CompactionOutcome::Skipped);
2725 96 : }
2726 96 : real_gc_cutoff
2727 : } else {
2728 12 : compact_lsn_range.end
2729 : };
2730 108 : if gc_cutoff > real_gc_cutoff {
2731 8 : warn!(
2732 0 : "provided compact_lsn_range.end={} is larger than the real_gc_cutoff={}, using the real gc cutoff",
2733 : gc_cutoff, real_gc_cutoff
2734 : );
2735 8 : gc_cutoff = real_gc_cutoff;
2736 100 : }
2737 108 : gc_cutoff
2738 : };
2739 140 : for (lsn, _timeline_id, _is_offloaded) in &gc_info.retain_lsns {
2740 140 : if lsn < &gc_cutoff {
2741 140 : retain_lsns_below_horizon.push(*lsn);
2742 140 : }
2743 : }
2744 108 : for lsn in gc_info.leases.keys() {
2745 0 : if lsn < &gc_cutoff {
2746 0 : retain_lsns_below_horizon.push(*lsn);
2747 0 : }
2748 : }
2749 108 : let mut selected_layers: Vec<Layer> = Vec::new();
2750 108 : drop(gc_info);
2751 : // Firstly, pick all the layers intersect or below the gc_cutoff, get the largest LSN in the selected layers.
2752 108 : let Some(max_layer_lsn) = layers
2753 108 : .iter_historic_layers()
2754 488 : .filter(|desc| desc.get_lsn_range().start <= gc_cutoff)
2755 416 : .map(|desc| desc.get_lsn_range().end)
2756 108 : .max()
2757 : else {
2758 0 : info!(
2759 0 : "no layers to compact with gc: no historic layers below gc_cutoff, gc_cutoff={}",
2760 : gc_cutoff
2761 : );
2762 0 : return Ok(CompactionOutcome::Done);
2763 : };
2764 : // Next, if the user specifies compact_lsn_range.start, we need to filter some layers out. All the layers (strictly) below
2765 : // the min_layer_lsn computed as below will be filtered out and the data will be accessed using the normal read path, as if
2766 : // it is a branch.
2767 108 : let Some(min_layer_lsn) = layers
2768 108 : .iter_historic_layers()
2769 488 : .filter(|desc| {
2770 488 : if compact_lsn_range.start == Lsn::INVALID {
2771 396 : true // select all layers below if start == Lsn(0)
2772 : } else {
2773 92 : desc.get_lsn_range().end > compact_lsn_range.start // strictly larger than compact_above_lsn
2774 : }
2775 488 : })
2776 452 : .map(|desc| desc.get_lsn_range().start)
2777 108 : .min()
2778 : else {
2779 0 : info!(
2780 0 : "no layers to compact with gc: no historic layers above compact_above_lsn, compact_above_lsn={}",
2781 : compact_lsn_range.end
2782 : );
2783 0 : return Ok(CompactionOutcome::Done);
2784 : };
2785 : // Then, pick all the layers that are below the max_layer_lsn. This is to ensure we can pick all single-key
2786 : // layers to compact.
2787 108 : let mut rewrite_layers = Vec::new();
2788 488 : for desc in layers.iter_historic_layers() {
2789 488 : if desc.get_lsn_range().end <= max_layer_lsn
2790 416 : && desc.get_lsn_range().start >= min_layer_lsn
2791 380 : && overlaps_with(&desc.get_key_range(), &compact_key_range)
2792 : {
2793 : // If the layer overlaps with the compaction key range, we need to read it to obtain all keys within the range,
2794 : // even if it might contain extra keys
2795 304 : selected_layers.push(guard.get_from_desc(&desc));
2796 304 : // If the layer is not fully contained within the key range, we need to rewrite it if it's a delta layer (it's fine
2797 304 : // to overlap image layers)
2798 304 : if desc.is_delta() && !fully_contains(&compact_key_range, &desc.get_key_range())
2799 4 : {
2800 4 : rewrite_layers.push(desc);
2801 300 : }
2802 184 : }
2803 : }
2804 108 : if selected_layers.is_empty() {
2805 4 : info!(
2806 0 : "no layers to compact with gc: no layers within the key range, gc_cutoff={}, key_range={}..{}",
2807 : gc_cutoff, compact_key_range.start, compact_key_range.end
2808 : );
2809 4 : return Ok(CompactionOutcome::Done);
2810 104 : }
2811 104 : retain_lsns_below_horizon.sort();
2812 104 : GcCompactionJobDescription {
2813 104 : selected_layers,
2814 104 : gc_cutoff,
2815 104 : retain_lsns_below_horizon,
2816 104 : min_layer_lsn,
2817 104 : max_layer_lsn,
2818 104 : compaction_key_range: compact_key_range,
2819 104 : rewrite_layers,
2820 104 : }
2821 : };
2822 104 : let (has_data_below, lowest_retain_lsn) = if compact_lsn_range.start != Lsn::INVALID {
2823 : // If we only compact above some LSN, we should get the history from the current branch below the specified LSN.
2824 : // We use job_desc.min_layer_lsn as if it's the lowest branch point.
2825 16 : (true, job_desc.min_layer_lsn)
2826 88 : } else if self.ancestor_timeline.is_some() {
2827 : // In theory, we can also use min_layer_lsn here, but using ancestor LSN makes sure the delta layers cover the
2828 : // LSN ranges all the way to the ancestor timeline.
2829 4 : (true, self.ancestor_lsn)
2830 : } else {
2831 84 : let res = job_desc
2832 84 : .retain_lsns_below_horizon
2833 84 : .first()
2834 84 : .copied()
2835 84 : .unwrap_or(job_desc.gc_cutoff);
2836 84 : if debug_mode {
2837 84 : assert_eq!(
2838 84 : res,
2839 84 : job_desc
2840 84 : .retain_lsns_below_horizon
2841 84 : .iter()
2842 84 : .min()
2843 84 : .copied()
2844 84 : .unwrap_or(job_desc.gc_cutoff)
2845 84 : );
2846 0 : }
2847 84 : (false, res)
2848 : };
2849 104 : info!(
2850 0 : "picked {} layers for compaction ({} layers need rewriting) with max_layer_lsn={} min_layer_lsn={} gc_cutoff={} lowest_retain_lsn={}, key_range={}..{}, has_data_below={}",
2851 0 : job_desc.selected_layers.len(),
2852 0 : job_desc.rewrite_layers.len(),
2853 : job_desc.max_layer_lsn,
2854 : job_desc.min_layer_lsn,
2855 : job_desc.gc_cutoff,
2856 : lowest_retain_lsn,
2857 : job_desc.compaction_key_range.start,
2858 : job_desc.compaction_key_range.end,
2859 : has_data_below,
2860 : );
2861 :
2862 104 : let time_analyze = timer.elapsed();
2863 104 : let timer = Instant::now();
2864 :
2865 408 : for layer in &job_desc.selected_layers {
2866 304 : debug!("read layer: {}", layer.layer_desc().key());
2867 : }
2868 108 : for layer in &job_desc.rewrite_layers {
2869 4 : debug!("rewrite layer: {}", layer.key());
2870 : }
2871 :
2872 104 : self.check_compaction_space(&job_desc.selected_layers)
2873 104 : .await?;
2874 :
2875 : // Generate statistics for the compaction
2876 408 : for layer in &job_desc.selected_layers {
2877 304 : let desc = layer.layer_desc();
2878 304 : if desc.is_delta() {
2879 172 : stat.visit_delta_layer(desc.file_size());
2880 172 : } else {
2881 132 : stat.visit_image_layer(desc.file_size());
2882 132 : }
2883 : }
2884 :
2885 : // Step 1: construct a k-merge iterator over all layers.
2886 : // Also, verify if the layer map can be split by drawing a horizontal line at every LSN start/end split point.
2887 104 : let layer_names = job_desc
2888 104 : .selected_layers
2889 104 : .iter()
2890 304 : .map(|layer| layer.layer_desc().layer_name())
2891 104 : .collect_vec();
2892 104 : if let Some(err) = check_valid_layermap(&layer_names) {
2893 0 : return Err(CompactionError::Other(anyhow!(
2894 0 : "gc-compaction layer map check failed because {}, cannot proceed with compaction due to potential data loss",
2895 0 : err
2896 0 : )));
2897 104 : }
2898 104 : // The maximum LSN we are processing in this compaction loop
2899 104 : let end_lsn = job_desc
2900 104 : .selected_layers
2901 104 : .iter()
2902 304 : .map(|l| l.layer_desc().lsn_range.end)
2903 104 : .max()
2904 104 : .unwrap();
2905 104 : let mut delta_layers = Vec::new();
2906 104 : let mut image_layers = Vec::new();
2907 104 : let mut downloaded_layers = Vec::new();
2908 104 : let mut total_downloaded_size = 0;
2909 104 : let mut total_layer_size = 0;
2910 408 : for layer in &job_desc.selected_layers {
2911 304 : if layer
2912 304 : .needs_download()
2913 304 : .await
2914 304 : .context("failed to check if layer needs download")
2915 304 : .map_err(CompactionError::Other)?
2916 304 : .is_some()
2917 0 : {
2918 0 : total_downloaded_size += layer.layer_desc().file_size;
2919 304 : }
2920 304 : total_layer_size += layer.layer_desc().file_size;
2921 304 : if cancel.is_cancelled() {
2922 0 : return Err(CompactionError::ShuttingDown);
2923 304 : }
2924 304 : if !no_yield {
2925 304 : let should_yield = self
2926 304 : .l0_compaction_trigger
2927 304 : .notified()
2928 304 : .now_or_never()
2929 304 : .is_some();
2930 304 : if should_yield {
2931 0 : tracing::info!(
2932 0 : "preempt gc-compaction when downloading layers: too many L0 layers"
2933 : );
2934 0 : return Ok(CompactionOutcome::YieldForL0);
2935 304 : }
2936 0 : }
2937 304 : let resident_layer = layer
2938 304 : .download_and_keep_resident(ctx)
2939 304 : .await
2940 304 : .context("failed to download and keep resident layer")
2941 304 : .map_err(CompactionError::Other)?;
2942 304 : downloaded_layers.push(resident_layer);
2943 : }
2944 104 : info!(
2945 0 : "finish downloading layers, downloaded={}, total={}, ratio={:.2}",
2946 0 : total_downloaded_size,
2947 0 : total_layer_size,
2948 0 : total_downloaded_size as f64 / total_layer_size as f64
2949 : );
2950 408 : for resident_layer in &downloaded_layers {
2951 304 : if resident_layer.layer_desc().is_delta() {
2952 172 : let layer = resident_layer
2953 172 : .get_as_delta(ctx)
2954 172 : .await
2955 172 : .context("failed to get delta layer")
2956 172 : .map_err(CompactionError::Other)?;
2957 172 : delta_layers.push(layer);
2958 : } else {
2959 132 : let layer = resident_layer
2960 132 : .get_as_image(ctx)
2961 132 : .await
2962 132 : .context("failed to get image layer")
2963 132 : .map_err(CompactionError::Other)?;
2964 132 : image_layers.push(layer);
2965 : }
2966 : }
2967 104 : let (dense_ks, sparse_ks) = self
2968 104 : .collect_gc_compaction_keyspace()
2969 104 : .await
2970 104 : .context("failed to collect gc compaction keyspace")
2971 104 : .map_err(CompactionError::Other)?;
2972 104 : let mut merge_iter = FilterIterator::create(
2973 104 : MergeIterator::create(&delta_layers, &image_layers, ctx),
2974 104 : dense_ks,
2975 104 : sparse_ks,
2976 104 : )
2977 104 : .context("failed to create filter iterator")
2978 104 : .map_err(CompactionError::Other)?;
2979 :
2980 104 : let time_download_layer = timer.elapsed();
2981 104 : let timer = Instant::now();
2982 104 :
2983 104 : // Step 2: Produce images+deltas.
2984 104 : let mut accumulated_values = Vec::new();
2985 104 : let mut last_key: Option<Key> = None;
2986 :
2987 : // Only create image layers when there is no ancestor branches. TODO: create covering image layer
2988 : // when some condition meet.
2989 104 : let mut image_layer_writer = if !has_data_below {
2990 : Some(
2991 84 : SplitImageLayerWriter::new(
2992 84 : self.conf,
2993 84 : self.timeline_id,
2994 84 : self.tenant_shard_id,
2995 84 : job_desc.compaction_key_range.start,
2996 84 : lowest_retain_lsn,
2997 84 : self.get_compaction_target_size(),
2998 84 : ctx,
2999 84 : )
3000 84 : .await
3001 84 : .context("failed to create image layer writer")
3002 84 : .map_err(CompactionError::Other)?,
3003 : )
3004 : } else {
3005 20 : None
3006 : };
3007 :
3008 104 : let mut delta_layer_writer = SplitDeltaLayerWriter::new(
3009 104 : self.conf,
3010 104 : self.timeline_id,
3011 104 : self.tenant_shard_id,
3012 104 : lowest_retain_lsn..end_lsn,
3013 104 : self.get_compaction_target_size(),
3014 104 : )
3015 104 : .await
3016 104 : .context("failed to create delta layer writer")
3017 104 : .map_err(CompactionError::Other)?;
3018 :
3019 : #[derive(Default)]
3020 : struct RewritingLayers {
3021 : before: Option<DeltaLayerWriter>,
3022 : after: Option<DeltaLayerWriter>,
3023 : }
3024 104 : let mut delta_layer_rewriters = HashMap::<Arc<PersistentLayerKey>, RewritingLayers>::new();
3025 :
3026 : /// When compacting not at a bottom range (=`[0,X)`) of the root branch, we "have data below" (`has_data_below=true`).
3027 : /// The two cases are compaction in ancestor branches and when `compact_lsn_range.start` is set.
3028 : /// In those cases, we need to pull up data from below the LSN range we're compaction.
3029 : ///
3030 : /// This function unifies the cases so that later code doesn't have to think about it.
3031 : ///
3032 : /// Currently, we always get the ancestor image for each key in the child branch no matter whether the image
3033 : /// is needed for reconstruction. This should be fixed in the future.
3034 : ///
3035 : /// Furthermore, we should do vectored get instead of a single get, or better, use k-merge for ancestor
3036 : /// images.
3037 1244 : async fn get_ancestor_image(
3038 1244 : this_tline: &Arc<Timeline>,
3039 1244 : key: Key,
3040 1244 : ctx: &RequestContext,
3041 1244 : has_data_below: bool,
3042 1244 : history_lsn_point: Lsn,
3043 1244 : ) -> anyhow::Result<Option<(Key, Lsn, Bytes)>> {
3044 1244 : if !has_data_below {
3045 1168 : return Ok(None);
3046 76 : };
3047 : // This function is implemented as a get of the current timeline at ancestor LSN, therefore reusing
3048 : // as much existing code as possible.
3049 76 : let img = this_tline.get(key, history_lsn_point, ctx).await?;
3050 76 : Ok(Some((key, history_lsn_point, img)))
3051 1244 : }
3052 :
3053 : // Actually, we can decide not to write to the image layer at all at this point because
3054 : // the key and LSN range are determined. However, to keep things simple here, we still
3055 : // create this writer, and discard the writer in the end.
3056 :
3057 104 : let mut keys_processed = 0;
3058 :
3059 1932 : while let Some(((key, lsn, val), desc)) = merge_iter
3060 1932 : .next_with_trace()
3061 1932 : .await
3062 1932 : .context("failed to get next key-value pair")
3063 1932 : .map_err(CompactionError::Other)?
3064 : {
3065 1828 : if cancel.is_cancelled() {
3066 0 : return Err(CompactionError::ShuttingDown);
3067 1828 : }
3068 1828 :
3069 1828 : if !no_yield {
3070 1828 : keys_processed += 1;
3071 1828 : if keys_processed % 1000 == 0 {
3072 0 : let should_yield = self
3073 0 : .l0_compaction_trigger
3074 0 : .notified()
3075 0 : .now_or_never()
3076 0 : .is_some();
3077 0 : if should_yield {
3078 0 : tracing::info!(
3079 0 : "preempt gc-compaction in the main loop: too many L0 layers"
3080 : );
3081 0 : return Ok(CompactionOutcome::YieldForL0);
3082 0 : }
3083 1828 : }
3084 0 : }
3085 1828 : if self.shard_identity.is_key_disposable(&key) {
3086 : // If this shard does not need to store this key, simply skip it.
3087 : //
3088 : // This is not handled in the filter iterator because shard is determined by hash.
3089 : // Therefore, it does not give us any performance benefit to do things like skip
3090 : // a whole layer file as handling key spaces (ranges).
3091 0 : if cfg!(debug_assertions) {
3092 0 : let shard = self.shard_identity.shard_index();
3093 0 : let owner = self.shard_identity.get_shard_number(&key);
3094 0 : panic!("key {key} does not belong on shard {shard}, owned by {owner}");
3095 0 : }
3096 0 : continue;
3097 1828 : }
3098 1828 : if !job_desc.compaction_key_range.contains(&key) {
3099 128 : if !desc.is_delta {
3100 120 : continue;
3101 8 : }
3102 8 : let rewriter = delta_layer_rewriters.entry(desc.clone()).or_default();
3103 8 : let rewriter = if key < job_desc.compaction_key_range.start {
3104 0 : if rewriter.before.is_none() {
3105 0 : rewriter.before = Some(
3106 0 : DeltaLayerWriter::new(
3107 0 : self.conf,
3108 0 : self.timeline_id,
3109 0 : self.tenant_shard_id,
3110 0 : desc.key_range.start,
3111 0 : desc.lsn_range.clone(),
3112 0 : ctx,
3113 0 : )
3114 0 : .await
3115 0 : .context("failed to create delta layer writer")
3116 0 : .map_err(CompactionError::Other)?,
3117 : );
3118 0 : }
3119 0 : rewriter.before.as_mut().unwrap()
3120 8 : } else if key >= job_desc.compaction_key_range.end {
3121 8 : if rewriter.after.is_none() {
3122 4 : rewriter.after = Some(
3123 4 : DeltaLayerWriter::new(
3124 4 : self.conf,
3125 4 : self.timeline_id,
3126 4 : self.tenant_shard_id,
3127 4 : job_desc.compaction_key_range.end,
3128 4 : desc.lsn_range.clone(),
3129 4 : ctx,
3130 4 : )
3131 4 : .await
3132 4 : .context("failed to create delta layer writer")
3133 4 : .map_err(CompactionError::Other)?,
3134 : );
3135 4 : }
3136 8 : rewriter.after.as_mut().unwrap()
3137 : } else {
3138 0 : unreachable!()
3139 : };
3140 8 : rewriter
3141 8 : .put_value(key, lsn, val, ctx)
3142 8 : .await
3143 8 : .context("failed to put value")
3144 8 : .map_err(CompactionError::Other)?;
3145 8 : continue;
3146 1700 : }
3147 1700 : match val {
3148 1220 : Value::Image(_) => stat.visit_image_key(&val),
3149 480 : Value::WalRecord(_) => stat.visit_wal_key(&val),
3150 : }
3151 1700 : if last_key.is_none() || last_key.as_ref() == Some(&key) {
3152 560 : if last_key.is_none() {
3153 104 : last_key = Some(key);
3154 456 : }
3155 560 : accumulated_values.push((key, lsn, val));
3156 : } else {
3157 1140 : let last_key: &mut Key = last_key.as_mut().unwrap();
3158 1140 : stat.on_unique_key_visited(); // TODO: adjust statistics for partial compaction
3159 1140 : let retention = self
3160 1140 : .generate_key_retention(
3161 1140 : *last_key,
3162 1140 : &accumulated_values,
3163 1140 : job_desc.gc_cutoff,
3164 1140 : &job_desc.retain_lsns_below_horizon,
3165 1140 : COMPACTION_DELTA_THRESHOLD,
3166 1140 : get_ancestor_image(self, *last_key, ctx, has_data_below, lowest_retain_lsn)
3167 1140 : .await
3168 1140 : .context("failed to get ancestor image")
3169 1140 : .map_err(CompactionError::Other)?,
3170 : )
3171 1140 : .await
3172 1140 : .context("failed to generate key retention")
3173 1140 : .map_err(CompactionError::Other)?;
3174 1140 : retention
3175 1140 : .pipe_to(
3176 1140 : *last_key,
3177 1140 : &mut delta_layer_writer,
3178 1140 : image_layer_writer.as_mut(),
3179 1140 : &mut stat,
3180 1140 : ctx,
3181 1140 : )
3182 1140 : .await
3183 1140 : .context("failed to pipe to delta layer writer")
3184 1140 : .map_err(CompactionError::Other)?;
3185 1140 : accumulated_values.clear();
3186 1140 : *last_key = key;
3187 1140 : accumulated_values.push((key, lsn, val));
3188 : }
3189 : }
3190 :
3191 : // TODO: move the below part to the loop body
3192 104 : let last_key = last_key.expect("no keys produced during compaction");
3193 104 : stat.on_unique_key_visited();
3194 :
3195 104 : let retention = self
3196 104 : .generate_key_retention(
3197 104 : last_key,
3198 104 : &accumulated_values,
3199 104 : job_desc.gc_cutoff,
3200 104 : &job_desc.retain_lsns_below_horizon,
3201 104 : COMPACTION_DELTA_THRESHOLD,
3202 104 : get_ancestor_image(self, last_key, ctx, has_data_below, lowest_retain_lsn)
3203 104 : .await
3204 104 : .context("failed to get ancestor image")
3205 104 : .map_err(CompactionError::Other)?,
3206 : )
3207 104 : .await
3208 104 : .context("failed to generate key retention")
3209 104 : .map_err(CompactionError::Other)?;
3210 104 : retention
3211 104 : .pipe_to(
3212 104 : last_key,
3213 104 : &mut delta_layer_writer,
3214 104 : image_layer_writer.as_mut(),
3215 104 : &mut stat,
3216 104 : ctx,
3217 104 : )
3218 104 : .await
3219 104 : .context("failed to pipe to delta layer writer")
3220 104 : .map_err(CompactionError::Other)?;
3221 : // end: move the above part to the loop body
3222 :
3223 104 : let time_main_loop = timer.elapsed();
3224 104 : let timer = Instant::now();
3225 104 :
3226 104 : let mut rewrote_delta_layers = Vec::new();
3227 108 : for (key, writers) in delta_layer_rewriters {
3228 4 : if let Some(delta_writer_before) = writers.before {
3229 0 : let (desc, path) = delta_writer_before
3230 0 : .finish(job_desc.compaction_key_range.start, ctx)
3231 0 : .await
3232 0 : .context("failed to finish delta layer writer")
3233 0 : .map_err(CompactionError::Other)?;
3234 0 : let layer = Layer::finish_creating(self.conf, self, desc, &path)
3235 0 : .context("failed to finish creating delta layer")
3236 0 : .map_err(CompactionError::Other)?;
3237 0 : rewrote_delta_layers.push(layer);
3238 4 : }
3239 4 : if let Some(delta_writer_after) = writers.after {
3240 4 : let (desc, path) = delta_writer_after
3241 4 : .finish(key.key_range.end, ctx)
3242 4 : .await
3243 4 : .context("failed to finish delta layer writer")
3244 4 : .map_err(CompactionError::Other)?;
3245 4 : let layer = Layer::finish_creating(self.conf, self, desc, &path)
3246 4 : .context("failed to finish creating delta layer")
3247 4 : .map_err(CompactionError::Other)?;
3248 4 : rewrote_delta_layers.push(layer);
3249 0 : }
3250 : }
3251 :
3252 148 : let discard = |key: &PersistentLayerKey| {
3253 148 : let key = key.clone();
3254 148 : async move { KeyHistoryRetention::discard_key(&key, self, dry_run).await }
3255 148 : };
3256 :
3257 104 : let produced_image_layers = if let Some(writer) = image_layer_writer {
3258 84 : if !dry_run {
3259 76 : let end_key = job_desc.compaction_key_range.end;
3260 76 : writer
3261 76 : .finish_with_discard_fn(self, ctx, end_key, discard)
3262 76 : .await
3263 76 : .context("failed to finish image layer writer")
3264 76 : .map_err(CompactionError::Other)?
3265 : } else {
3266 8 : drop(writer);
3267 8 : Vec::new()
3268 : }
3269 : } else {
3270 20 : Vec::new()
3271 : };
3272 :
3273 104 : let produced_delta_layers = if !dry_run {
3274 96 : delta_layer_writer
3275 96 : .finish_with_discard_fn(self, ctx, discard)
3276 96 : .await
3277 96 : .context("failed to finish delta layer writer")
3278 96 : .map_err(CompactionError::Other)?
3279 : } else {
3280 8 : drop(delta_layer_writer);
3281 8 : Vec::new()
3282 : };
3283 :
3284 : // TODO: make image/delta/rewrote_delta layers generation atomic. At this point, we already generated resident layers, and if
3285 : // compaction is cancelled at this point, we might have some layers that are not cleaned up.
3286 104 : let mut compact_to = Vec::new();
3287 104 : let mut keep_layers = HashSet::new();
3288 104 : let produced_delta_layers_len = produced_delta_layers.len();
3289 104 : let produced_image_layers_len = produced_image_layers.len();
3290 104 :
3291 104 : let layer_selection_by_key = job_desc
3292 104 : .selected_layers
3293 104 : .iter()
3294 304 : .map(|l| (l.layer_desc().key(), l.layer_desc().clone()))
3295 104 : .collect::<HashMap<_, _>>();
3296 :
3297 176 : for action in produced_delta_layers {
3298 72 : match action {
3299 44 : BatchWriterResult::Produced(layer) => {
3300 44 : if cfg!(debug_assertions) {
3301 44 : info!("produced delta layer: {}", layer.layer_desc().key());
3302 0 : }
3303 44 : stat.produce_delta_layer(layer.layer_desc().file_size());
3304 44 : compact_to.push(layer);
3305 : }
3306 28 : BatchWriterResult::Discarded(l) => {
3307 28 : if cfg!(debug_assertions) {
3308 28 : info!("discarded delta layer: {}", l);
3309 0 : }
3310 28 : if let Some(layer_desc) = layer_selection_by_key.get(&l) {
3311 28 : stat.discard_delta_layer(layer_desc.file_size());
3312 28 : } else {
3313 0 : tracing::warn!(
3314 0 : "discarded delta layer not in layer_selection: {}, produced a layer outside of the compaction key range?",
3315 : l
3316 : );
3317 0 : stat.discard_delta_layer(0);
3318 : }
3319 28 : keep_layers.insert(l);
3320 : }
3321 : }
3322 : }
3323 108 : for layer in &rewrote_delta_layers {
3324 4 : debug!(
3325 0 : "produced rewritten delta layer: {}",
3326 0 : layer.layer_desc().key()
3327 : );
3328 : // For now, we include rewritten delta layer size in the "produce_delta_layer". We could
3329 : // make it a separate statistics in the future.
3330 4 : stat.produce_delta_layer(layer.layer_desc().file_size());
3331 : }
3332 104 : compact_to.extend(rewrote_delta_layers);
3333 180 : for action in produced_image_layers {
3334 76 : match action {
3335 60 : BatchWriterResult::Produced(layer) => {
3336 60 : debug!("produced image layer: {}", layer.layer_desc().key());
3337 60 : stat.produce_image_layer(layer.layer_desc().file_size());
3338 60 : compact_to.push(layer);
3339 : }
3340 16 : BatchWriterResult::Discarded(l) => {
3341 16 : debug!("discarded image layer: {}", l);
3342 16 : if let Some(layer_desc) = layer_selection_by_key.get(&l) {
3343 16 : stat.discard_image_layer(layer_desc.file_size());
3344 16 : } else {
3345 0 : tracing::warn!(
3346 0 : "discarded image layer not in layer_selection: {}, produced a layer outside of the compaction key range?",
3347 : l
3348 : );
3349 0 : stat.discard_image_layer(0);
3350 : }
3351 16 : keep_layers.insert(l);
3352 : }
3353 : }
3354 : }
3355 :
3356 104 : let mut layer_selection = job_desc.selected_layers;
3357 :
3358 : // Partial compaction might select more data than it processes, e.g., if
3359 : // the compaction_key_range only partially overlaps:
3360 : //
3361 : // [---compaction_key_range---]
3362 : // [---A----][----B----][----C----][----D----]
3363 : //
3364 : // For delta layers, we will rewrite the layers so that it is cut exactly at
3365 : // the compaction key range, so we can always discard them. However, for image
3366 : // layers, as we do not rewrite them for now, we need to handle them differently.
3367 : // Assume image layers A, B, C, D are all in the `layer_selection`.
3368 : //
3369 : // The created image layers contain whatever is needed from B, C, and from
3370 : // `----]` of A, and from `[---` of D.
3371 : //
3372 : // In contrast, `[---A` and `D----]` have not been processed, so, we must
3373 : // keep that data.
3374 : //
3375 : // The solution for now is to keep A and D completely if they are image layers.
3376 : // (layer_selection is what we'll remove from the layer map, so, retain what
3377 : // is _not_ fully covered by compaction_key_range).
3378 408 : for layer in &layer_selection {
3379 304 : if !layer.layer_desc().is_delta() {
3380 132 : if !overlaps_with(
3381 132 : &layer.layer_desc().key_range,
3382 132 : &job_desc.compaction_key_range,
3383 132 : ) {
3384 0 : return Err(CompactionError::Other(anyhow!(
3385 0 : "violated constraint: image layer outside of compaction key range"
3386 0 : )));
3387 132 : }
3388 132 : if !fully_contains(
3389 132 : &job_desc.compaction_key_range,
3390 132 : &layer.layer_desc().key_range,
3391 132 : ) {
3392 16 : keep_layers.insert(layer.layer_desc().key());
3393 116 : }
3394 172 : }
3395 : }
3396 :
3397 304 : layer_selection.retain(|x| !keep_layers.contains(&x.layer_desc().key()));
3398 104 :
3399 104 : let time_final_phase = timer.elapsed();
3400 104 :
3401 104 : stat.time_final_phase_secs = time_final_phase.as_secs_f64();
3402 104 : stat.time_main_loop_secs = time_main_loop.as_secs_f64();
3403 104 : stat.time_acquire_lock_secs = time_acquire_lock.as_secs_f64();
3404 104 : stat.time_download_layer_secs = time_download_layer.as_secs_f64();
3405 104 : stat.time_analyze_secs = time_analyze.as_secs_f64();
3406 104 : stat.time_total_secs = begin_timer.elapsed().as_secs_f64();
3407 104 : stat.finalize();
3408 104 :
3409 104 : info!(
3410 0 : "gc-compaction statistics: {}",
3411 0 : serde_json::to_string(&stat)
3412 0 : .context("failed to serialize gc-compaction statistics")
3413 0 : .map_err(CompactionError::Other)?
3414 : );
3415 :
3416 104 : if dry_run {
3417 8 : return Ok(CompactionOutcome::Done);
3418 96 : }
3419 96 :
3420 96 : info!(
3421 0 : "produced {} delta layers and {} image layers, {} layers are kept",
3422 0 : produced_delta_layers_len,
3423 0 : produced_image_layers_len,
3424 0 : keep_layers.len()
3425 : );
3426 :
3427 : // Step 3: Place back to the layer map.
3428 :
3429 : // First, do a sanity check to ensure the newly-created layer map does not contain overlaps.
3430 96 : let all_layers = {
3431 96 : let guard = self.layers.read().await;
3432 96 : let layer_map = guard.layer_map()?;
3433 96 : layer_map.iter_historic_layers().collect_vec()
3434 96 : };
3435 96 :
3436 96 : let mut final_layers = all_layers
3437 96 : .iter()
3438 428 : .map(|layer| layer.layer_name())
3439 96 : .collect::<HashSet<_>>();
3440 304 : for layer in &layer_selection {
3441 208 : final_layers.remove(&layer.layer_desc().layer_name());
3442 208 : }
3443 204 : for layer in &compact_to {
3444 108 : final_layers.insert(layer.layer_desc().layer_name());
3445 108 : }
3446 96 : let final_layers = final_layers.into_iter().collect_vec();
3447 :
3448 : // TODO: move this check before we call `finish` on image layer writers. However, this will require us to get the layer name before we finish
3449 : // the writer, so potentially, we will need a function like `ImageLayerBatchWriter::get_all_pending_layer_keys` to get all the keys that are
3450 : // in the writer before finalizing the persistent layers. Now we would leave some dangling layers on the disk if the check fails.
3451 96 : if let Some(err) = check_valid_layermap(&final_layers) {
3452 0 : return Err(CompactionError::Other(anyhow!(
3453 0 : "gc-compaction layer map check failed after compaction because {}, compaction result not applied to the layer map due to potential data loss",
3454 0 : err
3455 0 : )));
3456 96 : }
3457 :
3458 : // Between the sanity check and this compaction update, there could be new layers being flushed, but it should be fine because we only
3459 : // operate on L1 layers.
3460 : {
3461 : // Gc-compaction will rewrite the history of a key. This could happen in two ways:
3462 : //
3463 : // 1. We create an image layer to replace all the deltas below the compact LSN. In this case, assume
3464 : // we have 2 delta layers A and B, both below the compact LSN. We create an image layer I to replace
3465 : // A and B at the compact LSN. If the read path finishes reading A, yields, and now we update the layer
3466 : // map, the read path then cannot find any keys below A, reporting a missing key error, while the key
3467 : // now gets stored in I at the compact LSN.
3468 : //
3469 : // --------------- ---------------
3470 : // delta1@LSN20 image1@LSN20
3471 : // --------------- (read path collects delta@LSN20, => --------------- (read path cannot find anything
3472 : // delta1@LSN10 yields) below LSN 20)
3473 : // ---------------
3474 : //
3475 : // 2. We create a delta layer to replace all the deltas below the compact LSN, and in the delta layers,
3476 : // we combines the history of a key into a single image. For example, we have deltas at LSN 1, 2, 3, 4,
3477 : // Assume one delta layer contains LSN 1, 2, 3 and the other contains LSN 4.
3478 : //
3479 : // We let gc-compaction combine delta 2, 3, 4 into an image at LSN 4, which produces a delta layer that
3480 : // contains the delta at LSN 1, the image at LSN 4. If the read path finishes reading the original delta
3481 : // layer containing 4, yields, and we update the layer map to put the delta layer.
3482 : //
3483 : // --------------- ---------------
3484 : // delta1@LSN4 image1@LSN4
3485 : // --------------- (read path collects delta@LSN4, => --------------- (read path collects LSN4 and LSN1,
3486 : // delta1@LSN1-3 yields) delta1@LSN1 which is an invalid history)
3487 : // --------------- ---------------
3488 : //
3489 : // Therefore, the gc-compaction layer update operation should wait for all ongoing reads, block all pending reads,
3490 : // and only allow reads to continue after the update is finished.
3491 :
3492 96 : let update_guard = self.gc_compaction_layer_update_lock.write().await;
3493 : // Acquiring the update guard ensures current read operations end and new read operations are blocked.
3494 : // TODO: can we use `latest_gc_cutoff` Rcu to achieve the same effect?
3495 96 : let mut guard = self.layers.write().await;
3496 96 : guard
3497 96 : .open_mut()?
3498 96 : .finish_gc_compaction(&layer_selection, &compact_to, &self.metrics);
3499 96 : drop(update_guard); // Allow new reads to start ONLY after we finished updating the layer map.
3500 96 : };
3501 96 :
3502 96 : // Schedule an index-only upload to update the `latest_gc_cutoff` in the index_part.json.
3503 96 : // Otherwise, after restart, the index_part only contains the old `latest_gc_cutoff` and
3504 96 : // find_gc_cutoffs will try accessing things below the cutoff. TODO: ideally, this should
3505 96 : // be batched into `schedule_compaction_update`.
3506 96 : let disk_consistent_lsn = self.disk_consistent_lsn.load();
3507 96 : self.schedule_uploads(disk_consistent_lsn, None)
3508 96 : .context("failed to schedule uploads")
3509 96 : .map_err(CompactionError::Other)?;
3510 : // If a layer gets rewritten throughout gc-compaction, we need to keep that layer only in `compact_to` instead
3511 : // of `compact_from`.
3512 96 : let compact_from = {
3513 96 : let mut compact_from = Vec::new();
3514 96 : let mut compact_to_set = HashMap::new();
3515 204 : for layer in &compact_to {
3516 108 : compact_to_set.insert(layer.layer_desc().key(), layer);
3517 108 : }
3518 304 : for layer in &layer_selection {
3519 208 : if let Some(to) = compact_to_set.get(&layer.layer_desc().key()) {
3520 0 : tracing::info!(
3521 0 : "skipping delete {} because found same layer key at different generation {}",
3522 : layer,
3523 : to
3524 : );
3525 208 : } else {
3526 208 : compact_from.push(layer.clone());
3527 208 : }
3528 : }
3529 96 : compact_from
3530 96 : };
3531 96 : self.remote_client
3532 96 : .schedule_compaction_update(&compact_from, &compact_to)?;
3533 :
3534 96 : drop(gc_lock);
3535 96 :
3536 96 : Ok(CompactionOutcome::Done)
3537 108 : }
3538 : }
3539 :
3540 : struct TimelineAdaptor {
3541 : timeline: Arc<Timeline>,
3542 :
3543 : keyspace: (Lsn, KeySpace),
3544 :
3545 : new_deltas: Vec<ResidentLayer>,
3546 : new_images: Vec<ResidentLayer>,
3547 : layers_to_delete: Vec<Arc<PersistentLayerDesc>>,
3548 : }
3549 :
3550 : impl TimelineAdaptor {
3551 0 : pub fn new(timeline: &Arc<Timeline>, keyspace: (Lsn, KeySpace)) -> Self {
3552 0 : Self {
3553 0 : timeline: timeline.clone(),
3554 0 : keyspace,
3555 0 : new_images: Vec::new(),
3556 0 : new_deltas: Vec::new(),
3557 0 : layers_to_delete: Vec::new(),
3558 0 : }
3559 0 : }
3560 :
3561 0 : pub async fn flush_updates(&mut self) -> Result<(), CompactionError> {
3562 0 : let layers_to_delete = {
3563 0 : let guard = self.timeline.layers.read().await;
3564 0 : self.layers_to_delete
3565 0 : .iter()
3566 0 : .map(|x| guard.get_from_desc(x))
3567 0 : .collect::<Vec<Layer>>()
3568 0 : };
3569 0 : self.timeline
3570 0 : .finish_compact_batch(&self.new_deltas, &self.new_images, &layers_to_delete)
3571 0 : .await?;
3572 :
3573 0 : self.timeline
3574 0 : .upload_new_image_layers(std::mem::take(&mut self.new_images))?;
3575 :
3576 0 : self.new_deltas.clear();
3577 0 : self.layers_to_delete.clear();
3578 0 : Ok(())
3579 0 : }
3580 : }
3581 :
3582 : #[derive(Clone)]
3583 : struct ResidentDeltaLayer(ResidentLayer);
3584 : #[derive(Clone)]
3585 : struct ResidentImageLayer(ResidentLayer);
3586 :
3587 : impl CompactionJobExecutor for TimelineAdaptor {
3588 : type Key = pageserver_api::key::Key;
3589 :
3590 : type Layer = OwnArc<PersistentLayerDesc>;
3591 : type DeltaLayer = ResidentDeltaLayer;
3592 : type ImageLayer = ResidentImageLayer;
3593 :
3594 : type RequestContext = crate::context::RequestContext;
3595 :
3596 0 : fn get_shard_identity(&self) -> &ShardIdentity {
3597 0 : self.timeline.get_shard_identity()
3598 0 : }
3599 :
3600 0 : async fn get_layers(
3601 0 : &mut self,
3602 0 : key_range: &Range<Key>,
3603 0 : lsn_range: &Range<Lsn>,
3604 0 : _ctx: &RequestContext,
3605 0 : ) -> anyhow::Result<Vec<OwnArc<PersistentLayerDesc>>> {
3606 0 : self.flush_updates().await?;
3607 :
3608 0 : let guard = self.timeline.layers.read().await;
3609 0 : let layer_map = guard.layer_map()?;
3610 :
3611 0 : let result = layer_map
3612 0 : .iter_historic_layers()
3613 0 : .filter(|l| {
3614 0 : overlaps_with(&l.lsn_range, lsn_range) && overlaps_with(&l.key_range, key_range)
3615 0 : })
3616 0 : .map(OwnArc)
3617 0 : .collect();
3618 0 : Ok(result)
3619 0 : }
3620 :
3621 0 : async fn get_keyspace(
3622 0 : &mut self,
3623 0 : key_range: &Range<Key>,
3624 0 : lsn: Lsn,
3625 0 : _ctx: &RequestContext,
3626 0 : ) -> anyhow::Result<Vec<Range<Key>>> {
3627 0 : if lsn == self.keyspace.0 {
3628 0 : Ok(pageserver_compaction::helpers::intersect_keyspace(
3629 0 : &self.keyspace.1.ranges,
3630 0 : key_range,
3631 0 : ))
3632 : } else {
3633 : // The current compaction implementation only ever requests the key space
3634 : // at the compaction end LSN.
3635 0 : anyhow::bail!("keyspace not available for requested lsn");
3636 : }
3637 0 : }
3638 :
3639 0 : async fn downcast_delta_layer(
3640 0 : &self,
3641 0 : layer: &OwnArc<PersistentLayerDesc>,
3642 0 : ctx: &RequestContext,
3643 0 : ) -> anyhow::Result<Option<ResidentDeltaLayer>> {
3644 0 : // this is a lot more complex than a simple downcast...
3645 0 : if layer.is_delta() {
3646 0 : let l = {
3647 0 : let guard = self.timeline.layers.read().await;
3648 0 : guard.get_from_desc(layer)
3649 : };
3650 0 : let result = l.download_and_keep_resident(ctx).await?;
3651 :
3652 0 : Ok(Some(ResidentDeltaLayer(result)))
3653 : } else {
3654 0 : Ok(None)
3655 : }
3656 0 : }
3657 :
3658 0 : async fn create_image(
3659 0 : &mut self,
3660 0 : lsn: Lsn,
3661 0 : key_range: &Range<Key>,
3662 0 : ctx: &RequestContext,
3663 0 : ) -> anyhow::Result<()> {
3664 0 : Ok(self.create_image_impl(lsn, key_range, ctx).await?)
3665 0 : }
3666 :
3667 0 : async fn create_delta(
3668 0 : &mut self,
3669 0 : lsn_range: &Range<Lsn>,
3670 0 : key_range: &Range<Key>,
3671 0 : input_layers: &[ResidentDeltaLayer],
3672 0 : ctx: &RequestContext,
3673 0 : ) -> anyhow::Result<()> {
3674 0 : debug!("Create new layer {}..{}", lsn_range.start, lsn_range.end);
3675 :
3676 0 : let mut all_entries = Vec::new();
3677 0 : for dl in input_layers.iter() {
3678 0 : all_entries.extend(dl.load_keys(ctx).await?);
3679 : }
3680 :
3681 : // The current stdlib sorting implementation is designed in a way where it is
3682 : // particularly fast where the slice is made up of sorted sub-ranges.
3683 0 : all_entries.sort_by_key(|DeltaEntry { key, lsn, .. }| (*key, *lsn));
3684 :
3685 0 : let mut writer = DeltaLayerWriter::new(
3686 0 : self.timeline.conf,
3687 0 : self.timeline.timeline_id,
3688 0 : self.timeline.tenant_shard_id,
3689 0 : key_range.start,
3690 0 : lsn_range.clone(),
3691 0 : ctx,
3692 0 : )
3693 0 : .await?;
3694 :
3695 0 : let mut dup_values = 0;
3696 0 :
3697 0 : // This iterator walks through all key-value pairs from all the layers
3698 0 : // we're compacting, in key, LSN order.
3699 0 : let mut prev: Option<(Key, Lsn)> = None;
3700 : for &DeltaEntry {
3701 0 : key, lsn, ref val, ..
3702 0 : } in all_entries.iter()
3703 : {
3704 0 : if prev == Some((key, lsn)) {
3705 : // This is a duplicate. Skip it.
3706 : //
3707 : // It can happen if compaction is interrupted after writing some
3708 : // layers but not all, and we are compacting the range again.
3709 : // The calculations in the algorithm assume that there are no
3710 : // duplicates, so the math on targeted file size is likely off,
3711 : // and we will create smaller files than expected.
3712 0 : dup_values += 1;
3713 0 : continue;
3714 0 : }
3715 :
3716 0 : let value = val.load(ctx).await?;
3717 :
3718 0 : writer.put_value(key, lsn, value, ctx).await?;
3719 :
3720 0 : prev = Some((key, lsn));
3721 : }
3722 :
3723 0 : if dup_values > 0 {
3724 0 : warn!("delta layer created with {} duplicate values", dup_values);
3725 0 : }
3726 :
3727 0 : fail_point!("delta-layer-writer-fail-before-finish", |_| {
3728 0 : Err(anyhow::anyhow!(
3729 0 : "failpoint delta-layer-writer-fail-before-finish"
3730 0 : ))
3731 0 : });
3732 :
3733 0 : let (desc, path) = writer.finish(prev.unwrap().0.next(), ctx).await?;
3734 0 : let new_delta_layer =
3735 0 : Layer::finish_creating(self.timeline.conf, &self.timeline, desc, &path)?;
3736 :
3737 0 : self.new_deltas.push(new_delta_layer);
3738 0 : Ok(())
3739 0 : }
3740 :
3741 0 : async fn delete_layer(
3742 0 : &mut self,
3743 0 : layer: &OwnArc<PersistentLayerDesc>,
3744 0 : _ctx: &RequestContext,
3745 0 : ) -> anyhow::Result<()> {
3746 0 : self.layers_to_delete.push(layer.clone().0);
3747 0 : Ok(())
3748 0 : }
3749 : }
3750 :
3751 : impl TimelineAdaptor {
3752 0 : async fn create_image_impl(
3753 0 : &mut self,
3754 0 : lsn: Lsn,
3755 0 : key_range: &Range<Key>,
3756 0 : ctx: &RequestContext,
3757 0 : ) -> Result<(), CreateImageLayersError> {
3758 0 : let timer = self.timeline.metrics.create_images_time_histo.start_timer();
3759 :
3760 0 : let image_layer_writer = ImageLayerWriter::new(
3761 0 : self.timeline.conf,
3762 0 : self.timeline.timeline_id,
3763 0 : self.timeline.tenant_shard_id,
3764 0 : key_range,
3765 0 : lsn,
3766 0 : ctx,
3767 0 : )
3768 0 : .await?;
3769 :
3770 0 : fail_point!("image-layer-writer-fail-before-finish", |_| {
3771 0 : Err(CreateImageLayersError::Other(anyhow::anyhow!(
3772 0 : "failpoint image-layer-writer-fail-before-finish"
3773 0 : )))
3774 0 : });
3775 :
3776 0 : let keyspace = KeySpace {
3777 0 : ranges: self.get_keyspace(key_range, lsn, ctx).await?,
3778 : };
3779 : // TODO set proper (stateful) start. The create_image_layer_for_rel_blocks function mostly
3780 0 : let outcome = self
3781 0 : .timeline
3782 0 : .create_image_layer_for_rel_blocks(
3783 0 : &keyspace,
3784 0 : image_layer_writer,
3785 0 : lsn,
3786 0 : ctx,
3787 0 : key_range.clone(),
3788 0 : IoConcurrency::sequential(),
3789 0 : )
3790 0 : .await?;
3791 :
3792 : if let ImageLayerCreationOutcome::Generated {
3793 0 : unfinished_image_layer,
3794 0 : } = outcome
3795 : {
3796 0 : let (desc, path) = unfinished_image_layer.finish(ctx).await?;
3797 0 : let image_layer =
3798 0 : Layer::finish_creating(self.timeline.conf, &self.timeline, desc, &path)?;
3799 0 : self.new_images.push(image_layer);
3800 0 : }
3801 :
3802 0 : timer.stop_and_record();
3803 0 :
3804 0 : Ok(())
3805 0 : }
3806 : }
3807 :
3808 : impl CompactionRequestContext for crate::context::RequestContext {}
3809 :
3810 : #[derive(Debug, Clone)]
3811 : pub struct OwnArc<T>(pub Arc<T>);
3812 :
3813 : impl<T> Deref for OwnArc<T> {
3814 : type Target = <Arc<T> as Deref>::Target;
3815 0 : fn deref(&self) -> &Self::Target {
3816 0 : &self.0
3817 0 : }
3818 : }
3819 :
3820 : impl<T> AsRef<T> for OwnArc<T> {
3821 0 : fn as_ref(&self) -> &T {
3822 0 : self.0.as_ref()
3823 0 : }
3824 : }
3825 :
3826 : impl CompactionLayer<Key> for OwnArc<PersistentLayerDesc> {
3827 0 : fn key_range(&self) -> &Range<Key> {
3828 0 : &self.key_range
3829 0 : }
3830 0 : fn lsn_range(&self) -> &Range<Lsn> {
3831 0 : &self.lsn_range
3832 0 : }
3833 0 : fn file_size(&self) -> u64 {
3834 0 : self.file_size
3835 0 : }
3836 0 : fn short_id(&self) -> std::string::String {
3837 0 : self.as_ref().short_id().to_string()
3838 0 : }
3839 0 : fn is_delta(&self) -> bool {
3840 0 : self.as_ref().is_delta()
3841 0 : }
3842 : }
3843 :
3844 : impl CompactionLayer<Key> for OwnArc<DeltaLayer> {
3845 0 : fn key_range(&self) -> &Range<Key> {
3846 0 : &self.layer_desc().key_range
3847 0 : }
3848 0 : fn lsn_range(&self) -> &Range<Lsn> {
3849 0 : &self.layer_desc().lsn_range
3850 0 : }
3851 0 : fn file_size(&self) -> u64 {
3852 0 : self.layer_desc().file_size
3853 0 : }
3854 0 : fn short_id(&self) -> std::string::String {
3855 0 : self.layer_desc().short_id().to_string()
3856 0 : }
3857 0 : fn is_delta(&self) -> bool {
3858 0 : true
3859 0 : }
3860 : }
3861 :
3862 : use crate::tenant::timeline::DeltaEntry;
3863 :
3864 : impl CompactionLayer<Key> for ResidentDeltaLayer {
3865 0 : fn key_range(&self) -> &Range<Key> {
3866 0 : &self.0.layer_desc().key_range
3867 0 : }
3868 0 : fn lsn_range(&self) -> &Range<Lsn> {
3869 0 : &self.0.layer_desc().lsn_range
3870 0 : }
3871 0 : fn file_size(&self) -> u64 {
3872 0 : self.0.layer_desc().file_size
3873 0 : }
3874 0 : fn short_id(&self) -> std::string::String {
3875 0 : self.0.layer_desc().short_id().to_string()
3876 0 : }
3877 0 : fn is_delta(&self) -> bool {
3878 0 : true
3879 0 : }
3880 : }
3881 :
3882 : impl CompactionDeltaLayer<TimelineAdaptor> for ResidentDeltaLayer {
3883 : type DeltaEntry<'a> = DeltaEntry<'a>;
3884 :
3885 0 : async fn load_keys(&self, ctx: &RequestContext) -> anyhow::Result<Vec<DeltaEntry<'_>>> {
3886 0 : self.0.get_as_delta(ctx).await?.index_entries(ctx).await
3887 0 : }
3888 : }
3889 :
3890 : impl CompactionLayer<Key> for ResidentImageLayer {
3891 0 : fn key_range(&self) -> &Range<Key> {
3892 0 : &self.0.layer_desc().key_range
3893 0 : }
3894 0 : fn lsn_range(&self) -> &Range<Lsn> {
3895 0 : &self.0.layer_desc().lsn_range
3896 0 : }
3897 0 : fn file_size(&self) -> u64 {
3898 0 : self.0.layer_desc().file_size
3899 0 : }
3900 0 : fn short_id(&self) -> std::string::String {
3901 0 : self.0.layer_desc().short_id().to_string()
3902 0 : }
3903 0 : fn is_delta(&self) -> bool {
3904 0 : false
3905 0 : }
3906 : }
3907 : impl CompactionImageLayer<TimelineAdaptor> for ResidentImageLayer {}
|