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