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