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