mz_persist_client/internal/gc.rs
1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10use std::borrow::Borrow;
11use std::collections::BTreeSet;
12use std::fmt::Debug;
13use std::marker::PhantomData;
14use std::mem;
15use std::sync::Arc;
16use std::time::Instant;
17
18use differential_dataflow::difference::Semigroup;
19use differential_dataflow::lattice::Lattice;
20use futures_util::StreamExt;
21use futures_util::stream::FuturesUnordered;
22use prometheus::Counter;
23use timely::progress::Timestamp;
24use tokio::sync::mpsc::UnboundedSender;
25use tokio::sync::{Semaphore, mpsc, oneshot};
26use tracing::{Instrument, Span, debug, debug_span, warn};
27
28use crate::async_runtime::IsolatedRuntime;
29use crate::batch::PartDeletes;
30use crate::cfg::GC_BLOB_DELETE_CONCURRENCY_LIMIT;
31
32use mz_ore::cast::CastFrom;
33use mz_ore::collections::HashSet;
34use mz_ore::soft_assert_or_log;
35use mz_persist::location::{Blob, SeqNo};
36use mz_persist_types::{Codec, Codec64};
37
38use crate::ShardId;
39use crate::internal::machine::{Machine, retry_external};
40use crate::internal::maintenance::RoutineMaintenance;
41use crate::internal::metrics::{GcStepTimings, RetryMetrics};
42use crate::internal::paths::{BlobKey, PartialBlobKey, PartialRollupKey};
43use crate::internal::state::HollowBlobRef;
44use crate::internal::state_versions::{InspectDiff, StateVersionsIter};
45
46#[derive(Debug, Clone, PartialEq)]
47pub struct GcReq {
48 pub shard_id: ShardId,
49 pub new_seqno_since: SeqNo,
50}
51
52#[derive(Debug)]
53pub struct GarbageCollector<K, V, T, D> {
54 sender: UnboundedSender<(GcReq, oneshot::Sender<RoutineMaintenance>)>,
55 _phantom: PhantomData<fn() -> (K, V, T, D)>,
56}
57
58impl<K, V, T, D> Clone for GarbageCollector<K, V, T, D> {
59 fn clone(&self) -> Self {
60 GarbageCollector {
61 sender: self.sender.clone(),
62 _phantom: PhantomData,
63 }
64 }
65}
66
67/// Cleanup for no longer necessary blobs and consensus versions.
68///
69/// - Every read handle, snapshot, and listener is given a capability on seqno
70/// with a very long lease (allowing for infrequent heartbeats). This is a
71/// guarantee that no blobs referenced by the state at that version will be
72/// deleted (even if they've been compacted in some newer version of the
73/// state). This is called a seqno_since in the code as it has obvious
74/// parallels to how sinces work at the shard/collection level. (Is reusing
75/// "since" here a good idea? We could also call it a "seqno capability" or
76/// something else instead.)
77/// - Every state transition via apply_unbatched_cmd has the opportunity to
78/// determine that the overall seqno_since for the shard has changed. In the
79/// common case in production, this will be in response to a snapshot
80/// finishing or a listener emitting some batch.
81/// - It would be nice if this only ever happened in response to read-y things
82/// (there'd be a nice parallel to how compaction background work is only
83/// spawned by write activity), but if there are no readers, we still very
84/// much want to continue to garbage collect. Notably, if there are no
85/// readers, we naturally only need to hold a capability on the current
86/// version of state. This means that if there are only writers, a write
87/// commands will result in the seqno_since advancing immediately from the
88/// previous version of the state to the new one.
89/// - Like Compacter, GarbageCollector uses a heuristic to ignore some requests
90/// to save work. In this case, the tradeoff is between consensus traffic
91/// (plus a bit of cpu) and keeping blobs around longer than strictly
92/// necessary. This is correct because a process could always die while
93/// executing one of these requests (or be slow and still working on it when
94/// the next request is generated), so we anyway need to handle them being
95/// dropped.
96/// - GarbageCollector works by `Consensus::scan`-ing for every live version of
97/// state (ignoring what the request things the prev_state_seqno was for the
98/// reasons mentioned immediately above). It then walks through them in a
99/// loop, accumulating a BTreeSet of every referenced blob key. When it finds
100/// the version corresponding to the new_seqno_since, it removes every blob in
101/// that version of the state from the BTreeSet and exits the loop. This
102/// results in the BTreeSet containing every blob eligible for deletion. It
103/// deletes those blobs and then truncates the state to the new_seqno_since to
104/// indicate that this work doesn't need to be done again.
105/// - Note that these requests are being processed concurrently, so it's always
106/// possible that some future request has already deleted the blobs and
107/// truncated consensus. It's also possible that this is the future request.
108/// As a result, the only guarantee that we get is that the current version of
109/// head is >= new_seqno_since.
110/// - (Aside: The above also means that if Blob is not linearizable, there is a
111/// possible race where a blob gets deleted before it written and thus is
112/// leaked. We anyway always have the possibility of a write process being
113/// killed between when it writes a blob and links it into state, so this is
114/// fine; it'll be caught and fixed by the same mechanism.)
115impl<K, V, T, D> GarbageCollector<K, V, T, D>
116where
117 K: Debug + Codec,
118 V: Debug + Codec,
119 T: Timestamp + Lattice + Codec64 + Sync,
120 D: Semigroup + Codec64,
121{
122 pub fn new(machine: Machine<K, V, T, D>, isolated_runtime: Arc<IsolatedRuntime>) -> Self {
123 let (gc_req_sender, mut gc_req_recv) =
124 mpsc::unbounded_channel::<(GcReq, oneshot::Sender<RoutineMaintenance>)>();
125
126 // spin off a single task responsible for executing GC requests.
127 // work is enqueued into the task through a channel
128 let _worker_handle = mz_ore::task::spawn(|| "PersistGcWorker", async move {
129 while let Some((req, completer)) = gc_req_recv.recv().await {
130 let mut consolidated_req = req;
131 let mut gc_completed_senders = vec![completer];
132
133 // check if any further gc requests have built up. we'll merge their requests
134 // together and run a single GC pass to satisfy all of them
135 while let Ok((req, completer)) = gc_req_recv.try_recv() {
136 assert_eq!(req.shard_id, consolidated_req.shard_id);
137 gc_completed_senders.push(completer);
138 consolidated_req.new_seqno_since =
139 std::cmp::max(req.new_seqno_since, consolidated_req.new_seqno_since);
140 }
141
142 let merged_requests = gc_completed_senders.len() - 1;
143 if merged_requests > 0 {
144 machine
145 .applier
146 .metrics
147 .gc
148 .merged
149 .inc_by(u64::cast_from(merged_requests));
150 debug!(
151 "Merged {} gc requests together for shard {}",
152 merged_requests, consolidated_req.shard_id
153 );
154 }
155
156 let gc_span = debug_span!(parent: None, "gc_and_truncate", shard_id=%consolidated_req.shard_id);
157 gc_span.follows_from(&Span::current());
158
159 let start = Instant::now();
160 machine.applier.metrics.gc.started.inc();
161 let (mut maintenance, _stats) = {
162 let name = format!("gc_and_truncate ({})", &consolidated_req.shard_id);
163 let machine = machine.clone();
164 isolated_runtime
165 .spawn_named(|| name, async move {
166 Self::gc_and_truncate(&machine, consolidated_req)
167 .instrument(gc_span)
168 .await
169 })
170 .await
171 .expect("gc_and_truncate failed")
172 };
173 machine.applier.metrics.gc.finished.inc();
174 machine.applier.shard_metrics.gc_finished.inc();
175 machine
176 .applier
177 .metrics
178 .gc
179 .seconds
180 .inc_by(start.elapsed().as_secs_f64());
181
182 // inform all callers who enqueued GC reqs that their work is complete
183 for sender in gc_completed_senders {
184 // we can safely ignore errors here, it's possible the caller
185 // wasn't interested in waiting and dropped their receiver.
186 // maintenance will be somewhat-arbitrarily assigned to the first oneshot.
187 let _ = sender.send(mem::take(&mut maintenance));
188 }
189 }
190 });
191
192 GarbageCollector {
193 sender: gc_req_sender,
194 _phantom: PhantomData,
195 }
196 }
197
198 /// Enqueues a [GcReq] to be consumed by the GC background task when available.
199 ///
200 /// Returns a future that indicates when GC has cleaned up to at least [GcReq::new_seqno_since]
201 pub fn gc_and_truncate_background(
202 &self,
203 req: GcReq,
204 ) -> Option<oneshot::Receiver<RoutineMaintenance>> {
205 let (gc_completed_sender, gc_completed_receiver) = oneshot::channel();
206 let new_gc_sender = self.sender.clone();
207 let send = new_gc_sender.send((req, gc_completed_sender));
208
209 if let Err(e) = send {
210 // In the steady state we expect this to always succeed, but during
211 // shutdown it is possible the destination task has already spun down
212 warn!(
213 "gc_and_truncate_background failed to send gc request: {}",
214 e
215 );
216 return None;
217 }
218
219 Some(gc_completed_receiver)
220 }
221
222 pub(crate) async fn gc_and_truncate(
223 machine: &Machine<K, V, T, D>,
224 req: GcReq,
225 ) -> (RoutineMaintenance, GcResults) {
226 let mut step_start = Instant::now();
227 let mut report_step_timing = |counter: &Counter| {
228 let now = Instant::now();
229 counter.inc_by(now.duration_since(step_start).as_secs_f64());
230 step_start = now;
231 };
232 assert_eq!(req.shard_id, machine.shard_id());
233
234 // Double check our GC req: seqno_since will never regress
235 // so we can verify it's not somehow greater than the last-
236 // known seqno_since
237 if req.new_seqno_since > machine.applier.seqno_since() {
238 machine
239 .applier
240 .fetch_and_update_state(Some(req.new_seqno_since))
241 .await;
242 let current_seqno_since = machine.applier.seqno_since();
243 assert!(
244 req.new_seqno_since <= current_seqno_since,
245 "invalid gc req: {:?} vs machine seqno_since {}",
246 req,
247 current_seqno_since
248 );
249 }
250
251 // First, check the latest known state to this process to see
252 // if there's relevant GC work for this seqno_since
253 let gc_rollups =
254 GcRollups::new(machine.applier.rollups_lte_seqno(req.new_seqno_since), &req);
255 let rollups_to_remove_from_state = gc_rollups.rollups_to_remove_from_state();
256 report_step_timing(&machine.applier.metrics.gc.steps.find_removable_rollups);
257
258 let mut gc_results = GcResults::default();
259
260 if rollups_to_remove_from_state.is_empty() {
261 // If there are no rollups to remove from state (either the work has already
262 // been done, or the there aren't enough rollups <= seqno_since to have any
263 // to delete), we can safely exit.
264 machine.applier.metrics.gc.noop.inc();
265 return (RoutineMaintenance::default(), gc_results);
266 }
267
268 debug!(
269 "Finding all rollups <= ({}). Will truncate: {:?}. Will remove rollups from state: {:?}",
270 req.new_seqno_since,
271 gc_rollups.truncate_seqnos().collect::<Vec<_>>(),
272 rollups_to_remove_from_state,
273 );
274
275 let mut states = machine
276 .applier
277 .state_versions
278 .fetch_all_live_states(req.shard_id)
279 .await
280 .expect("state is initialized")
281 .check_ts_codec()
282 .expect("ts codec has not changed");
283 let initial_seqno = states.state().seqno;
284 report_step_timing(&machine.applier.metrics.gc.steps.fetch_seconds);
285
286 machine
287 .applier
288 .shard_metrics
289 .gc_live_diffs
290 .set(u64::cast_from(states.len()));
291
292 debug!(
293 "gc seqno_since: ({}) got {} versions from scan",
294 req.new_seqno_since,
295 states.len()
296 );
297
298 Self::incrementally_delete_and_truncate(
299 &mut states,
300 &gc_rollups,
301 machine,
302 &mut report_step_timing,
303 &mut gc_results,
304 )
305 .await;
306
307 // Now that the blobs are deleted / Consensus is truncated, remove
308 // the rollups from state. Doing this at the end ensures that our
309 // invariant is maintained that the current state contains a rollup
310 // to the earliest state in Consensus, and ensures that if GC crashes
311 // part-way through, we still have a reference to these rollups to
312 // resume their deletion.
313 //
314 // This does mean that if GC crashes part-way through we would
315 // repeat work when it resumes. However the redundant work should
316 // be minimal as Consensus is incrementally truncated, allowing
317 // the next run of GC to skip any work needed for rollups less
318 // than the last truncation.
319 //
320 // In short, while this step is not incremental, it does not need
321 // to be for GC to efficiently resume. And in fact, making it
322 // incremental could be quite expensive (e.g. more CaS operations).
323 let (removed_rollups, maintenance) =
324 machine.remove_rollups(rollups_to_remove_from_state).await;
325 report_step_timing(&machine.applier.metrics.gc.steps.remove_rollups_from_state);
326 debug!("CaS removed rollups from state: {:?}", removed_rollups);
327 gc_results.rollups_removed_from_state = removed_rollups;
328
329 // Everything here and below is not strictly needed for GC to complete,
330 // but it's a good opportunity, while we have all live states in hand,
331 // to run some metrics and assertions.
332
333 // Apply all remaining live states to rollup some metrics, like how many
334 // parts are being held (in Blob) that are not part of the latest state.
335 let mut seqno_held_parts = 0;
336 while let Some(_) = states.next(|diff| match diff {
337 InspectDiff::FromInitial(_) => {}
338 InspectDiff::Diff(diff) => {
339 diff.blob_deletes().for_each(|blob| match blob {
340 HollowBlobRef::Batch(batch) => {
341 seqno_held_parts += batch.part_count();
342 }
343 HollowBlobRef::Rollup(_) => {}
344 });
345 }
346 }) {}
347
348 machine
349 .applier
350 .shard_metrics
351 .gc_seqno_held_parts
352 .set(u64::cast_from(seqno_held_parts));
353
354 // verify that the "current" state (as of `fetch_all_live_states`) contains
355 // a rollup to the earliest state we fetched. this invariant isn't affected
356 // by the GC work we just performed, but it is a property of GC correctness
357 // overall / is a convenient place to run the assertion.
358 let valid_pre_gc_state = states
359 .state()
360 .collections
361 .rollups
362 .contains_key(&initial_seqno);
363
364 // this should never be true in the steady-state, but may be true the
365 // first time GC runs after fixing any correctness bugs related to our
366 // state version invariants. we'll make it an error so we can track
367 // any violations in Sentry, but opt not to panic because the root
368 // cause of the violation cannot be from this GC run (in fact, this
369 // GC run, assuming it's correct, should have fixed the violation!)
370 soft_assert_or_log!(
371 valid_pre_gc_state,
372 "earliest state fetched during GC did not have corresponding rollup: rollups = {:?}, state seqno = {}",
373 states.state().collections.rollups,
374 initial_seqno
375 );
376
377 report_step_timing(
378 &machine
379 .applier
380 .metrics
381 .gc
382 .steps
383 .post_gc_calculations_seconds,
384 );
385
386 (maintenance, gc_results)
387 }
388
389 /// Physically deletes all blobs from Blob and live diffs from Consensus that
390 /// are safe to delete, given the `seqno_since`, ensuring that the earliest
391 /// live diff in Consensus has a rollup of seqno `<= seqno_since`.
392 ///
393 /// Internally, performs deletions for each rollup encountered, ensuring that
394 /// incremental progress is made even if the process is interrupted before
395 /// completing all gc work.
396 async fn incrementally_delete_and_truncate<F>(
397 states: &mut StateVersionsIter<T>,
398 gc_rollups: &GcRollups,
399 machine: &Machine<K, V, T, D>,
400 timer: &mut F,
401 gc_results: &mut GcResults,
402 ) where
403 F: FnMut(&Counter),
404 {
405 assert_eq!(states.state().shard_id, machine.shard_id());
406 let shard_id = states.state().shard_id;
407 let mut batch_parts_to_delete = PartDeletes::default();
408 let mut rollups_to_delete: BTreeSet<PartialRollupKey> = BTreeSet::new();
409
410 for truncate_lt in gc_rollups.truncate_seqnos() {
411 assert!(batch_parts_to_delete.is_empty());
412 assert!(rollups_to_delete.is_empty());
413
414 // our state is already past the truncation point. there's no work to do --
415 // some process already truncated this far
416 if states.state().seqno >= truncate_lt {
417 continue;
418 }
419
420 // By our invariant, `states` should always begin on a rollup.
421 assert!(
422 gc_rollups.contains_seqno(&states.state().seqno),
423 "must start with a present rollup before searching for blobs: rollups = {:#?}, state seqno = {}",
424 gc_rollups,
425 states.state().seqno
426 );
427
428 Self::find_removable_blobs(
429 states,
430 truncate_lt,
431 &machine.applier.metrics.gc.steps,
432 timer,
433 &mut batch_parts_to_delete,
434 &mut rollups_to_delete,
435 );
436
437 // After finding removable blobs, our state should be exactly `truncate_lt`,
438 // to ensure we've seen all blob deletions in the diffs needed to reach
439 // this seqno.
440 //
441 // That we can always reach `truncate_lt` given the live diffs we fetched
442 // earlier is a little subtle:
443 // * Our GC request was generated after `seqno_since` was written.
444 // * If our initial seqno on this loop was < `truncate_lt`, then our read
445 // to `fetch_all_live_states` must have seen live diffs through at least
446 // `seqno_since`, because the diffs were not yet truncated.
447 // * `seqno_since` >= `truncate_lt`, therefore we must have enough live
448 // diffs to reach `truncate_lt`.
449 assert_eq!(states.state().seqno, truncate_lt);
450 // `truncate_lt` _is_ the seqno of a rollup, but let's very explicitly
451 // assert that we're about to truncate everything less than a rollup
452 // to maintain our invariant.
453 assert!(
454 gc_rollups.contains_seqno(&states.state().seqno),
455 "must start with a present rollup after searching for blobs: rollups = {:#?}, state seqno = {}",
456 gc_rollups,
457 states.state().seqno
458 );
459
460 // Extra paranoia: verify that none of the blobs we're about to delete
461 // are in our current state (we should only be truncating blobs from
462 // before this state!)
463 states.state().blobs().for_each(|blob| match blob {
464 HollowBlobRef::Batch(batch) => {
465 for live_part in &batch.parts {
466 assert!(!batch_parts_to_delete.contains(live_part));
467 }
468 }
469 HollowBlobRef::Rollup(live_rollup) => {
470 assert_eq!(rollups_to_delete.get(&live_rollup.key), None);
471 // And double check that the rollups we're about to delete are
472 // earlier than our truncation point:
473 match BlobKey::parse_ids(&live_rollup.key.complete(&shard_id)) {
474 Ok((_shard, PartialBlobKey::Rollup(rollup_seqno, _rollup))) => {
475 assert!(rollup_seqno < truncate_lt);
476 }
477 _ => {
478 panic!("invalid rollup during deletion: {:?}", live_rollup);
479 }
480 }
481 }
482 });
483
484 gc_results.truncated_consensus_to.push(truncate_lt);
485 gc_results.batch_parts_deleted_from_blob += batch_parts_to_delete.len();
486 gc_results.rollups_deleted_from_blob += rollups_to_delete.len();
487
488 Self::delete_and_truncate(
489 truncate_lt,
490 &mut batch_parts_to_delete,
491 &mut rollups_to_delete,
492 machine,
493 timer,
494 )
495 .await;
496 }
497 }
498
499 /// Iterates through `states`, accumulating all deleted blobs (both batch parts
500 /// and rollups) until reaching the seqno `truncate_lt`.
501 ///
502 /// * The initial seqno of `states` MUST be less than `truncate_lt`.
503 /// * The seqno of `states` after this fn will be exactly `truncate_lt`.
504 fn find_removable_blobs<F>(
505 states: &mut StateVersionsIter<T>,
506 truncate_lt: SeqNo,
507 metrics: &GcStepTimings,
508 timer: &mut F,
509 batch_parts_to_delete: &mut PartDeletes<T>,
510 rollups_to_delete: &mut BTreeSet<PartialRollupKey>,
511 ) where
512 F: FnMut(&Counter),
513 {
514 assert!(states.state().seqno < truncate_lt);
515 while let Some(state) = states.next(|diff| match diff {
516 InspectDiff::FromInitial(_) => {}
517 InspectDiff::Diff(diff) => {
518 diff.blob_deletes().for_each(|blob| match blob {
519 HollowBlobRef::Batch(batch) => {
520 for part in &batch.parts {
521 // we use BTreeSets for fast lookups elsewhere, but we should never
522 // see repeat blob insertions within a single GC run, otherwise we
523 // have a logic error or our diffs are incorrect (!)
524 assert!(batch_parts_to_delete.add(part));
525 }
526 }
527 HollowBlobRef::Rollup(rollup) => {
528 assert!(rollups_to_delete.insert(rollup.key.to_owned()));
529 }
530 });
531 }
532 }) {
533 if state.seqno == truncate_lt {
534 break;
535 }
536 }
537 timer(&metrics.find_deletable_blobs_seconds);
538 }
539
540 /// Deletes `batch_parts` and `rollups` from Blob.
541 /// Truncates Consensus to `truncate_lt`.
542 async fn delete_and_truncate<F>(
543 truncate_lt: SeqNo,
544 batch_parts: &mut PartDeletes<T>,
545 rollups: &mut BTreeSet<PartialRollupKey>,
546 machine: &Machine<K, V, T, D>,
547 timer: &mut F,
548 ) where
549 F: FnMut(&Counter),
550 {
551 let shard_id = machine.shard_id();
552 let concurrency_limit = GC_BLOB_DELETE_CONCURRENCY_LIMIT.get(&machine.applier.cfg);
553 let delete_semaphore = Semaphore::new(concurrency_limit);
554
555 let batch_parts = std::mem::take(batch_parts);
556 batch_parts
557 .delete(
558 machine.applier.state_versions.blob.borrow(),
559 shard_id,
560 concurrency_limit,
561 &*machine.applier.metrics,
562 &machine.applier.metrics.retries.external.batch_delete,
563 )
564 .instrument(debug_span!("batch::delete"))
565 .await;
566 timer(&machine.applier.metrics.gc.steps.delete_batch_part_seconds);
567
568 Self::delete_all(
569 machine.applier.state_versions.blob.borrow(),
570 rollups.iter().map(|k| k.complete(&shard_id)),
571 &machine.applier.metrics.retries.external.rollup_delete,
572 debug_span!("rollup::delete"),
573 &delete_semaphore,
574 )
575 .await;
576 rollups.clear();
577 timer(&machine.applier.metrics.gc.steps.delete_rollup_seconds);
578
579 machine
580 .applier
581 .state_versions
582 .truncate_diffs(&shard_id, truncate_lt)
583 .await;
584 timer(&machine.applier.metrics.gc.steps.truncate_diff_seconds);
585 }
586
587 // There's also a bulk delete API in s3 if the performance of this
588 // becomes an issue. Maybe make Blob::delete take a list of keys?
589 //
590 // https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
591 async fn delete_all(
592 blob: &dyn Blob,
593 keys: impl Iterator<Item = BlobKey>,
594 metrics: &RetryMetrics,
595 span: Span,
596 semaphore: &Semaphore,
597 ) {
598 let futures = FuturesUnordered::new();
599 for key in keys {
600 futures.push(
601 retry_external(metrics, move || {
602 let key = key.clone();
603 async move {
604 let _permit = semaphore
605 .acquire()
606 .await
607 .expect("acquiring permit from open semaphore");
608 blob.delete(&key).await.map(|_| ())
609 }
610 })
611 .instrument(span.clone()),
612 )
613 }
614
615 futures.collect().await
616 }
617}
618
619#[derive(Debug, Default)]
620pub(crate) struct GcResults {
621 pub(crate) batch_parts_deleted_from_blob: usize,
622 pub(crate) rollups_deleted_from_blob: usize,
623 pub(crate) truncated_consensus_to: Vec<SeqNo>,
624 pub(crate) rollups_removed_from_state: Vec<SeqNo>,
625}
626
627#[derive(Debug)]
628struct GcRollups {
629 rollups_lte_seqno_since: Vec<(SeqNo, PartialRollupKey)>,
630 rollup_seqnos: HashSet<SeqNo>,
631}
632
633impl GcRollups {
634 fn new(rollups_lte_seqno_since: Vec<(SeqNo, PartialRollupKey)>, gc_req: &GcReq) -> Self {
635 assert!(
636 rollups_lte_seqno_since
637 .iter()
638 .all(|(seqno, _rollup)| *seqno <= gc_req.new_seqno_since)
639 );
640 let rollup_seqnos = rollups_lte_seqno_since.iter().map(|(x, _)| *x).collect();
641 Self {
642 rollups_lte_seqno_since,
643 rollup_seqnos,
644 }
645 }
646
647 fn contains_seqno(&self, seqno: &SeqNo) -> bool {
648 self.rollup_seqnos.contains(seqno)
649 }
650
651 /// Returns the seqnos we can safely truncate state to when performing
652 /// incremental GC (all rollups with seqnos <= seqno_since).
653 fn truncate_seqnos(&self) -> impl Iterator<Item = SeqNo> + '_ {
654 self.rollups_lte_seqno_since
655 .iter()
656 .map(|(seqno, _rollup)| *seqno)
657 }
658
659 /// Returns the rollups we can safely remove from state (all rollups
660 /// `<` than the latest rollup `<=` seqno_since).
661 ///
662 /// See the full explanation in [crate::internal::state_versions::StateVersions]
663 /// for how this is derived.
664 fn rollups_to_remove_from_state(&self) -> &[(SeqNo, PartialRollupKey)] {
665 match self.rollups_lte_seqno_since.split_last() {
666 None => &[],
667 Some((_rollup_to_keep, rollups_to_remove_from_state)) => rollups_to_remove_from_state,
668 }
669 }
670}