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::Monoid;
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: Monoid + 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                };
172                machine.applier.metrics.gc.finished.inc();
173                machine.applier.shard_metrics.gc_finished.inc();
174                machine
175                    .applier
176                    .metrics
177                    .gc
178                    .seconds
179                    .inc_by(start.elapsed().as_secs_f64());
180
181                // inform all callers who enqueued GC reqs that their work is complete
182                for sender in gc_completed_senders {
183                    // we can safely ignore errors here, it's possible the caller
184                    // wasn't interested in waiting and dropped their receiver.
185                    // maintenance will be somewhat-arbitrarily assigned to the first oneshot.
186                    let _ = sender.send(mem::take(&mut maintenance));
187                }
188            }
189        });
190
191        GarbageCollector {
192            sender: gc_req_sender,
193            _phantom: PhantomData,
194        }
195    }
196
197    /// Enqueues a [GcReq] to be consumed by the GC background task when available.
198    ///
199    /// Returns a future that indicates when GC has cleaned up to at least [GcReq::new_seqno_since]
200    pub fn gc_and_truncate_background(
201        &self,
202        req: GcReq,
203    ) -> Option<oneshot::Receiver<RoutineMaintenance>> {
204        let (gc_completed_sender, gc_completed_receiver) = oneshot::channel();
205        let new_gc_sender = self.sender.clone();
206        let send = new_gc_sender.send((req, gc_completed_sender));
207
208        if let Err(e) = send {
209            // In the steady state we expect this to always succeed, but during
210            // shutdown it is possible the destination task has already spun down
211            warn!(
212                "gc_and_truncate_background failed to send gc request: {}",
213                e
214            );
215            return None;
216        }
217
218        Some(gc_completed_receiver)
219    }
220
221    pub(crate) async fn gc_and_truncate(
222        machine: &Machine<K, V, T, D>,
223        req: GcReq,
224    ) -> (RoutineMaintenance, GcResults) {
225        let mut step_start = Instant::now();
226        let mut report_step_timing = |counter: &Counter| {
227            let now = Instant::now();
228            counter.inc_by(now.duration_since(step_start).as_secs_f64());
229            step_start = now;
230        };
231        assert_eq!(req.shard_id, machine.shard_id());
232
233        // Double check our GC req: seqno_since will never regress
234        // so we can verify it's not somehow greater than the last-
235        // known seqno_since
236        if req.new_seqno_since > machine.applier.seqno_since() {
237            machine
238                .applier
239                .fetch_and_update_state(Some(req.new_seqno_since))
240                .await;
241            let current_seqno_since = machine.applier.seqno_since();
242            assert!(
243                req.new_seqno_since <= current_seqno_since,
244                "invalid gc req: {:?} vs machine seqno_since {}",
245                req,
246                current_seqno_since
247            );
248        }
249
250        // First, check the latest known state to this process to see
251        // if there's relevant GC work for this seqno_since
252        let gc_rollups =
253            GcRollups::new(machine.applier.rollups_lte_seqno(req.new_seqno_since), &req);
254        let rollups_to_remove_from_state = gc_rollups.rollups_to_remove_from_state();
255        report_step_timing(&machine.applier.metrics.gc.steps.find_removable_rollups);
256
257        let mut gc_results = GcResults::default();
258
259        if rollups_to_remove_from_state.is_empty() {
260            // If there are no rollups to remove from state (either the work has already
261            // been done, or the there aren't enough rollups <= seqno_since to have any
262            // to delete), we can safely exit.
263            machine.applier.metrics.gc.noop.inc();
264            return (RoutineMaintenance::default(), gc_results);
265        }
266
267        debug!(
268            "Finding all rollups <= ({}). Will truncate: {:?}. Will remove rollups from state: {:?}",
269            req.new_seqno_since,
270            gc_rollups.truncate_seqnos().collect::<Vec<_>>(),
271            rollups_to_remove_from_state,
272        );
273
274        let mut states = machine
275            .applier
276            .state_versions
277            .fetch_all_live_states(req.shard_id)
278            .await
279            .expect("state is initialized")
280            .check_ts_codec()
281            .expect("ts codec has not changed");
282        let initial_seqno = states.state().seqno;
283        report_step_timing(&machine.applier.metrics.gc.steps.fetch_seconds);
284
285        machine
286            .applier
287            .shard_metrics
288            .gc_live_diffs
289            .set(u64::cast_from(states.len()));
290
291        debug!(
292            "gc seqno_since: ({}) got {} versions from scan",
293            req.new_seqno_since,
294            states.len()
295        );
296
297        Self::incrementally_delete_and_truncate(
298            &mut states,
299            &gc_rollups,
300            machine,
301            &mut report_step_timing,
302            &mut gc_results,
303        )
304        .await;
305
306        // Now that the blobs are deleted / Consensus is truncated, remove
307        // the rollups from state. Doing this at the end ensures that our
308        // invariant is maintained that the current state contains a rollup
309        // to the earliest state in Consensus, and ensures that if GC crashes
310        // part-way through, we still have a reference to these rollups to
311        // resume their deletion.
312        //
313        // This does mean that if GC crashes part-way through we would
314        // repeat work when it resumes. However the redundant work should
315        // be minimal as Consensus is incrementally truncated, allowing
316        // the next run of GC to skip any work needed for rollups less
317        // than the last truncation.
318        //
319        // In short, while this step is not incremental, it does not need
320        // to be for GC to efficiently resume. And in fact, making it
321        // incremental could be quite expensive (e.g. more CaS operations).
322        let (removed_rollups, maintenance) =
323            machine.remove_rollups(rollups_to_remove_from_state).await;
324        report_step_timing(&machine.applier.metrics.gc.steps.remove_rollups_from_state);
325        debug!("CaS removed rollups from state: {:?}", removed_rollups);
326        gc_results.rollups_removed_from_state = removed_rollups;
327
328        // Everything here and below is not strictly needed for GC to complete,
329        // but it's a good opportunity, while we have all live states in hand,
330        // to run some metrics and assertions.
331
332        // Apply all remaining live states to rollup some metrics, like how many
333        // parts are being held (in Blob) that are not part of the latest state.
334        let mut seqno_held_parts = 0;
335        while let Some(_) = states.next(|diff| match diff {
336            InspectDiff::FromInitial(_) => {}
337            InspectDiff::Diff(diff) => {
338                seqno_held_parts += diff.part_deletes().count();
339            }
340        }) {}
341
342        machine
343            .applier
344            .shard_metrics
345            .gc_seqno_held_parts
346            .set(u64::cast_from(seqno_held_parts));
347
348        // verify that the "current" state (as of `fetch_all_live_states`) contains
349        // a rollup to the earliest state we fetched. this invariant isn't affected
350        // by the GC work we just performed, but it is a property of GC correctness
351        // overall / is a convenient place to run the assertion.
352        let valid_pre_gc_state = states
353            .state()
354            .collections
355            .rollups
356            .contains_key(&initial_seqno);
357
358        // this should never be true in the steady-state, but may be true the
359        // first time GC runs after fixing any correctness bugs related to our
360        // state version invariants. we'll make it an error so we can track
361        // any violations in Sentry, but opt not to panic because the root
362        // cause of the violation cannot be from this GC run (in fact, this
363        // GC run, assuming it's correct, should have fixed the violation!)
364        soft_assert_or_log!(
365            valid_pre_gc_state,
366            "earliest state fetched during GC did not have corresponding rollup: rollups = {:?}, state seqno = {}",
367            states.state().collections.rollups,
368            initial_seqno
369        );
370
371        report_step_timing(
372            &machine
373                .applier
374                .metrics
375                .gc
376                .steps
377                .post_gc_calculations_seconds,
378        );
379
380        (maintenance, gc_results)
381    }
382
383    /// Physically deletes all blobs from Blob and live diffs from Consensus that
384    /// are safe to delete, given the `seqno_since`, ensuring that the earliest
385    /// live diff in Consensus has a rollup of seqno `<= seqno_since`.
386    ///
387    /// Internally, performs deletions for each rollup encountered, ensuring that
388    /// incremental progress is made even if the process is interrupted before
389    /// completing all gc work.
390    async fn incrementally_delete_and_truncate<F>(
391        states: &mut StateVersionsIter<T>,
392        gc_rollups: &GcRollups,
393        machine: &Machine<K, V, T, D>,
394        timer: &mut F,
395        gc_results: &mut GcResults,
396    ) where
397        F: FnMut(&Counter),
398    {
399        assert_eq!(states.state().shard_id, machine.shard_id());
400        let shard_id = states.state().shard_id;
401        let mut batch_parts_to_delete = PartDeletes::default();
402        let mut rollups_to_delete: BTreeSet<PartialRollupKey> = BTreeSet::new();
403
404        for truncate_lt in gc_rollups.truncate_seqnos() {
405            assert!(batch_parts_to_delete.is_empty());
406            assert!(rollups_to_delete.is_empty());
407
408            // our state is already past the truncation point. there's no work to do --
409            // some process already truncated this far
410            if states.state().seqno >= truncate_lt {
411                continue;
412            }
413
414            // By our invariant, `states` should always begin on a rollup.
415            assert!(
416                gc_rollups.contains_seqno(&states.state().seqno),
417                "must start with a present rollup before searching for blobs: rollups = {:#?}, state seqno = {}",
418                gc_rollups,
419                states.state().seqno
420            );
421
422            Self::find_removable_blobs(
423                states,
424                truncate_lt,
425                &machine.applier.metrics.gc.steps,
426                timer,
427                &mut batch_parts_to_delete,
428                &mut rollups_to_delete,
429            );
430
431            // After finding removable blobs, our state should be exactly `truncate_lt`,
432            // to ensure we've seen all blob deletions in the diffs needed to reach
433            // this seqno.
434            //
435            // That we can always reach `truncate_lt` given the live diffs we fetched
436            // earlier is a little subtle:
437            // * Our GC request was generated after `seqno_since` was written.
438            // * If our initial seqno on this loop was < `truncate_lt`, then our read
439            //   to `fetch_all_live_states` must have seen live diffs through at least
440            //   `seqno_since`, because the diffs were not yet truncated.
441            // * `seqno_since` >= `truncate_lt`, therefore we must have enough live
442            //   diffs to reach `truncate_lt`.
443            assert_eq!(states.state().seqno, truncate_lt);
444            // `truncate_lt` _is_ the seqno of a rollup, but let's very explicitly
445            // assert that we're about to truncate everything less than a rollup
446            // to maintain our invariant.
447            assert!(
448                gc_rollups.contains_seqno(&states.state().seqno),
449                "must start with a present rollup after searching for blobs: rollups = {:#?}, state seqno = {}",
450                gc_rollups,
451                states.state().seqno
452            );
453
454            // Extra paranoia: verify that none of the blobs we're about to delete
455            // are in our current state (we should only be truncating blobs from
456            // before this state!)
457            states.state().blobs().for_each(|blob| match blob {
458                HollowBlobRef::Batch(batch) => {
459                    for live_part in &batch.parts {
460                        assert!(!batch_parts_to_delete.contains(live_part));
461                    }
462                }
463                HollowBlobRef::Rollup(live_rollup) => {
464                    assert_eq!(rollups_to_delete.get(&live_rollup.key), None);
465                    // And double check that the rollups we're about to delete are
466                    // earlier than our truncation point:
467                    match BlobKey::parse_ids(&live_rollup.key.complete(&shard_id)) {
468                        Ok((_shard, PartialBlobKey::Rollup(rollup_seqno, _rollup))) => {
469                            assert!(rollup_seqno < truncate_lt);
470                        }
471                        _ => {
472                            panic!("invalid rollup during deletion: {:?}", live_rollup);
473                        }
474                    }
475                }
476            });
477
478            gc_results.truncated_consensus_to.push(truncate_lt);
479            gc_results.batch_parts_deleted_from_blob += batch_parts_to_delete.len();
480            gc_results.rollups_deleted_from_blob += rollups_to_delete.len();
481
482            Self::delete_and_truncate(
483                truncate_lt,
484                &mut batch_parts_to_delete,
485                &mut rollups_to_delete,
486                machine,
487                timer,
488            )
489            .await;
490        }
491    }
492
493    /// Iterates through `states`, accumulating all deleted blobs (both batch parts
494    /// and rollups) until reaching the seqno `truncate_lt`.
495    ///
496    /// * The initial seqno of `states` MUST be less than `truncate_lt`.
497    /// * The seqno of `states` after this fn will be exactly `truncate_lt`.
498    fn find_removable_blobs<F>(
499        states: &mut StateVersionsIter<T>,
500        truncate_lt: SeqNo,
501        metrics: &GcStepTimings,
502        timer: &mut F,
503        batch_parts_to_delete: &mut PartDeletes<T>,
504        rollups_to_delete: &mut BTreeSet<PartialRollupKey>,
505    ) where
506        F: FnMut(&Counter),
507    {
508        assert!(states.state().seqno < truncate_lt);
509        while let Some(state) = states.next(|diff| match diff {
510            InspectDiff::FromInitial(_) => {}
511            InspectDiff::Diff(diff) => {
512                diff.rollup_deletes().for_each(|rollup| {
513                    // we use BTreeSets for fast lookups elsewhere, but we should never
514                    // see repeat rollup insertions within a single GC run, otherwise we
515                    // have a logic error or our diffs are incorrect (!)
516                    assert!(rollups_to_delete.insert(rollup.key.to_owned()));
517                });
518                diff.part_deletes().for_each(|part| {
519                    assert!(batch_parts_to_delete.add(part));
520                });
521            }
522        }) {
523            if state.seqno == truncate_lt {
524                break;
525            }
526        }
527        timer(&metrics.find_deletable_blobs_seconds);
528    }
529
530    /// Deletes `batch_parts` and `rollups` from Blob.
531    /// Truncates Consensus to `truncate_lt`.
532    async fn delete_and_truncate<F>(
533        truncate_lt: SeqNo,
534        batch_parts: &mut PartDeletes<T>,
535        rollups: &mut BTreeSet<PartialRollupKey>,
536        machine: &Machine<K, V, T, D>,
537        timer: &mut F,
538    ) where
539        F: FnMut(&Counter),
540    {
541        let shard_id = machine.shard_id();
542        let concurrency_limit = GC_BLOB_DELETE_CONCURRENCY_LIMIT.get(&machine.applier.cfg);
543        let delete_semaphore = Semaphore::new(concurrency_limit);
544
545        let batch_parts = std::mem::take(batch_parts);
546        batch_parts
547            .delete(
548                machine.applier.state_versions.blob.borrow(),
549                shard_id,
550                concurrency_limit,
551                &*machine.applier.metrics,
552                &machine.applier.metrics.retries.external.batch_delete,
553            )
554            .instrument(debug_span!("batch::delete"))
555            .await;
556        timer(&machine.applier.metrics.gc.steps.delete_batch_part_seconds);
557
558        Self::delete_all(
559            machine.applier.state_versions.blob.borrow(),
560            rollups.iter().map(|k| k.complete(&shard_id)),
561            &machine.applier.metrics.retries.external.rollup_delete,
562            debug_span!("rollup::delete"),
563            &delete_semaphore,
564        )
565        .await;
566        rollups.clear();
567        timer(&machine.applier.metrics.gc.steps.delete_rollup_seconds);
568
569        machine
570            .applier
571            .state_versions
572            .truncate_diffs(&shard_id, truncate_lt)
573            .await;
574        timer(&machine.applier.metrics.gc.steps.truncate_diff_seconds);
575    }
576
577    // There's also a bulk delete API in s3 if the performance of this
578    // becomes an issue. Maybe make Blob::delete take a list of keys?
579    //
580    // https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
581    async fn delete_all(
582        blob: &dyn Blob,
583        keys: impl Iterator<Item = BlobKey>,
584        metrics: &RetryMetrics,
585        span: Span,
586        semaphore: &Semaphore,
587    ) {
588        let futures = FuturesUnordered::new();
589        for key in keys {
590            futures.push(
591                retry_external(metrics, move || {
592                    let key = key.clone();
593                    async move {
594                        let _permit = semaphore
595                            .acquire()
596                            .await
597                            .expect("acquiring permit from open semaphore");
598                        blob.delete(&key).await.map(|_| ())
599                    }
600                })
601                .instrument(span.clone()),
602            )
603        }
604
605        futures.collect().await
606    }
607}
608
609#[derive(Debug, Default)]
610pub(crate) struct GcResults {
611    pub(crate) batch_parts_deleted_from_blob: usize,
612    pub(crate) rollups_deleted_from_blob: usize,
613    pub(crate) truncated_consensus_to: Vec<SeqNo>,
614    pub(crate) rollups_removed_from_state: Vec<SeqNo>,
615}
616
617#[derive(Debug)]
618struct GcRollups {
619    rollups_lte_seqno_since: Vec<(SeqNo, PartialRollupKey)>,
620    rollup_seqnos: HashSet<SeqNo>,
621}
622
623impl GcRollups {
624    fn new(rollups_lte_seqno_since: Vec<(SeqNo, PartialRollupKey)>, gc_req: &GcReq) -> Self {
625        assert!(
626            rollups_lte_seqno_since
627                .iter()
628                .all(|(seqno, _rollup)| *seqno <= gc_req.new_seqno_since)
629        );
630        let rollup_seqnos = rollups_lte_seqno_since.iter().map(|(x, _)| *x).collect();
631        Self {
632            rollups_lte_seqno_since,
633            rollup_seqnos,
634        }
635    }
636
637    fn contains_seqno(&self, seqno: &SeqNo) -> bool {
638        self.rollup_seqnos.contains(seqno)
639    }
640
641    /// Returns the seqnos we can safely truncate state to when performing
642    /// incremental GC (all rollups with seqnos <= seqno_since).
643    fn truncate_seqnos(&self) -> impl Iterator<Item = SeqNo> + '_ {
644        self.rollups_lte_seqno_since
645            .iter()
646            .map(|(seqno, _rollup)| *seqno)
647    }
648
649    /// Returns the rollups we can safely remove from state (all rollups
650    /// `<` than the latest rollup `<=` seqno_since).
651    ///
652    /// See the full explanation in [crate::internal::state_versions::StateVersions]
653    /// for how this is derived.
654    fn rollups_to_remove_from_state(&self) -> &[(SeqNo, PartialRollupKey)] {
655        match self.rollups_lte_seqno_since.split_last() {
656            None => &[],
657            Some((_rollup_to_keep, rollups_to_remove_from_state)) => rollups_to_remove_from_state,
658        }
659    }
660}