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