-
Notifications
You must be signed in to change notification settings - Fork 490
/
Copy pathcompaction.rs
3612 lines (3340 loc) · 152 KB
/
compaction.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
//! New compaction implementation. The algorithm itself is implemented in the
//! compaction crate. This file implements the callbacks and structs that allow
//! the algorithm to drive the process.
//!
//! The old legacy algorithm is implemented directly in `timeline.rs`.
use std::collections::{BinaryHeap, HashMap, HashSet, VecDeque};
use std::ops::{Deref, Range};
use std::sync::Arc;
use super::layer_manager::LayerManager;
use super::{
CompactFlags, CompactOptions, CreateImageLayersError, DurationRecorder, GetVectoredError,
ImageLayerCreationMode, LastImageLayerCreationStatus, RecordedDuration, Timeline,
};
use anyhow::{anyhow, bail, Context};
use bytes::Bytes;
use enumset::EnumSet;
use fail::fail_point;
use itertools::Itertools;
use once_cell::sync::Lazy;
use pageserver_api::key::KEY_SIZE;
use pageserver_api::keyspace::ShardedRange;
use pageserver_api::models::CompactInfoResponse;
use pageserver_api::shard::{ShardCount, ShardIdentity, TenantShardId};
use serde::Serialize;
use tokio::sync::{OwnedSemaphorePermit, Semaphore};
use tokio_util::sync::CancellationToken;
use tracing::{debug, info, info_span, trace, warn, Instrument};
use utils::critical;
use utils::id::TimelineId;
use crate::context::{AccessStatsBehavior, RequestContext, RequestContextBuilder};
use crate::page_cache;
use crate::statvfs::Statvfs;
use crate::tenant::checks::check_valid_layermap;
use crate::tenant::gc_block::GcBlock;
use crate::tenant::layer_map::LayerMap;
use crate::tenant::remote_timeline_client::WaitCompletionError;
use crate::tenant::storage_layer::batch_split_writer::{
BatchWriterResult, SplitDeltaLayerWriter, SplitImageLayerWriter,
};
use crate::tenant::storage_layer::filter_iterator::FilterIterator;
use crate::tenant::storage_layer::merge_iterator::MergeIterator;
use crate::tenant::storage_layer::{
AsLayerDesc, PersistentLayerDesc, PersistentLayerKey, ValueReconstructState,
};
use crate::tenant::timeline::{drop_rlock, DeltaLayerWriter, ImageLayerWriter};
use crate::tenant::timeline::{ImageLayerCreationOutcome, IoConcurrency};
use crate::tenant::timeline::{Layer, ResidentLayer};
use crate::tenant::{gc_block, DeltaLayer, MaybeOffloaded};
use crate::virtual_file::{MaybeFatalIo, VirtualFile};
use pageserver_api::config::tenant_conf_defaults::DEFAULT_CHECKPOINT_DISTANCE;
use pageserver_api::key::Key;
use pageserver_api::keyspace::KeySpace;
use pageserver_api::record::NeonWalRecord;
use pageserver_api::value::Value;
use utils::lsn::Lsn;
use pageserver_compaction::helpers::{fully_contains, overlaps_with};
use pageserver_compaction::interface::*;
use super::CompactionError;
/// Maximum number of deltas before generating an image layer in bottom-most compaction.
const COMPACTION_DELTA_THRESHOLD: usize = 5;
#[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)]
pub struct GcCompactionJobId(pub usize);
impl std::fmt::Display for GcCompactionJobId {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "{}", self.0)
}
}
#[derive(Debug, Clone)]
pub enum GcCompactionQueueItem {
MetaJob {
/// Compaction options
options: CompactOptions,
/// Whether the compaction is triggered automatically (determines whether we need to update L2 LSN)
auto: bool,
},
SubCompactionJob(CompactOptions),
Notify(GcCompactionJobId, Option<Lsn>),
}
impl GcCompactionQueueItem {
pub fn into_compact_info_resp(
self,
id: GcCompactionJobId,
running: bool,
) -> Option<CompactInfoResponse> {
match self {
GcCompactionQueueItem::MetaJob { options, .. } => Some(CompactInfoResponse {
compact_key_range: options.compact_key_range,
compact_lsn_range: options.compact_lsn_range,
sub_compaction: options.sub_compaction,
running,
job_id: id.0,
}),
GcCompactionQueueItem::SubCompactionJob(options) => Some(CompactInfoResponse {
compact_key_range: options.compact_key_range,
compact_lsn_range: options.compact_lsn_range,
sub_compaction: options.sub_compaction,
running,
job_id: id.0,
}),
GcCompactionQueueItem::Notify(_, _) => None,
}
}
}
#[derive(Default)]
struct GcCompactionGuardItems {
notify: Option<tokio::sync::oneshot::Sender<()>>,
gc_guard: Option<gc_block::Guard>,
permit: Option<OwnedSemaphorePermit>,
}
struct GcCompactionQueueInner {
running: Option<(GcCompactionJobId, GcCompactionQueueItem)>,
queued: VecDeque<(GcCompactionJobId, GcCompactionQueueItem)>,
guards: HashMap<GcCompactionJobId, GcCompactionGuardItems>,
last_id: GcCompactionJobId,
}
impl GcCompactionQueueInner {
fn next_id(&mut self) -> GcCompactionJobId {
let id = self.last_id;
self.last_id = GcCompactionJobId(id.0 + 1);
id
}
}
/// A structure to store gc_compaction jobs.
pub struct GcCompactionQueue {
/// All items in the queue, and the currently-running job.
inner: std::sync::Mutex<GcCompactionQueueInner>,
/// Ensure only one thread is consuming the queue.
consumer_lock: tokio::sync::Mutex<()>,
}
static CONCURRENT_GC_COMPACTION_TASKS: Lazy<Arc<Semaphore>> = Lazy::new(|| {
// Only allow two timelines on one pageserver to run gc compaction at a time.
Arc::new(Semaphore::new(2))
});
impl GcCompactionQueue {
pub fn new() -> Self {
GcCompactionQueue {
inner: std::sync::Mutex::new(GcCompactionQueueInner {
running: None,
queued: VecDeque::new(),
guards: HashMap::new(),
last_id: GcCompactionJobId(0),
}),
consumer_lock: tokio::sync::Mutex::new(()),
}
}
pub fn cancel_scheduled(&self) {
let mut guard = self.inner.lock().unwrap();
guard.queued.clear();
// TODO: if there is a running job, we should keep the gc guard. However, currently, the cancel
// API is only used for testing purposes, so we can drop everything here.
guard.guards.clear();
}
/// Schedule a manual compaction job.
pub fn schedule_manual_compaction(
&self,
options: CompactOptions,
notify: Option<tokio::sync::oneshot::Sender<()>>,
) -> GcCompactionJobId {
let mut guard = self.inner.lock().unwrap();
let id = guard.next_id();
guard.queued.push_back((
id,
GcCompactionQueueItem::MetaJob {
options,
auto: false,
},
));
guard.guards.entry(id).or_default().notify = notify;
info!("scheduled compaction job id={}", id);
id
}
/// Schedule an auto compaction job.
fn schedule_auto_compaction(
&self,
options: CompactOptions,
permit: OwnedSemaphorePermit,
) -> GcCompactionJobId {
let mut guard = self.inner.lock().unwrap();
let id = guard.next_id();
guard.queued.push_back((
id,
GcCompactionQueueItem::MetaJob {
options,
auto: true,
},
));
guard.guards.entry(id).or_default().permit = Some(permit);
id
}
/// Trigger an auto compaction.
pub async fn trigger_auto_compaction(&self, timeline: &Arc<Timeline>) {
let (
gc_compaction_enabled,
gc_compaction_initial_threshold_kb,
gc_compaction_ratio_percent,
) = timeline.get_gc_compaction_settings();
if !gc_compaction_enabled {
return;
}
if self.remaining_jobs_num() > 0 {
// Only schedule auto compaction when the queue is empty
return;
}
let Ok(permit) = CONCURRENT_GC_COMPACTION_TASKS.clone().try_acquire_owned() else {
// Only allow one compaction run at a time
return;
};
let l2_lsn = timeline.get_l2_lsn();
let layers = {
let guard = timeline.layers.read().await;
let layer_map = guard.layer_map().unwrap();
layer_map.iter_historic_layers().collect_vec()
};
let mut l2_size: u64 = 0;
let mut l1_size = 0;
let gc_cutoff = *timeline.get_latest_gc_cutoff_lsn();
for layer in layers {
if layer.lsn_range.start <= l2_lsn {
l2_size += layer.file_size();
} else if layer.lsn_range.start <= gc_cutoff {
l1_size += layer.file_size();
}
}
fn trigger_compaction(
l1_size: u64,
l2_size: u64,
gc_compaction_initial_threshold_kb: u64,
gc_compaction_ratio_percent: u64,
) -> bool {
const AUTO_TRIGGER_LIMIT: u64 = 150 * 1024 * 1024 * 1024; // 150GB
if l1_size >= AUTO_TRIGGER_LIMIT || l2_size >= AUTO_TRIGGER_LIMIT {
// Do not auto-trigger when physical size >= 150GB
return false;
}
// initial trigger
if l2_size == 0 && l1_size >= gc_compaction_initial_threshold_kb * 1024 {
info!(
"trigger auto-compaction because l1_size={} >= gc_compaction_initial_threshold_kb={}",
l1_size,
gc_compaction_initial_threshold_kb
);
return true;
}
// size ratio trigger
if l2_size == 0 {
return false;
}
if l1_size as f64 / l2_size as f64 >= (gc_compaction_ratio_percent as f64 / 100.0) {
info!(
"trigger auto-compaction because l1_size={} / l2_size={} > gc_compaction_ratio_percent={}",
l1_size,
l2_size,
gc_compaction_ratio_percent
);
return true;
}
false
}
if trigger_compaction(
l1_size,
l2_size,
gc_compaction_initial_threshold_kb,
gc_compaction_ratio_percent,
) {
self.schedule_auto_compaction(
CompactOptions {
flags: {
let mut flags = EnumSet::new();
flags |= CompactFlags::EnhancedGcBottomMostCompaction;
flags
},
sub_compaction: true,
compact_key_range: None,
compact_lsn_range: None,
sub_compaction_max_job_size_mb: None,
},
permit,
);
info!(
"scheduled auto gc-compaction: l1_size={}, l2_size={}, l2_lsn={}, gc_cutoff={}",
l1_size, l2_size, l2_lsn, gc_cutoff
);
} else {
info!(
"did not trigger auto gc-compaction: l1_size={}, l2_size={}, l2_lsn={}, gc_cutoff={}",
l1_size, l2_size, l2_lsn, gc_cutoff
);
}
}
/// Notify the caller the job has finished and unblock GC.
fn notify_and_unblock(&self, id: GcCompactionJobId) {
info!("compaction job id={} finished", id);
let mut guard = self.inner.lock().unwrap();
if let Some(items) = guard.guards.remove(&id) {
drop(items.gc_guard);
if let Some(tx) = items.notify {
let _ = tx.send(());
}
}
}
async fn handle_sub_compaction(
&self,
id: GcCompactionJobId,
options: CompactOptions,
timeline: &Arc<Timeline>,
gc_block: &GcBlock,
auto: bool,
) -> Result<(), CompactionError> {
info!("running scheduled enhanced gc bottom-most compaction with sub-compaction, splitting compaction jobs");
let jobs = timeline
.gc_compaction_split_jobs(
GcCompactJob::from_compact_options(options.clone()),
options.sub_compaction_max_job_size_mb,
)
.await
.map_err(CompactionError::Other)?;
if jobs.is_empty() {
info!("no jobs to run, skipping scheduled compaction task");
self.notify_and_unblock(id);
} else {
let gc_guard = match gc_block.start().await {
Ok(guard) => guard,
Err(e) => {
return Err(CompactionError::Other(anyhow!(
"cannot run gc-compaction because gc is blocked: {}",
e
)));
}
};
let jobs_len = jobs.len();
let mut pending_tasks = Vec::new();
// gc-compaction might pick more layers or fewer layers to compact. The L2 LSN does not need to be accurate.
// And therefore, we simply assume the maximum LSN of all jobs is the expected L2 LSN.
let expected_l2_lsn = jobs.iter().map(|job| job.compact_lsn_range.end).max();
for job in jobs {
// Unfortunately we need to convert the `GcCompactJob` back to `CompactionOptions`
// until we do further refactors to allow directly call `compact_with_gc`.
let mut flags: EnumSet<CompactFlags> = EnumSet::default();
flags |= CompactFlags::EnhancedGcBottomMostCompaction;
if job.dry_run {
flags |= CompactFlags::DryRun;
}
let options = CompactOptions {
flags,
sub_compaction: false,
compact_key_range: Some(job.compact_key_range.into()),
compact_lsn_range: Some(job.compact_lsn_range.into()),
sub_compaction_max_job_size_mb: None,
};
pending_tasks.push(GcCompactionQueueItem::SubCompactionJob(options));
}
if !auto {
pending_tasks.push(GcCompactionQueueItem::Notify(id, None));
} else {
pending_tasks.push(GcCompactionQueueItem::Notify(id, expected_l2_lsn));
}
{
let mut guard = self.inner.lock().unwrap();
guard.guards.entry(id).or_default().gc_guard = Some(gc_guard);
let mut tasks = Vec::new();
for task in pending_tasks {
let id = guard.next_id();
tasks.push((id, task));
}
tasks.reverse();
for item in tasks {
guard.queued.push_front(item);
}
}
info!("scheduled enhanced gc bottom-most compaction with sub-compaction, split into {} jobs", jobs_len);
}
Ok(())
}
/// Take a job from the queue and process it. Returns if there are still pending tasks.
pub async fn iteration(
&self,
cancel: &CancellationToken,
ctx: &RequestContext,
gc_block: &GcBlock,
timeline: &Arc<Timeline>,
) -> Result<CompactionOutcome, CompactionError> {
let _one_op_at_a_time_guard = self.consumer_lock.lock().await;
let has_pending_tasks;
let Some((id, item)) = ({
let mut guard = self.inner.lock().unwrap();
if let Some((id, item)) = guard.queued.pop_front() {
guard.running = Some((id, item.clone()));
has_pending_tasks = !guard.queued.is_empty();
Some((id, item))
} else {
has_pending_tasks = false;
None
}
}) else {
self.trigger_auto_compaction(timeline).await;
// Always yield after triggering auto-compaction
return Ok(CompactionOutcome::Done);
};
match item {
GcCompactionQueueItem::MetaJob { options, auto } => {
if !options
.flags
.contains(CompactFlags::EnhancedGcBottomMostCompaction)
{
warn!("ignoring scheduled compaction task: scheduled task must be gc compaction: {:?}", options);
} else if options.sub_compaction {
info!("running scheduled enhanced gc bottom-most compaction with sub-compaction, splitting compaction jobs");
self.handle_sub_compaction(id, options, timeline, gc_block, auto)
.await?;
} else {
// Auto compaction always enables sub-compaction so we don't need to handle update_l2_lsn
// in this branch.
let gc_guard = match gc_block.start().await {
Ok(guard) => guard,
Err(e) => {
return Err(CompactionError::Other(anyhow!(
"cannot run gc-compaction because gc is blocked: {}",
e
)));
}
};
{
let mut guard = self.inner.lock().unwrap();
guard.guards.entry(id).or_default().gc_guard = Some(gc_guard);
}
let _ = timeline
.compact_with_options(cancel, options, ctx)
.instrument(info_span!("scheduled_compact_timeline", %timeline.timeline_id))
.await?;
self.notify_and_unblock(id);
}
}
GcCompactionQueueItem::SubCompactionJob(options) => {
// TODO: error handling, clear the queue if any task fails?
let _ = timeline
.compact_with_options(cancel, options, ctx)
.instrument(info_span!("scheduled_compact_timeline", %timeline.timeline_id))
.await?;
}
GcCompactionQueueItem::Notify(id, l2_lsn) => {
self.notify_and_unblock(id);
if let Some(l2_lsn) = l2_lsn {
if l2_lsn >= timeline.get_l2_lsn() {
info!("l2_lsn updated to {}", l2_lsn);
timeline
.update_l2_lsn(l2_lsn)
.map_err(CompactionError::Other)?;
}
}
}
}
{
let mut guard = self.inner.lock().unwrap();
guard.running = None;
}
Ok(if has_pending_tasks {
CompactionOutcome::Pending
} else {
CompactionOutcome::Done
})
}
#[allow(clippy::type_complexity)]
pub fn remaining_jobs(
&self,
) -> (
Option<(GcCompactionJobId, GcCompactionQueueItem)>,
VecDeque<(GcCompactionJobId, GcCompactionQueueItem)>,
) {
let guard = self.inner.lock().unwrap();
(guard.running.clone(), guard.queued.clone())
}
pub fn remaining_jobs_num(&self) -> usize {
let guard = self.inner.lock().unwrap();
guard.queued.len() + if guard.running.is_some() { 1 } else { 0 }
}
}
/// A job description for the gc-compaction job. This structure describes the rectangle range that the job will
/// process. The exact layers that need to be compacted/rewritten will be generated when `compact_with_gc` gets
/// called.
#[derive(Debug, Clone)]
pub(crate) struct GcCompactJob {
pub dry_run: bool,
/// The key range to be compacted. The compaction algorithm will only regenerate key-value pairs within this range
/// [left inclusive, right exclusive), and other pairs will be rewritten into new files if necessary.
pub compact_key_range: Range<Key>,
/// The LSN range to be compacted. The compaction algorithm will use this range to determine the layers to be
/// selected for the compaction, and it does not guarantee the generated layers will have exactly the same LSN range
/// as specified here. The true range being compacted is `min_lsn/max_lsn` in [`GcCompactionJobDescription`].
/// min_lsn will always <= the lower bound specified here, and max_lsn will always >= the upper bound specified here.
pub compact_lsn_range: Range<Lsn>,
}
impl GcCompactJob {
pub fn from_compact_options(options: CompactOptions) -> Self {
GcCompactJob {
dry_run: options.flags.contains(CompactFlags::DryRun),
compact_key_range: options
.compact_key_range
.map(|x| x.into())
.unwrap_or(Key::MIN..Key::MAX),
compact_lsn_range: options
.compact_lsn_range
.map(|x| x.into())
.unwrap_or(Lsn::INVALID..Lsn::MAX),
}
}
}
/// A job description for the gc-compaction job. This structure is generated when `compact_with_gc` is called
/// and contains the exact layers we want to compact.
pub struct GcCompactionJobDescription {
/// All layers to read in the compaction job
selected_layers: Vec<Layer>,
/// GC cutoff of the job. This is the lowest LSN that will be accessed by the read/GC path and we need to
/// keep all deltas <= this LSN or generate an image == this LSN.
gc_cutoff: Lsn,
/// LSNs to retain for the job. Read path will use this LSN so we need to keep deltas <= this LSN or
/// generate an image == this LSN.
retain_lsns_below_horizon: Vec<Lsn>,
/// Maximum layer LSN processed in this compaction, that is max(end_lsn of layers). Exclusive. All data
/// \>= this LSN will be kept and will not be rewritten.
max_layer_lsn: Lsn,
/// Minimum layer LSN processed in this compaction, that is min(start_lsn of layers). Inclusive.
/// All access below (strict lower than `<`) this LSN will be routed through the normal read path instead of
/// k-merge within gc-compaction.
min_layer_lsn: Lsn,
/// Only compact layers overlapping with this range.
compaction_key_range: Range<Key>,
/// When partial compaction is enabled, these layers need to be rewritten to ensure no overlap.
/// This field is here solely for debugging. The field will not be read once the compaction
/// description is generated.
rewrite_layers: Vec<Arc<PersistentLayerDesc>>,
}
/// The result of bottom-most compaction for a single key at each LSN.
#[derive(Debug)]
#[cfg_attr(test, derive(PartialEq))]
pub struct KeyLogAtLsn(pub Vec<(Lsn, Value)>);
/// The result of bottom-most compaction.
#[derive(Debug)]
#[cfg_attr(test, derive(PartialEq))]
pub(crate) struct KeyHistoryRetention {
/// Stores logs to reconstruct the value at the given LSN, that is to say, logs <= LSN or image == LSN.
pub(crate) below_horizon: Vec<(Lsn, KeyLogAtLsn)>,
/// Stores logs to reconstruct the value at any LSN above the horizon, that is to say, log > LSN.
pub(crate) above_horizon: KeyLogAtLsn,
}
impl KeyHistoryRetention {
/// Hack: skip delta layer if we need to produce a layer of a same key-lsn.
///
/// This can happen if we have removed some deltas in "the middle" of some existing layer's key-lsn-range.
/// For example, consider the case where a single delta with range [0x10,0x50) exists.
/// And we have branches at LSN 0x10, 0x20, 0x30.
/// Then we delete branch @ 0x20.
/// Bottom-most compaction may now delete the delta [0x20,0x30).
/// And that wouldnt' change the shape of the layer.
///
/// Note that bottom-most-gc-compaction never _adds_ new data in that case, only removes.
///
/// `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.
async fn discard_key(key: &PersistentLayerKey, tline: &Arc<Timeline>, dry_run: bool) -> bool {
if dry_run {
return true;
}
if LayerMap::is_l0(&key.key_range, key.is_delta) {
// gc-compaction should not produce L0 deltas, otherwise it will break the layer order.
// We should ignore such layers.
return true;
}
let layer_generation;
{
let guard = tline.layers.read().await;
if !guard.contains_key(key) {
return false;
}
layer_generation = guard.get_from_key(key).metadata().generation;
}
if layer_generation == tline.generation {
info!(
key=%key,
?layer_generation,
"discard layer due to duplicated layer key in the same generation",
);
true
} else {
false
}
}
/// Pipe a history of a single key to the writers.
///
/// If `image_writer` is none, the images will be placed into the delta layers.
/// The delta writer will contain all images and deltas (below and above the horizon) except the bottom-most images.
#[allow(clippy::too_many_arguments)]
async fn pipe_to(
self,
key: Key,
delta_writer: &mut SplitDeltaLayerWriter,
mut image_writer: Option<&mut SplitImageLayerWriter>,
stat: &mut CompactionStatistics,
ctx: &RequestContext,
) -> anyhow::Result<()> {
let mut first_batch = true;
for (cutoff_lsn, KeyLogAtLsn(logs)) in self.below_horizon {
if first_batch {
if logs.len() == 1 && logs[0].1.is_image() {
let Value::Image(img) = &logs[0].1 else {
unreachable!()
};
stat.produce_image_key(img);
if let Some(image_writer) = image_writer.as_mut() {
image_writer.put_image(key, img.clone(), ctx).await?;
} else {
delta_writer
.put_value(key, cutoff_lsn, Value::Image(img.clone()), ctx)
.await?;
}
} else {
for (lsn, val) in logs {
stat.produce_key(&val);
delta_writer.put_value(key, lsn, val, ctx).await?;
}
}
first_batch = false;
} else {
for (lsn, val) in logs {
stat.produce_key(&val);
delta_writer.put_value(key, lsn, val, ctx).await?;
}
}
}
let KeyLogAtLsn(above_horizon_logs) = self.above_horizon;
for (lsn, val) in above_horizon_logs {
stat.produce_key(&val);
delta_writer.put_value(key, lsn, val, ctx).await?;
}
Ok(())
}
}
#[derive(Debug, Serialize, Default)]
struct CompactionStatisticsNumSize {
num: u64,
size: u64,
}
#[derive(Debug, Serialize, Default)]
pub struct CompactionStatistics {
delta_layer_visited: CompactionStatisticsNumSize,
image_layer_visited: CompactionStatisticsNumSize,
delta_layer_produced: CompactionStatisticsNumSize,
image_layer_produced: CompactionStatisticsNumSize,
num_delta_layer_discarded: usize,
num_image_layer_discarded: usize,
num_unique_keys_visited: usize,
wal_keys_visited: CompactionStatisticsNumSize,
image_keys_visited: CompactionStatisticsNumSize,
wal_produced: CompactionStatisticsNumSize,
image_produced: CompactionStatisticsNumSize,
}
impl CompactionStatistics {
fn estimated_size_of_value(val: &Value) -> usize {
match val {
Value::Image(img) => img.len(),
Value::WalRecord(NeonWalRecord::Postgres { rec, .. }) => rec.len(),
_ => std::mem::size_of::<NeonWalRecord>(),
}
}
fn estimated_size_of_key() -> usize {
KEY_SIZE // TODO: distinguish image layer and delta layer (count LSN in delta layer)
}
fn visit_delta_layer(&mut self, size: u64) {
self.delta_layer_visited.num += 1;
self.delta_layer_visited.size += size;
}
fn visit_image_layer(&mut self, size: u64) {
self.image_layer_visited.num += 1;
self.image_layer_visited.size += size;
}
fn on_unique_key_visited(&mut self) {
self.num_unique_keys_visited += 1;
}
fn visit_wal_key(&mut self, val: &Value) {
self.wal_keys_visited.num += 1;
self.wal_keys_visited.size +=
Self::estimated_size_of_value(val) as u64 + Self::estimated_size_of_key() as u64;
}
fn visit_image_key(&mut self, val: &Value) {
self.image_keys_visited.num += 1;
self.image_keys_visited.size +=
Self::estimated_size_of_value(val) as u64 + Self::estimated_size_of_key() as u64;
}
fn produce_key(&mut self, val: &Value) {
match val {
Value::Image(img) => self.produce_image_key(img),
Value::WalRecord(_) => self.produce_wal_key(val),
}
}
fn produce_wal_key(&mut self, val: &Value) {
self.wal_produced.num += 1;
self.wal_produced.size +=
Self::estimated_size_of_value(val) as u64 + Self::estimated_size_of_key() as u64;
}
fn produce_image_key(&mut self, val: &Bytes) {
self.image_produced.num += 1;
self.image_produced.size += val.len() as u64 + Self::estimated_size_of_key() as u64;
}
fn discard_delta_layer(&mut self) {
self.num_delta_layer_discarded += 1;
}
fn discard_image_layer(&mut self) {
self.num_image_layer_discarded += 1;
}
fn produce_delta_layer(&mut self, size: u64) {
self.delta_layer_produced.num += 1;
self.delta_layer_produced.size += size;
}
fn produce_image_layer(&mut self, size: u64) {
self.image_layer_produced.num += 1;
self.image_layer_produced.size += size;
}
}
#[derive(Default, Debug, Clone, Copy, PartialEq, Eq)]
pub enum CompactionOutcome {
#[default]
/// No layers need to be compacted after this round. Compaction doesn't need
/// to be immediately scheduled.
Done,
/// Still has pending layers to be compacted after this round. Ideally, the scheduler
/// should immediately schedule another compaction.
Pending,
/// A timeline needs L0 compaction. Yield and schedule an immediate L0 compaction pass (only
/// guaranteed when `compaction_l0_first` is enabled).
YieldForL0,
/// Compaction was skipped, because the timeline is ineligible for compaction.
Skipped,
}
impl Timeline {
/// TODO: cancellation
///
/// Returns whether the compaction has pending tasks.
pub(crate) async fn compact_legacy(
self: &Arc<Self>,
cancel: &CancellationToken,
options: CompactOptions,
ctx: &RequestContext,
) -> Result<CompactionOutcome, CompactionError> {
if options
.flags
.contains(CompactFlags::EnhancedGcBottomMostCompaction)
{
self.compact_with_gc(cancel, options, ctx)
.await
.map_err(CompactionError::Other)?;
return Ok(CompactionOutcome::Done);
}
if options.flags.contains(CompactFlags::DryRun) {
return Err(CompactionError::Other(anyhow!(
"dry-run mode is not supported for legacy compaction for now"
)));
}
if options.compact_key_range.is_some() || options.compact_lsn_range.is_some() {
// maybe useful in the future? could implement this at some point
return Err(CompactionError::Other(anyhow!(
"compaction range is not supported for legacy compaction for now"
)));
}
// High level strategy for compaction / image creation:
//
// 1. First, do a L0 compaction to ensure we move the L0
// layers into the historic layer map get flat levels of
// layers. If we did not compact all L0 layers, we will
// prioritize compacting the timeline again and not do
// any of the compactions below.
//
// 2. Then, calculate the desired "partitioning" of the
// currently in-use key space. The goal is to partition the
// key space into roughly fixed-size chunks, but also take into
// account any existing image layers, and try to align the
// chunk boundaries with the existing image layers to avoid
// too much churn. Also try to align chunk boundaries with
// relation boundaries. In principle, we don't know about
// relation boundaries here, we just deal with key-value
// pairs, and the code in pgdatadir_mapping.rs knows how to
// map relations into key-value pairs. But in practice we know
// that 'field6' is the block number, and the fields 1-5
// identify a relation. This is just an optimization,
// though.
//
// 3. Once we know the partitioning, for each partition,
// decide if it's time to create a new image layer. The
// criteria is: there has been too much "churn" since the last
// image layer? The "churn" is fuzzy concept, it's a
// combination of too many delta files, or too much WAL in
// total in the delta file. Or perhaps: if creating an image
// file would allow to delete some older files.
//
// 4. In the end, if the tenant gets auto-sharded, we will run
// a shard-ancestor compaction.
// Is the timeline being deleted?
if self.is_stopping() {
trace!("Dropping out of compaction on timeline shutdown");
return Err(CompactionError::ShuttingDown);
}
let target_file_size = self.get_checkpoint_distance();
// Define partitioning schema if needed
let l0_l1_boundary_lsn = {
// We do the repartition on the L0-L1 boundary. All data below the boundary
// are compacted by L0 with low read amplification, thus making the `repartition`
// function run fast.
let guard = self.layers.read().await;
let l0_min_lsn = guard
.layer_map()?
.level0_deltas()
.iter()
.map(|l| l.get_lsn_range().start)
.min()
.unwrap_or(self.get_disk_consistent_lsn());
l0_min_lsn.max(self.get_ancestor_lsn())
};
// 1. L0 Compact
let l0_outcome = {
let timer = self.metrics.compact_time_histo.start_timer();
let l0_outcome = self
.compact_level0(
target_file_size,
options.flags.contains(CompactFlags::ForceL0Compaction),
ctx,
)
.await?;
timer.stop_and_record();
l0_outcome
};
if options.flags.contains(CompactFlags::OnlyL0Compaction) {
return Ok(l0_outcome);
}
// Yield if we have pending L0 compaction. The scheduler will do another pass.
if (l0_outcome == CompactionOutcome::Pending || l0_outcome == CompactionOutcome::YieldForL0)
&& !options.flags.contains(CompactFlags::NoYield)
{
info!("image/ancestor compaction yielding for L0 compaction");
return Ok(CompactionOutcome::YieldForL0);
}
if l0_l1_boundary_lsn < self.partitioning.read().1 {
// We never go backwards when repartition and create image layers.
info!("skipping image layer generation because repartition LSN is greater than L0-L1 boundary LSN.");
} else {
// 2. Repartition and create image layers if necessary
match self
.repartition(
l0_l1_boundary_lsn,
self.get_compaction_target_size(),
options.flags,
ctx,
)
.await
{
Ok(((dense_partitioning, sparse_partitioning), lsn)) => {
// Disables access_stats updates, so that the files we read remain candidates for eviction after we're done with them
let image_ctx = RequestContextBuilder::extend(ctx)
.access_stats_behavior(AccessStatsBehavior::Skip)
.build();
let mut partitioning = dense_partitioning;
partitioning
.parts
.extend(sparse_partitioning.into_dense().parts);
// 3. Create new image layers for partitions that have been modified "enough".
let (image_layers, outcome) = self
.create_image_layers(
&partitioning,
lsn,
if options
.flags
.contains(CompactFlags::ForceImageLayerCreation)
{
ImageLayerCreationMode::Force
} else {
ImageLayerCreationMode::Try
},
&image_ctx,
self.last_image_layer_creation_status
.load()
.as_ref()
.clone(),
!options.flags.contains(CompactFlags::NoYield),
)
.await
.inspect_err(|err| {
if let CreateImageLayersError::GetVectoredError(
GetVectoredError::MissingKey(_),
) = err
{
critical!("missing key during compaction: {err:?}");
}
})?;
self.last_image_layer_creation_status
.store(Arc::new(outcome.clone()));
self.upload_new_image_layers(image_layers)?;
if let LastImageLayerCreationStatus::Incomplete { .. } = outcome {
// Yield and do not do any other kind of compaction.
info!("skipping shard ancestor compaction due to pending image layer generation tasks (preempted by L0 compaction).");
return Ok(CompactionOutcome::YieldForL0);
}
}
Err(err) => {
// no partitioning? This is normal, if the timeline was just created
// as an empty timeline. Also in unit tests, when we use the timeline
// as a simple key-value store, ignoring the datadir layout. Log the
// error but continue.
//
// Suppress error when it's due to cancellation
if !self.cancel.is_cancelled() && !err.is_cancelled() {
tracing::error!(
"could not compact, repartitioning keyspace failed: {err:?}"
);
}
}
};
}
let partition_count = self.partitioning.read().0 .0.parts.len();
// 4. Shard ancestor compaction
if self.shard_identity.count >= ShardCount::new(2) {
// Limit the number of layer rewrites to the number of partitions: this means its
// runtime should be comparable to a full round of image layer creations, rather than
// being potentially much longer.
let rewrite_max = partition_count;
self.compact_shard_ancestors(rewrite_max, ctx).await?;
}
Ok(CompactionOutcome::Done)
}
/// Check for layers that are elegible to be rewritten:
/// - Shard splitting: After a shard split, ancestor layers beyond pitr_interval, so that
/// we don't indefinitely retain keys in this shard that aren't needed.
/// - For future use: layers beyond pitr_interval that are in formats we would
/// rather not maintain compatibility with indefinitely.
///
/// Note: this phase may read and write many gigabytes of data: use rewrite_max to bound