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::ops::{Deref, Range};
8 : use std::sync::Arc;
9 :
10 : use super::Timeline;
11 :
12 : use async_trait::async_trait;
13 : use fail::fail_point;
14 : use tokio_util::sync::CancellationToken;
15 : use tracing::{debug, trace, warn};
16 :
17 : use crate::context::RequestContext;
18 : use crate::tenant::storage_layer::{AsLayerDesc, PersistentLayerDesc};
19 : use crate::tenant::timeline::{is_rel_fsm_block_key, is_rel_vm_block_key};
20 : use crate::tenant::timeline::{DeltaLayerWriter, ImageLayerWriter};
21 : use crate::tenant::timeline::{Layer, ResidentLayer};
22 : use crate::tenant::DeltaLayer;
23 : use crate::tenant::PageReconstructError;
24 : use crate::ZERO_PAGE;
25 :
26 : use crate::keyspace::KeySpace;
27 : use crate::repository::Key;
28 :
29 : use utils::lsn::Lsn;
30 :
31 : use pageserver_compaction::helpers::overlaps_with;
32 : use pageserver_compaction::interface::*;
33 :
34 : use super::CompactionError;
35 :
36 : impl Timeline {
37 : /// Entry point for new tiered compaction algorithm.
38 : ///
39 : /// All the real work is in the implementation in the pageserver_compaction
40 : /// crate. The code here would apply to any algorithm implemented by the
41 : /// same interface, but tiered is the only one at the moment.
42 : ///
43 : /// TODO: cancellation
44 0 : pub(crate) async fn compact_tiered(
45 0 : self: &Arc<Self>,
46 0 : _cancel: &CancellationToken,
47 0 : ctx: &RequestContext,
48 0 : ) -> Result<(), CompactionError> {
49 0 : let fanout = self.get_compaction_threshold() as u64;
50 0 : let target_file_size = self.get_checkpoint_distance();
51 :
52 : // Find the top of the historical layers
53 0 : let end_lsn = {
54 0 : let guard = self.layers.read().await;
55 0 : let layers = guard.layer_map();
56 :
57 0 : let l0_deltas = layers.get_level0_deltas()?;
58 0 : drop(guard);
59 0 :
60 0 : // As an optimization, if we find that there are too few L0 layers,
61 0 : // bail out early. We know that the compaction algorithm would do
62 0 : // nothing in that case.
63 0 : if l0_deltas.len() < fanout as usize {
64 : // doesn't need compacting
65 0 : return Ok(());
66 0 : }
67 0 : l0_deltas.iter().map(|l| l.lsn_range.end).max().unwrap()
68 0 : };
69 0 :
70 0 : // Is the timeline being deleted?
71 0 : if self.is_stopping() {
72 0 : trace!("Dropping out of compaction on timeline shutdown");
73 0 : return Err(CompactionError::ShuttingDown);
74 0 : }
75 :
76 0 : let keyspace = self.collect_keyspace(end_lsn, ctx).await?;
77 0 : let mut adaptor = TimelineAdaptor::new(self, (end_lsn, keyspace));
78 0 : let ctx_adaptor = RequestContextAdaptor(ctx.clone());
79 0 :
80 0 : pageserver_compaction::compact_tiered::compact_tiered(
81 0 : &mut adaptor,
82 0 : end_lsn,
83 0 : target_file_size,
84 0 : fanout,
85 0 : &ctx_adaptor,
86 0 : )
87 0 : .await?;
88 :
89 0 : adaptor.flush_updates().await?;
90 0 : Ok(())
91 0 : }
92 : }
93 :
94 : struct TimelineAdaptor {
95 : timeline: Arc<Timeline>,
96 :
97 : keyspace: (Lsn, KeySpace),
98 :
99 : new_deltas: Vec<ResidentLayer>,
100 : new_images: Vec<ResidentLayer>,
101 : layers_to_delete: Vec<Arc<PersistentLayerDesc>>,
102 : }
103 :
104 : impl TimelineAdaptor {
105 0 : pub fn new(timeline: &Arc<Timeline>, keyspace: (Lsn, KeySpace)) -> Self {
106 0 : Self {
107 0 : timeline: timeline.clone(),
108 0 : keyspace,
109 0 : new_images: Vec::new(),
110 0 : new_deltas: Vec::new(),
111 0 : layers_to_delete: Vec::new(),
112 0 : }
113 0 : }
114 :
115 0 : pub async fn flush_updates(&mut self) -> anyhow::Result<()> {
116 0 : let layers_to_delete = {
117 0 : let guard = self.timeline.layers.read().await;
118 0 : self.layers_to_delete
119 0 : .iter()
120 0 : .map(|x| guard.get_from_desc(x))
121 0 : .collect::<Vec<Layer>>()
122 0 : };
123 0 : self.timeline
124 0 : .finish_compact_batch(&self.new_deltas, &self.new_images, &layers_to_delete)
125 0 : .await?;
126 0 : self.new_images.clear();
127 0 : self.new_deltas.clear();
128 0 : self.layers_to_delete.clear();
129 0 : Ok(())
130 0 : }
131 : }
132 :
133 0 : #[derive(Clone)]
134 : struct ResidentDeltaLayer(ResidentLayer);
135 0 : #[derive(Clone)]
136 : struct ResidentImageLayer(ResidentLayer);
137 :
138 : #[async_trait]
139 : impl CompactionJobExecutor for TimelineAdaptor {
140 : type Key = crate::repository::Key;
141 :
142 : type Layer = OwnArc<PersistentLayerDesc>;
143 : type DeltaLayer = ResidentDeltaLayer;
144 : type ImageLayer = ResidentImageLayer;
145 :
146 : type RequestContext = RequestContextAdaptor;
147 :
148 0 : async fn get_layers(
149 0 : &mut self,
150 0 : key_range: &Range<Key>,
151 0 : lsn_range: &Range<Lsn>,
152 0 : _ctx: &RequestContextAdaptor,
153 0 : ) -> anyhow::Result<Vec<OwnArc<PersistentLayerDesc>>> {
154 0 : self.flush_updates().await?;
155 :
156 0 : let guard = self.timeline.layers.read().await;
157 0 : let layer_map = guard.layer_map();
158 0 :
159 0 : let result = layer_map
160 0 : .iter_historic_layers()
161 0 : .filter(|l| {
162 0 : overlaps_with(&l.lsn_range, lsn_range) && overlaps_with(&l.key_range, key_range)
163 0 : })
164 0 : .map(OwnArc)
165 0 : .collect();
166 0 : Ok(result)
167 0 : }
168 :
169 0 : async fn get_keyspace(
170 0 : &mut self,
171 0 : key_range: &Range<Key>,
172 0 : lsn: Lsn,
173 0 : _ctx: &RequestContextAdaptor,
174 0 : ) -> anyhow::Result<Vec<Range<Key>>> {
175 0 : if lsn == self.keyspace.0 {
176 0 : Ok(pageserver_compaction::helpers::intersect_keyspace(
177 0 : &self.keyspace.1.ranges,
178 0 : key_range,
179 0 : ))
180 : } else {
181 : // The current compaction implementatin only ever requests the key space
182 : // at the compaction end LSN.
183 0 : anyhow::bail!("keyspace not available for requested lsn");
184 : }
185 0 : }
186 :
187 0 : async fn downcast_delta_layer(
188 0 : &self,
189 0 : layer: &OwnArc<PersistentLayerDesc>,
190 0 : ) -> anyhow::Result<Option<ResidentDeltaLayer>> {
191 : // this is a lot more complex than a simple downcast...
192 0 : if layer.is_delta() {
193 0 : let l = {
194 0 : let guard = self.timeline.layers.read().await;
195 0 : guard.get_from_desc(layer)
196 : };
197 0 : let result = l.download_and_keep_resident().await?;
198 :
199 0 : Ok(Some(ResidentDeltaLayer(result)))
200 : } else {
201 0 : Ok(None)
202 : }
203 0 : }
204 :
205 0 : async fn create_image(
206 0 : &mut self,
207 0 : lsn: Lsn,
208 0 : key_range: &Range<Key>,
209 0 : ctx: &RequestContextAdaptor,
210 0 : ) -> anyhow::Result<()> {
211 0 : Ok(self.create_image_impl(lsn, key_range, ctx).await?)
212 0 : }
213 :
214 0 : async fn create_delta(
215 0 : &mut self,
216 0 : lsn_range: &Range<Lsn>,
217 0 : key_range: &Range<Key>,
218 0 : input_layers: &[ResidentDeltaLayer],
219 0 : ctx: &RequestContextAdaptor,
220 0 : ) -> anyhow::Result<()> {
221 0 : debug!("Create new layer {}..{}", lsn_range.start, lsn_range.end);
222 :
223 0 : let mut all_entries = Vec::new();
224 0 : for dl in input_layers.iter() {
225 0 : all_entries.extend(dl.load_keys(ctx).await?);
226 : }
227 :
228 : // The current stdlib sorting implementation is designed in a way where it is
229 : // particularly fast where the slice is made up of sorted sub-ranges.
230 0 : all_entries.sort_by_key(|DeltaEntry { key, lsn, .. }| (*key, *lsn));
231 :
232 0 : let mut writer = DeltaLayerWriter::new(
233 0 : self.timeline.conf,
234 0 : self.timeline.timeline_id,
235 0 : self.timeline.tenant_shard_id,
236 0 : key_range.start,
237 0 : lsn_range.clone(),
238 0 : )
239 0 : .await?;
240 :
241 0 : let mut dup_values = 0;
242 0 :
243 0 : // This iterator walks through all key-value pairs from all the layers
244 0 : // we're compacting, in key, LSN order.
245 0 : let mut prev: Option<(Key, Lsn)> = None;
246 : for &DeltaEntry {
247 0 : key, lsn, ref val, ..
248 0 : } in all_entries.iter()
249 : {
250 0 : if prev == Some((key, lsn)) {
251 : // This is a duplicate. Skip it.
252 : //
253 : // It can happen if compaction is interrupted after writing some
254 : // layers but not all, and we are compacting the range again.
255 : // The calculations in the algorithm assume that there are no
256 : // duplicates, so the math on targeted file size is likely off,
257 : // and we will create smaller files than expected.
258 0 : dup_values += 1;
259 0 : continue;
260 0 : }
261 :
262 0 : let value = val.load(ctx).await?;
263 :
264 0 : writer.put_value(key, lsn, value).await?;
265 :
266 0 : prev = Some((key, lsn));
267 : }
268 :
269 0 : if dup_values > 0 {
270 0 : warn!("delta layer created with {} duplicate values", dup_values);
271 0 : }
272 :
273 0 : fail_point!("delta-layer-writer-fail-before-finish", |_| {
274 0 : Err(anyhow::anyhow!(
275 0 : "failpoint delta-layer-writer-fail-before-finish"
276 0 : ))
277 0 : });
278 :
279 0 : let new_delta_layer = writer
280 0 : .finish(prev.unwrap().0.next(), &self.timeline)
281 0 : .await?;
282 :
283 0 : self.new_deltas.push(new_delta_layer);
284 0 : Ok(())
285 0 : }
286 :
287 0 : async fn delete_layer(
288 0 : &mut self,
289 0 : layer: &OwnArc<PersistentLayerDesc>,
290 0 : _ctx: &RequestContextAdaptor,
291 0 : ) -> anyhow::Result<()> {
292 0 : self.layers_to_delete.push(layer.clone().0);
293 0 : Ok(())
294 0 : }
295 : }
296 :
297 : impl TimelineAdaptor {
298 0 : async fn create_image_impl(
299 0 : &mut self,
300 0 : lsn: Lsn,
301 0 : key_range: &Range<Key>,
302 0 : ctx: &RequestContextAdaptor,
303 0 : ) -> Result<(), PageReconstructError> {
304 0 : let timer = self.timeline.metrics.create_images_time_histo.start_timer();
305 :
306 0 : let mut image_layer_writer = ImageLayerWriter::new(
307 0 : self.timeline.conf,
308 0 : self.timeline.timeline_id,
309 0 : self.timeline.tenant_shard_id,
310 0 : key_range,
311 0 : lsn,
312 0 : )
313 0 : .await?;
314 :
315 0 : fail_point!("image-layer-writer-fail-before-finish", |_| {
316 0 : Err(PageReconstructError::Other(anyhow::anyhow!(
317 0 : "failpoint image-layer-writer-fail-before-finish"
318 0 : )))
319 0 : });
320 0 : let keyspace_ranges = self.get_keyspace(key_range, lsn, ctx).await?;
321 0 : for range in &keyspace_ranges {
322 0 : let mut key = range.start;
323 0 : while key < range.end {
324 0 : let img = match self.timeline.get(key, lsn, ctx).await {
325 0 : Ok(img) => img,
326 0 : Err(err) => {
327 0 : // If we fail to reconstruct a VM or FSM page, we can zero the
328 0 : // page without losing any actual user data. That seems better
329 0 : // than failing repeatedly and getting stuck.
330 0 : //
331 0 : // We had a bug at one point, where we truncated the FSM and VM
332 0 : // in the pageserver, but the Postgres didn't know about that
333 0 : // and continued to generate incremental WAL records for pages
334 0 : // that didn't exist in the pageserver. Trying to replay those
335 0 : // WAL records failed to find the previous image of the page.
336 0 : // This special case allows us to recover from that situation.
337 0 : // See https://github.com/neondatabase/neon/issues/2601.
338 0 : //
339 0 : // Unfortunately we cannot do this for the main fork, or for
340 0 : // any metadata keys, keys, as that would lead to actual data
341 0 : // loss.
342 0 : if is_rel_fsm_block_key(key) || is_rel_vm_block_key(key) {
343 0 : warn!("could not reconstruct FSM or VM key {key}, filling with zeros: {err:?}");
344 0 : ZERO_PAGE.clone()
345 : } else {
346 0 : return Err(err);
347 : }
348 : }
349 : };
350 0 : image_layer_writer.put_image(key, img).await?;
351 0 : key = key.next();
352 : }
353 : }
354 0 : let image_layer = image_layer_writer.finish(&self.timeline).await?;
355 :
356 0 : self.new_images.push(image_layer);
357 0 :
358 0 : timer.stop_and_record();
359 0 :
360 0 : Ok(())
361 0 : }
362 : }
363 :
364 : pub struct RequestContextAdaptor(pub RequestContext);
365 :
366 : impl std::ops::Deref for RequestContextAdaptor {
367 : type Target = RequestContext;
368 :
369 0 : fn deref(&self) -> &Self::Target {
370 0 : &self.0
371 0 : }
372 : }
373 :
374 : impl CompactionRequestContext for RequestContextAdaptor {}
375 :
376 0 : #[derive(Debug, Clone)]
377 : pub struct OwnArc<T>(pub Arc<T>);
378 :
379 : impl<T> Deref for OwnArc<T> {
380 : type Target = <Arc<T> as Deref>::Target;
381 0 : fn deref(&self) -> &Self::Target {
382 0 : &self.0
383 0 : }
384 : }
385 :
386 : impl<T> AsRef<T> for OwnArc<T> {
387 0 : fn as_ref(&self) -> &T {
388 0 : self.0.as_ref()
389 0 : }
390 : }
391 :
392 : impl CompactionLayer<Key> for OwnArc<PersistentLayerDesc> {
393 0 : fn key_range(&self) -> &Range<Key> {
394 0 : &self.key_range
395 0 : }
396 0 : fn lsn_range(&self) -> &Range<Lsn> {
397 0 : &self.lsn_range
398 0 : }
399 0 : fn file_size(&self) -> u64 {
400 0 : self.file_size
401 0 : }
402 0 : fn short_id(&self) -> std::string::String {
403 0 : self.as_ref().short_id().to_string()
404 0 : }
405 0 : fn is_delta(&self) -> bool {
406 0 : self.as_ref().is_delta()
407 0 : }
408 : }
409 :
410 : impl CompactionLayer<Key> for OwnArc<DeltaLayer> {
411 0 : fn key_range(&self) -> &Range<Key> {
412 0 : &self.layer_desc().key_range
413 0 : }
414 0 : fn lsn_range(&self) -> &Range<Lsn> {
415 0 : &self.layer_desc().lsn_range
416 0 : }
417 0 : fn file_size(&self) -> u64 {
418 0 : self.layer_desc().file_size
419 0 : }
420 0 : fn short_id(&self) -> std::string::String {
421 0 : self.layer_desc().short_id().to_string()
422 0 : }
423 0 : fn is_delta(&self) -> bool {
424 0 : true
425 0 : }
426 : }
427 :
428 : use crate::tenant::timeline::DeltaEntry;
429 :
430 : impl CompactionLayer<Key> for ResidentDeltaLayer {
431 0 : fn key_range(&self) -> &Range<Key> {
432 0 : &self.0.layer_desc().key_range
433 0 : }
434 0 : fn lsn_range(&self) -> &Range<Lsn> {
435 0 : &self.0.layer_desc().lsn_range
436 0 : }
437 0 : fn file_size(&self) -> u64 {
438 0 : self.0.layer_desc().file_size
439 0 : }
440 0 : fn short_id(&self) -> std::string::String {
441 0 : self.0.layer_desc().short_id().to_string()
442 0 : }
443 0 : fn is_delta(&self) -> bool {
444 0 : true
445 0 : }
446 : }
447 :
448 : #[async_trait]
449 : impl CompactionDeltaLayer<TimelineAdaptor> for ResidentDeltaLayer {
450 : type DeltaEntry<'a> = DeltaEntry<'a>;
451 :
452 0 : async fn load_keys<'a>(
453 0 : &self,
454 0 : ctx: &RequestContextAdaptor,
455 0 : ) -> anyhow::Result<Vec<DeltaEntry<'_>>> {
456 0 : self.0.load_keys(ctx).await
457 0 : }
458 : }
459 :
460 : impl CompactionLayer<Key> for ResidentImageLayer {
461 0 : fn key_range(&self) -> &Range<Key> {
462 0 : &self.0.layer_desc().key_range
463 0 : }
464 0 : fn lsn_range(&self) -> &Range<Lsn> {
465 0 : &self.0.layer_desc().lsn_range
466 0 : }
467 0 : fn file_size(&self) -> u64 {
468 0 : self.0.layer_desc().file_size
469 0 : }
470 0 : fn short_id(&self) -> std::string::String {
471 0 : self.0.layer_desc().short_id().to_string()
472 0 : }
473 0 : fn is_delta(&self) -> bool {
474 0 : false
475 0 : }
476 : }
477 : impl CompactionImageLayer<TimelineAdaptor> for ResidentImageLayer {}
|