Skip to main content

mz_compute/sink/
materialized_view.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
10//! A dataflow sink that writes input records to a persist shard.
11//!
12//! This implementation is both parallel and self-correcting.
13//!
14//!  * parallel: Multiple workers can participate in writing updates for the same times, letting
15//!    sink throughput scale with the number of workers allocated to the replica.
16//!  * self-correcting: The sink continually compares the contents of the persist shard with the
17//!    contents of the input collection and writes down the difference. If the persist shard ends
18//!    up with undesired contents for any reason, this is corrected the next time the sink manages
19//!    to append to the shard.
20//!
21//! ### Operators
22//!
23//! The persist sink consists of a graph of operators.
24//!
25//!    desired                    persist <---------------.
26//!       |                          |                    |
27//!       |                          |                    |
28//!       |---------------------.    |                    |
29//!       |                     |    |                    |
30//!       |                     |    |                    |
31//!       v                     v    v                    |
32//!   +--------+              +--------+              +--------+
33//!   |  mint  | --descs-.--> | write  | --batches--> | append |
34//!   +--------+          \   +--------+          .-> +--------+
35//!                        \_____________________/
36//!
37//!  * `mint` mints batch descriptions, i.e., `(lower, upper)` bounds of batches that should be
38//!    written. The persist API requires that all workers write batches with the same bounds, so
39//!    they can be appended as a single logical batch. To ensure this, the `mint` operator only
40//!    runs on a single worker that broadcasts minted descriptions to all workers. Batch bounds are
41//!    picked based on the frontiers of the `desired` stream and the output persist shard.
42//!  * `write` stages batch data in persist, based on the batch descriptions received from the
43//!    `mint` operator, but without appending it to the persist shard. This is a multi-worker
44//!    operator, with each worker writing batches of the data that arrives at its local inputs. To
45//!    do so it reads from the `desired` and `persist` streams and produces the difference between
46//!    them to write back out, ensuring that the final contents of the persist shard match
47//!    `desired`.
48//!  * `append` appends the batches minted by `mint` and written by `write` to the persist shard.
49//!    This is a multi-worker operator, where workers are responsible for different subsets of
50//!    batch descriptions. If a worker is responsible for a given batch description, it waits for
51//!    all workers to stage their batches for that batch description, then appends all the batches
52//!    together as a single logical batch.
53//!
54//! Note that while the above graph suggests that `mint` and `write` both receive copies of the
55//! `desired` stream, the actual implementation passes that stream through `mint` and lets `write`
56//! read the passed-through stream, to avoid cloning data.
57//!
58//! Also note that the `append` operator's implementation would perhaps be more natural as a
59//! single-worker implementation. The purpose of sharing the work between all workers is to avoid a
60//! work imbalance where one worker is overloaded (doing both appends and the consequent persist
61//! maintenance work) while others are comparatively idle.
62//!
63//! The persist sink is written to be robust to the presence of other conflicting instances (e.g.
64//! from other replicas) writing to the same persist shard. Each of the three operators needs to be
65//! able to handle conflicting writes that unexpectedly change the contents of the output persist
66//! shard.
67//!
68//! ### Frontiers
69//!
70//! The `desired` frontier tracks the progress of the upstream dataflow, but may be rounded up to
71//! the next refresh time for dataflows that follow a refresh schedule other than "on commit".
72//!
73//! The `persist` frontier tracks the `upper` frontier of the target persist shard, with one
74//! exception: When the `persist_source` that reads back the shard is rendered, it will start
75//! reading at its `since` frontier. So if the shard's `since` is initially greater than its
76//! `upper`, the `persist` frontier too will be in advance of the shard `upper`, until the `upper`
77//! has caught up. To avoid getting confused by this edge case, the `mint` operator does not use
78//! the `persist` stream to observe the shard frontier but keeps its own `WriteHandle` instead.
79//!
80//! The `descs` frontier communicates which `lower` bounds may still be emitted in batch
81//! descriptions. All future batch descriptions will have a `lower` that is greater or equal to the
82//! current `descs` frontier.
83//!
84//! The `batches` frontier communicates for which `lower` bounds batches may still be written. All
85//! batches for descriptions with `lower`s less than the current `batches` frontier have already
86//! been written.
87//!
88//! ### Invariants
89//!
90//! The implementation upholds several invariants that can be relied upon to simplify the
91//! implementation:
92//!
93//!  1. `lower`s in minted batch descriptions are unique and strictly increasing. That is, the
94//!     `mint` operator will never mint the same `lower` twice and a minted `lower` is always
95//!     greater than any previously minted ones.
96//!  2. `upper`s in minted batch descriptions are monotonically increasing.
97//!  3. From (1) follows that there is always at most one "valid" batch description in flight in
98//!     the operator graph. "Valid" here means that the described batch can be appended to the
99//!     persist shard.
100//!
101//! The main simplification these invariants allow is that operators only need to keep track of the
102//! most recent batch description and/or `lower`. Previous batch descriptions are not valid
103//! anymore, so there is no reason to hold any state or perform any work in support of them.
104//!
105//! ### Read-only Mode
106//!
107//! The persist sink can optionally be initialized in read-only mode. In this mode it is passive
108//! and avoids any writes to persist. Activating the `read_only_rx` transitions the sink into write
109//! mode, where it commences normal operation.
110//!
111//! Read-only mode is implemented by the `mint` operator. To disable writes, the `mint` operator
112//! simply avoids minting any batch descriptions. Since both the `write` and the `append` operator
113//! require batch descriptions to write/append batches, this suppresses any persist communication.
114//! At the same time, the `write` operator still observes changes to the `desired` and `persist`
115//! collections, allowing it to keep its correction buffer up-to-date.
116
117use std::any::Any;
118use std::cell::RefCell;
119use std::pin::pin;
120use std::rc::Rc;
121use std::sync::Arc;
122
123use differential_dataflow::{AsCollection, Hashable, VecCollection};
124use futures::StreamExt;
125use mz_compute_types::dyncfgs::MV_SINK_ADVANCE_PERSIST_FRONTIERS;
126use mz_compute_types::sinks::{ComputeSinkDesc, MaterializedViewSinkConnection};
127use mz_dyncfg::ConfigSet;
128use mz_ore::cast::CastFrom;
129use mz_persist_client::batch::{Batch, ProtoBatch};
130use mz_persist_client::cache::PersistClientCache;
131use mz_persist_client::metrics::SinkMetrics;
132use mz_persist_client::operators::shard_source::{ErrorHandler, SnapshotMode};
133use mz_persist_client::write::WriteHandle;
134use mz_persist_client::{Diagnostics, PersistClient};
135use mz_persist_types::codec_impls::UnitSchema;
136use mz_repr::{Diff, GlobalId, Row, Timestamp};
137use mz_storage_types::StorageDiff;
138use mz_storage_types::controller::CollectionMetadata;
139use mz_storage_types::errors::DataflowError;
140use mz_storage_types::sources::SourceData;
141use mz_timely_util::builder_async::PressOnDropButton;
142use mz_timely_util::builder_async::{Event, OperatorBuilder};
143use mz_timely_util::probe::{Handle, ProbeNotify};
144use serde::{Deserialize, Serialize};
145use timely::PartialOrder;
146use timely::container::CapacityContainerBuilder;
147use timely::dataflow::channels::pact::{Exchange, Pipeline};
148use timely::dataflow::operators::vec::Broadcast;
149use timely::dataflow::operators::{Capability, CapabilitySet, probe};
150use timely::dataflow::{Scope, StreamVec};
151use timely::progress::Antichain;
152use tokio::sync::watch;
153use tracing::trace;
154
155use crate::compute_state::ComputeState;
156use crate::render::StartSignal;
157use crate::render::sinks::SinkRender;
158use crate::sink::correction::{Correction, Logging};
159use crate::sink::refresh::apply_refresh;
160
161impl<'scope> SinkRender<'scope> for MaterializedViewSinkConnection<CollectionMetadata> {
162    fn render_sink(
163        &self,
164        compute_state: &mut ComputeState,
165        sink: &ComputeSinkDesc<CollectionMetadata>,
166        sink_id: GlobalId,
167        as_of: Antichain<Timestamp>,
168        start_signal: StartSignal,
169        mut ok_collection: VecCollection<'scope, Timestamp, Row, Diff>,
170        mut err_collection: VecCollection<'scope, Timestamp, DataflowError, Diff>,
171        _ct_times: Option<VecCollection<'scope, Timestamp, (), Diff>>,
172        output_probe: &Handle<Timestamp>,
173    ) -> Option<Rc<dyn Any>> {
174        // Attach probes reporting the compute frontier.
175        // The `apply_refresh` operator can round up frontiers, making it impossible to accurately
176        // track the progress of the computation, so we need to attach probes before it.
177        let probe = probe::Handle::default();
178        ok_collection = ok_collection
179            .probe_with(&probe)
180            .inner
181            .probe_notify_with(vec![output_probe.clone()])
182            .as_collection();
183        let collection_state = compute_state.expect_collection_mut(sink_id);
184        collection_state.compute_probe = Some(probe);
185
186        // If a `RefreshSchedule` was specified, round up timestamps.
187        if let Some(refresh_schedule) = &sink.refresh_schedule {
188            ok_collection = apply_refresh(ok_collection, refresh_schedule.clone());
189            err_collection = apply_refresh(err_collection, refresh_schedule.clone());
190        }
191
192        if sink.up_to != Antichain::default() {
193            unimplemented!(
194                "UP TO is not supported for persist sinks yet, and shouldn't have been accepted during parsing/planning"
195            )
196        }
197
198        let read_only_rx = collection_state.read_only_rx.clone();
199
200        let token = persist_sink(
201            sink_id,
202            &self.storage_metadata,
203            ok_collection,
204            err_collection,
205            as_of,
206            compute_state,
207            start_signal,
208            read_only_rx,
209        );
210        Some(token)
211    }
212}
213
214/// Type of the `desired` stream, split into `Ok` and `Err` streams.
215type DesiredStreams<'s> = OkErr<
216    StreamVec<'s, Timestamp, (Row, Timestamp, Diff)>,
217    StreamVec<'s, Timestamp, (DataflowError, Timestamp, Diff)>,
218>;
219
220/// Type of the `persist` stream, split into `Ok` and `Err` streams.
221type PersistStreams<'s> = OkErr<
222    StreamVec<'s, Timestamp, (Row, Timestamp, Diff)>,
223    StreamVec<'s, Timestamp, (DataflowError, Timestamp, Diff)>,
224>;
225
226/// Type of the `descs` stream.
227type DescsStream<'s> = StreamVec<'s, Timestamp, BatchDescription>;
228
229/// Type of the `batches` stream.
230type BatchesStream<'s> = StreamVec<'s, Timestamp, (BatchDescription, ProtoBatch)>;
231
232/// Type of the shared sink write frontier.
233type SharedSinkFrontier = Rc<RefCell<Antichain<Timestamp>>>;
234
235/// Renders an MV sink writing the given desired collection into the `target` persist collection.
236pub(super) fn persist_sink<'s>(
237    sink_id: GlobalId,
238    target: &CollectionMetadata,
239    ok_collection: VecCollection<'s, Timestamp, Row, Diff>,
240    err_collection: VecCollection<'s, Timestamp, DataflowError, Diff>,
241    as_of: Antichain<Timestamp>,
242    compute_state: &mut ComputeState,
243    start_signal: StartSignal,
244    read_only_rx: watch::Receiver<bool>,
245) -> Rc<dyn Any>
246where
247{
248    let scope = ok_collection.scope();
249    let desired = OkErr::new(ok_collection.inner, err_collection.inner);
250
251    // Read back the persist shard.
252    let (persist, persist_token) =
253        persist_source(scope, sink_id, target.clone(), compute_state, start_signal);
254
255    let persist_api = PersistApi {
256        persist_clients: Arc::clone(&compute_state.persist_clients),
257        collection: target.clone(),
258        shard_name: sink_id.to_string(),
259        purpose: format!("MV sink {sink_id}"),
260    };
261
262    let (desired, descs, sink_frontier, mint_token) = mint::render(
263        sink_id,
264        persist_api.clone(),
265        as_of.clone(),
266        read_only_rx,
267        desired,
268    );
269
270    let (batches, write_token) = write::render(
271        sink_id,
272        persist_api.clone(),
273        as_of,
274        desired,
275        persist,
276        descs.clone(),
277        Rc::clone(&compute_state.worker_config),
278    );
279
280    let append_token = append::render(sink_id, persist_api, descs, batches);
281
282    // Report sink frontier updates to the `ComputeState`.
283    let collection = compute_state.expect_collection_mut(sink_id);
284    collection.sink_write_frontier = Some(sink_frontier);
285
286    Rc::new((persist_token, mint_token, write_token, append_token))
287}
288
289/// Generic wrapper around ok/err pairs (e.g. streams, frontiers), to simplify code dealing with
290/// such pairs.
291struct OkErr<O, E> {
292    ok: O,
293    err: E,
294}
295
296impl<O, E> OkErr<O, E> {
297    fn new(ok: O, err: E) -> Self {
298        Self { ok, err }
299    }
300}
301
302impl OkErr<Antichain<Timestamp>, Antichain<Timestamp>> {
303    fn new_frontiers() -> Self {
304        Self {
305            ok: Antichain::from_elem(Timestamp::MIN),
306            err: Antichain::from_elem(Timestamp::MIN),
307        }
308    }
309
310    /// Return the overall frontier, i.e., the minimum of `ok` and `err`.
311    fn frontier(&self) -> &Antichain<Timestamp> {
312        if PartialOrder::less_equal(&self.ok, &self.err) {
313            &self.ok
314        } else {
315            &self.err
316        }
317    }
318}
319
320/// Advance the given `frontier` to `new`, if the latter one is greater.
321///
322/// Returns whether `frontier` was advanced.
323fn advance(frontier: &mut Antichain<Timestamp>, new: Antichain<Timestamp>) -> bool {
324    if PartialOrder::less_than(frontier, &new) {
325        *frontier = new;
326        true
327    } else {
328        false
329    }
330}
331
332/// A persist API specialized to a single collection.
333#[derive(Clone)]
334struct PersistApi {
335    persist_clients: Arc<PersistClientCache>,
336    collection: CollectionMetadata,
337    shard_name: String,
338    purpose: String,
339}
340
341impl PersistApi {
342    async fn open_client(&self) -> PersistClient {
343        self.persist_clients
344            .open(self.collection.persist_location.clone())
345            .await
346            .unwrap_or_else(|error| panic!("error opening persist client: {error}"))
347    }
348
349    async fn open_writer(&self) -> WriteHandle<SourceData, (), Timestamp, StorageDiff> {
350        self.open_client()
351            .await
352            .open_writer(
353                self.collection.data_shard,
354                Arc::new(self.collection.relation_desc.clone()),
355                Arc::new(UnitSchema),
356                Diagnostics {
357                    shard_name: self.shard_name.clone(),
358                    handle_purpose: self.purpose.clone(),
359                },
360            )
361            .await
362            .unwrap_or_else(|error| panic!("error opening persist writer: {error}"))
363    }
364
365    async fn open_metrics(&self) -> SinkMetrics {
366        let client = self.open_client().await;
367        client.metrics().sink.clone()
368    }
369}
370
371/// Instantiate a persist source reading back the `target` collection.
372fn persist_source<'s>(
373    scope: Scope<'s, Timestamp>,
374    sink_id: GlobalId,
375    target: CollectionMetadata,
376    compute_state: &ComputeState,
377    start_signal: StartSignal,
378) -> (PersistStreams<'s>, Vec<PressOnDropButton>) {
379    // There is no guarantee that the sink as-of is beyond the persist shard's since. If it isn't,
380    // instantiating a `persist_source` with it would panic. So instead we leave it to
381    // `persist_source` to select an appropriate as-of. We only care about times beyond the current
382    // shard upper anyway.
383    //
384    // TODO(teskje): Ideally we would select the as-of as `join(sink_as_of, since, upper)`, to
385    // allow `persist_source` to omit as much historical detail as possible. However, we don't know
386    // the shard frontiers and we cannot get them here as that requires an `async` context. We
387    // should consider extending the `persist_source` API to allow as-of selection based on the
388    // shard's current frontiers.
389    let as_of = None;
390
391    let until = Antichain::new();
392    let map_filter_project = None;
393
394    let (ok_stream, err_stream, token) = mz_storage_operators::persist_source::persist_source(
395        scope,
396        sink_id,
397        Arc::clone(&compute_state.persist_clients),
398        &compute_state.txns_ctx,
399        target,
400        None,
401        as_of,
402        SnapshotMode::Include,
403        until,
404        map_filter_project,
405        compute_state.dataflow_max_inflight_bytes(),
406        start_signal,
407        ErrorHandler::Halt("compute persist sink"),
408    );
409
410    let streams = OkErr::new(ok_stream, err_stream);
411    (streams, token)
412}
413
414/// A description for a batch of updates to be written.
415///
416/// Batch descriptions are produced by the `mint` operator and consumed by the `write` and `append`
417/// operators, where they inform which batches should be written or appended, respectively.
418///
419/// Each batch description also contains the index of its "append worker", i.e. the worker that is
420/// responsible for appending the written batches to the output shard.
421#[derive(Clone, Serialize, Deserialize)]
422struct BatchDescription {
423    lower: Antichain<Timestamp>,
424    upper: Antichain<Timestamp>,
425    append_worker: usize,
426}
427
428impl BatchDescription {
429    fn new(lower: Antichain<Timestamp>, upper: Antichain<Timestamp>, append_worker: usize) -> Self {
430        assert!(PartialOrder::less_than(&lower, &upper));
431        Self {
432            lower,
433            upper,
434            append_worker,
435        }
436    }
437}
438
439impl std::fmt::Debug for BatchDescription {
440    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
441        write!(
442            f,
443            "({:?}, {:?})@{}",
444            self.lower.elements(),
445            self.upper.elements(),
446            self.append_worker,
447        )
448    }
449}
450
451/// Construct a name for the given sub-operator.
452fn operator_name(sink_id: GlobalId, sub_operator: &str) -> String {
453    format!("mv_sink({sink_id})::{sub_operator}")
454}
455
456/// Implementation of the `mint` operator.
457mod mint {
458    use super::*;
459
460    /// Render the `mint` operator.
461    ///
462    /// The parameters passed in are:
463    ///  * `sink_id`: The `GlobalId` of the sink export.
464    ///  * `persist_api`: An object providing access to the output persist shard.
465    ///  * `as_of`: The first time for which the sink may produce output.
466    ///  * `read_only_tx`: A receiver that reports the sink is in read-only mode.
467    ///  * `desired`: The ok/err streams that should be sinked to persist.
468    pub fn render<'s>(
469        sink_id: GlobalId,
470        persist_api: PersistApi,
471        as_of: Antichain<Timestamp>,
472        mut read_only_rx: watch::Receiver<bool>,
473        desired: DesiredStreams<'s>,
474    ) -> (
475        DesiredStreams<'s>,
476        DescsStream<'s>,
477        SharedSinkFrontier,
478        PressOnDropButton,
479    ) {
480        let scope = desired.ok.scope();
481        let worker_id = scope.index();
482        let worker_count = scope.peers();
483
484        // Determine the active worker for the mint operator.
485        let active_worker_id = usize::cast_from(sink_id.hashed()) % scope.peers();
486
487        let sink_frontier = Rc::new(RefCell::new(Antichain::from_elem(Timestamp::MIN)));
488        let shared_frontier = Rc::clone(&sink_frontier);
489
490        let name = operator_name(sink_id, "mint");
491        let mut op = OperatorBuilder::new(name, scope);
492
493        let (ok_output, ok_stream) = op.new_output::<CapacityContainerBuilder<_>>();
494        let (err_output, err_stream) = op.new_output::<CapacityContainerBuilder<_>>();
495        let desired_outputs = OkErr::new(ok_output, err_output);
496        let desired_output_streams = OkErr::new(ok_stream, err_stream);
497
498        let (desc_output, desc_output_stream) = op.new_output::<CapacityContainerBuilder<_>>();
499
500        let mut desired_inputs = OkErr {
501            ok: op.new_input_for(desired.ok, Pipeline, &desired_outputs.ok),
502            err: op.new_input_for(desired.err, Pipeline, &desired_outputs.err),
503        };
504
505        let button = op.build(move |capabilities| async move {
506            // Passing through the `desired` streams only requires data capabilities, so we can
507            // immediately drop their initial capabilities here.
508            let [_, _, desc_cap]: [_; 3] =
509                capabilities.try_into().expect("one capability per output");
510
511            // Non-active workers just pass the `desired` and `persist` data through.
512            if worker_id != active_worker_id {
513                drop(desc_cap);
514                shared_frontier.borrow_mut().clear();
515
516                loop {
517                    tokio::select! {
518                        Some(event) = desired_inputs.ok.next() => {
519                            if let Event::Data(cap, mut data) = event {
520                                desired_outputs.ok.give_container(&cap, &mut data);
521                            }
522                        }
523                        Some(event) = desired_inputs.err.next() => {
524                            if let Event::Data(cap, mut data) = event {
525                                desired_outputs.err.give_container(&cap, &mut data);
526                            }
527                        }
528                        // All inputs are exhausted, so we can shut down.
529                        else => return,
530                    }
531                }
532            }
533
534            let mut cap_set = CapabilitySet::from_elem(desc_cap);
535
536            let read_only = *read_only_rx.borrow_and_update();
537            let mut state = State::new(sink_id, worker_count, as_of, read_only);
538
539            // Create a stream that reports advancements of the target shard's frontier and updates
540            // the shared sink frontier.
541            //
542            // We collect the persist frontier from a write handle directly, rather than inspecting
543            // the `persist` stream, because the latter has two annoying glitches:
544            //  (a) It starts at the shard's read frontier, not its write frontier.
545            //  (b) It can lag behind if there are spikes in ingested data.
546            let mut persist_frontiers = pin!(async_stream::stream! {
547                let mut writer = persist_api.open_writer().await;
548                let mut frontier = Antichain::from_elem(Timestamp::MIN);
549                while !frontier.is_empty() {
550                    writer.wait_for_upper_past(&frontier).await;
551                    frontier = writer.upper().clone();
552                    shared_frontier.borrow_mut().clone_from(&frontier);
553                    yield frontier.clone();
554                }
555            });
556
557            loop {
558                // Read from the inputs, pass through all data to the respective outputs, and keep
559                // track of the input frontiers. When a frontier advances we might have to mint a
560                // new batch description.
561                let maybe_desc = tokio::select! {
562                    Some(event) = desired_inputs.ok.next() => {
563                        match event {
564                            Event::Data(cap, mut data) => {
565                                desired_outputs.ok.give_container(&cap, &mut data);
566                                None
567                            }
568                            Event::Progress(frontier) => {
569                                state.advance_desired_ok_frontier(frontier);
570                                state.maybe_mint_batch_description()
571                            }
572                        }
573                    }
574                    Some(event) = desired_inputs.err.next() => {
575                        match event {
576                            Event::Data(cap, mut data) => {
577                                desired_outputs.err.give_container(&cap, &mut data);
578                                None
579                            }
580                            Event::Progress(frontier) => {
581                                state.advance_desired_err_frontier(frontier);
582                                state.maybe_mint_batch_description()
583                            }
584                        }
585                    }
586                    Some(frontier) = persist_frontiers.next() => {
587                        state.advance_persist_frontier(frontier);
588                        state.maybe_mint_batch_description()
589                    }
590                    Ok(()) = read_only_rx.changed(), if read_only => {
591                        state.allow_writes();
592                        state.maybe_mint_batch_description()
593                    }
594                    // All inputs are exhausted, so we can shut down.
595                    else => return,
596                };
597
598                if let Some(desc) = maybe_desc {
599                    let lower_ts = *desc.lower.as_option().expect("not empty");
600                    let cap = cap_set.delayed(&lower_ts);
601                    desc_output.give(&cap, desc);
602
603                    // We only emit strictly increasing `lower`s, so we can let our output frontier
604                    // advance beyond the current `lower`.
605                    cap_set.downgrade([lower_ts.step_forward()]);
606                } else {
607                    // The next emitted `lower` will be at least the `persist` frontier, so we can
608                    // advance our output frontier as far.
609                    let _ = cap_set.try_downgrade(state.persist_frontier.iter());
610                }
611            }
612        });
613
614        (
615            desired_output_streams,
616            desc_output_stream,
617            sink_frontier,
618            button.press_on_drop(),
619        )
620    }
621
622    /// State maintained by the `mint` operator.
623    struct State {
624        sink_id: GlobalId,
625        /// The number of workers in the Timely cluster.
626        worker_count: usize,
627        /// The frontiers of the `desired` inputs.
628        desired_frontiers: OkErr<Antichain<Timestamp>, Antichain<Timestamp>>,
629        /// The frontier of the target persist shard.
630        persist_frontier: Antichain<Timestamp>,
631        /// The append worker for the next batch description, chosen in round-robin fashion.
632        next_append_worker: usize,
633        /// The last `lower` we have emitted in a batch description, if any. Whenever the
634        /// `persist_frontier` moves beyond this frontier, we need to mint a new description.
635        last_lower: Option<Antichain<Timestamp>>,
636        /// Whether we are operating in read-only mode.
637        ///
638        /// In read-only mode, minting of batch descriptions is disabled.
639        read_only: bool,
640    }
641
642    impl State {
643        fn new(
644            sink_id: GlobalId,
645            worker_count: usize,
646            as_of: Antichain<Timestamp>,
647            read_only: bool,
648        ) -> Self {
649            // Initializing `persist_frontier` to the `as_of` ensures that the first minted batch
650            // description will have a `lower` of `as_of` or beyond, and thus that we don't spend
651            // work needlessly writing batches at previous times.
652            let persist_frontier = as_of;
653
654            Self {
655                sink_id,
656                worker_count,
657                desired_frontiers: OkErr::new_frontiers(),
658                persist_frontier,
659                next_append_worker: 0,
660                last_lower: None,
661                read_only,
662            }
663        }
664
665        fn trace<S: AsRef<str>>(&self, message: S) {
666            let message = message.as_ref();
667            trace!(
668                sink_id = %self.sink_id,
669                desired_frontier = ?self.desired_frontiers.frontier().elements(),
670                persist_frontier = ?self.persist_frontier.elements(),
671                last_lower = ?self.last_lower.as_ref().map(|f| f.elements()),
672                message,
673            );
674        }
675
676        fn advance_desired_ok_frontier(&mut self, frontier: Antichain<Timestamp>) {
677            if advance(&mut self.desired_frontiers.ok, frontier) {
678                self.trace("advanced `desired` ok frontier");
679            }
680        }
681
682        fn advance_desired_err_frontier(&mut self, frontier: Antichain<Timestamp>) {
683            if advance(&mut self.desired_frontiers.err, frontier) {
684                self.trace("advanced `desired` err frontier");
685            }
686        }
687
688        fn advance_persist_frontier(&mut self, frontier: Antichain<Timestamp>) {
689            if advance(&mut self.persist_frontier, frontier) {
690                self.trace("advanced `persist` frontier");
691            }
692        }
693
694        fn allow_writes(&mut self) {
695            if self.read_only {
696                self.read_only = false;
697                self.trace("disabled read-only mode");
698            }
699        }
700
701        fn maybe_mint_batch_description(&mut self) -> Option<BatchDescription> {
702            let desired_frontier = self.desired_frontiers.frontier();
703            let persist_frontier = &self.persist_frontier;
704
705            // We only mint new batch descriptions when:
706            //  1. We are _not_ in read-only mode.
707            //  2. The `desired` frontier is ahead of the `persist` frontier.
708            //  3. The `persist` frontier advanced since we last emitted a batch description.
709            let desired_ahead = PartialOrder::less_than(persist_frontier, desired_frontier);
710            let persist_advanced = self.last_lower.as_ref().map_or(true, |lower| {
711                PartialOrder::less_than(lower, persist_frontier)
712            });
713
714            if self.read_only || !desired_ahead || !persist_advanced {
715                return None;
716            }
717
718            let lower = persist_frontier.clone();
719            let upper = desired_frontier.clone();
720            let append_worker = self.next_append_worker;
721            let desc = BatchDescription::new(lower, upper, append_worker);
722
723            self.next_append_worker = (append_worker + 1) % self.worker_count;
724            self.last_lower = Some(desc.lower.clone());
725
726            self.trace(format!("minted batch description: {desc:?}"));
727            Some(desc)
728        }
729    }
730}
731
732/// Implementation of the `write` operator.
733mod write {
734    use super::*;
735
736    /// Render the `write` operator.
737    ///
738    /// The parameters passed in are:
739    ///  * `sink_id`: The `GlobalId` of the sink export.
740    ///  * `persist_api`: An object providing access to the output persist shard.
741    ///  * `as_of`: The first time for which the sink may produce output.
742    ///  * `desired`: The ok/err streams that should be sinked to persist.
743    ///  * `persist`: The ok/err streams read back from the output persist shard.
744    ///  * `descs`: The stream of batch descriptions produced by the `mint` operator.
745    pub fn render<'s>(
746        sink_id: GlobalId,
747        persist_api: PersistApi,
748        as_of: Antichain<Timestamp>,
749        desired: DesiredStreams<'s>,
750        persist: PersistStreams<'s>,
751        descs: DescsStream<'s>,
752        worker_config: Rc<ConfigSet>,
753    ) -> (BatchesStream<'s>, PressOnDropButton) {
754        let scope = desired.ok.scope();
755        let worker_id = scope.index();
756
757        let name = operator_name(sink_id, "write");
758        let mut op = OperatorBuilder::new(name, scope.clone());
759
760        let mut logging = None;
761        if let (Some(compute_logger), Some(differential_logger)) = (
762            scope.worker().logger_for("materialize/compute"),
763            scope.worker().logger_for("differential/arrange"),
764        ) {
765            let operator_info = op.operator_info();
766            logging = Some(Logging::new(
767                compute_logger,
768                differential_logger.into(),
769                operator_info.global_id,
770                operator_info.address.to_vec(),
771            ));
772        }
773
774        let (batches_output, batches_output_stream) =
775            op.new_output::<CapacityContainerBuilder<_>>();
776
777        // It is important that we exchange the `desired` and `persist` data the same way, so
778        // updates that cancel each other out end up on the same worker.
779        let exchange_ok = |(d, _, _): &(Row, Timestamp, Diff)| d.hashed();
780        let exchange_err = |(d, _, _): &(DataflowError, Timestamp, Diff)| d.hashed();
781
782        let mut desired_inputs = OkErr::new(
783            op.new_disconnected_input(desired.ok, Exchange::new(exchange_ok)),
784            op.new_disconnected_input(desired.err, Exchange::new(exchange_err)),
785        );
786        let mut persist_inputs = OkErr::new(
787            op.new_disconnected_input(persist.ok, Exchange::new(exchange_ok)),
788            op.new_disconnected_input(persist.err, Exchange::new(exchange_err)),
789        );
790        let mut descs_input = op.new_input_for(descs.broadcast(), Pipeline, &batches_output);
791
792        let button = op.build(move |capabilities| async move {
793            // We will use the data capabilities from the `descs` input to produce output, so no
794            // need to hold onto the initial capabilities.
795            drop(capabilities);
796
797            let writer = persist_api.open_writer().await;
798            let sink_metrics = persist_api.open_metrics().await;
799            let mut state = State::new(
800                sink_id,
801                worker_id,
802                writer,
803                sink_metrics,
804                logging,
805                as_of,
806                &worker_config,
807            );
808
809            loop {
810                // Read from the inputs, extract `desired` updates as positive contributions to
811                // `correction` and `persist` updates as negative contributions. If either the
812                // `desired` or `persist` frontier advances, or if we receive a new batch description,
813                // we might have to write a new batch.
814                let maybe_batch = tokio::select! {
815                    Some(event) = desired_inputs.ok.next() => {
816                        match event {
817                            Event::Data(_cap, mut data) => {
818                                state.corrections.ok.insert(&mut data);
819                                None
820                            }
821                            Event::Progress(frontier) => {
822                                state.advance_desired_ok_frontier(frontier);
823                                state.maybe_write_batch().await
824                            }
825                        }
826                    }
827                    Some(event) = desired_inputs.err.next() => {
828                        match event {
829                            Event::Data(_cap, mut data) => {
830                                state.corrections.err.insert(&mut data);
831                                None
832                            }
833                            Event::Progress(frontier) => {
834                                state.advance_desired_err_frontier(frontier);
835                                state.maybe_write_batch().await
836                            }
837                        }
838                    }
839                    Some(event) = persist_inputs.ok.next() => {
840                        match event {
841                            Event::Data(_cap, mut data) => {
842                                state.corrections.ok.insert_negated(&mut data);
843                                None
844                            }
845                            Event::Progress(frontier) => {
846                                state.advance_persist_ok_frontier(frontier);
847                                state.maybe_write_batch().await
848                            }
849                        }
850                    }
851                    Some(event) = persist_inputs.err.next() => {
852                        match event {
853                            Event::Data(_cap, mut data) => {
854                                state.corrections.err.insert_negated(&mut data);
855                                None
856                            }
857                            Event::Progress(frontier) => {
858                                state.advance_persist_err_frontier(frontier);
859                                state.maybe_write_batch().await
860                            }
861                        }
862                    }
863                    Some(event) = descs_input.next() => {
864                        match event {
865                            Event::Data(cap, data) => {
866                                for desc in data {
867                                    state.absorb_batch_description(desc, cap.clone());
868                                }
869                                state.maybe_write_batch().await
870                            }
871                            Event::Progress(_frontier) => None,
872                        }
873                    }
874                    // All inputs are exhausted, so we can shut down.
875                    else => return,
876                };
877
878                if let Some((index, batch, cap)) = maybe_batch {
879                    batches_output.give(&cap, (index, batch));
880                }
881            }
882        });
883
884        (batches_output_stream, button.press_on_drop())
885    }
886
887    /// State maintained by the `write` operator.
888    struct State {
889        sink_id: GlobalId,
890        worker_id: usize,
891        persist_writer: WriteHandle<SourceData, (), Timestamp, StorageDiff>,
892        /// Contains `desired - persist`, reflecting the updates we would like to commit to
893        /// `persist` in order to "correct" it to track `desired`. This collection is only modified
894        /// by updates received from either the `desired` or `persist` inputs.
895        corrections: OkErr<Correction<Row>, Correction<DataflowError>>,
896        /// The frontiers of the `desired` inputs.
897        desired_frontiers: OkErr<Antichain<Timestamp>, Antichain<Timestamp>>,
898        /// The frontiers of the `persist` inputs.
899        ///
900        /// Note that this is _not_ the same as the write frontier of the output persist shard! It
901        /// usually is, but during snapshot processing, these frontiers will start at the shard's
902        /// read frontier, so they can be beyond its write frontier. This is important as it means
903        /// we must not discard batch descriptions based on these persist frontiers: A batch
904        /// description might still be valid even if its `lower` is before the persist frontiers we
905        /// observe.
906        persist_frontiers: OkErr<Antichain<Timestamp>, Antichain<Timestamp>>,
907        /// The current valid batch description and associated output capability, if any.
908        batch_description: Option<(BatchDescription, Capability<Timestamp>)>,
909        /// A request to force a consolidation of `corrections` once both `desired_frontiers` and
910        /// `persist_frontiers` become greater than the given frontier.
911        ///
912        /// Normally we force a consolidation whenever we write a batch, but there are periods
913        /// (like read-only mode) when that doesn't happen, and we need to manually force
914        /// consolidation instead. Currently this is only used to ensure we quickly get rid of the
915        /// snapshot updates.
916        force_consolidation_after: Option<Antichain<Timestamp>>,
917    }
918
919    impl State {
920        fn new(
921            sink_id: GlobalId,
922            worker_id: usize,
923            persist_writer: WriteHandle<SourceData, (), Timestamp, StorageDiff>,
924            metrics: SinkMetrics,
925            logging: Option<Logging>,
926            as_of: Antichain<Timestamp>,
927            worker_config: &ConfigSet,
928        ) -> Self {
929            let worker_metrics = metrics.for_worker(worker_id);
930
931            // Force a consolidation of `corrections` after the snapshot updates have been fully
932            // processed, to ensure we get rid of those as quickly as possible.
933            let force_consolidation_after = Some(as_of.clone());
934
935            let mut state = Self {
936                sink_id,
937                worker_id,
938                persist_writer,
939                corrections: OkErr::new(
940                    Correction::new(
941                        metrics.clone(),
942                        worker_metrics.clone(),
943                        logging.clone(),
944                        worker_config,
945                    ),
946                    Correction::new(metrics, worker_metrics, logging, worker_config),
947                ),
948                desired_frontiers: OkErr::new_frontiers(),
949                persist_frontiers: OkErr::new_frontiers(),
950                batch_description: None,
951                force_consolidation_after,
952            };
953
954            // Immediately advance the persist frontier tracking to the `as_of`.
955            // This is important to ensure the persist sink doesn't get stuck if the output shard's
956            // initial frontier is less than the `as_of`. The `mint` operator first emits a batch
957            // description with `lower = as_of`, and the `write` operator only emits a batch when
958            // its observed persist frontier is >= the batch description's `lower`, which (assuming
959            // no other writers) would be never if we didn't advance the observed persist frontier
960            // to the `as_of`.
961            if MV_SINK_ADVANCE_PERSIST_FRONTIERS.get(worker_config) {
962                state.advance_persist_ok_frontier(as_of.clone());
963                state.advance_persist_err_frontier(as_of);
964            }
965
966            state
967        }
968
969        fn trace<S: AsRef<str>>(&self, message: S) {
970            let message = message.as_ref();
971            trace!(
972                sink_id = %self.sink_id,
973                worker = %self.worker_id,
974                desired_frontier = ?self.desired_frontiers.frontier().elements(),
975                persist_frontier = ?self.persist_frontiers.frontier().elements(),
976                batch_description = ?self.batch_description.as_ref().map(|(d, _)| d),
977                message,
978            );
979        }
980
981        fn advance_desired_ok_frontier(&mut self, frontier: Antichain<Timestamp>) {
982            if advance(&mut self.desired_frontiers.ok, frontier) {
983                self.apply_desired_frontier_advancement();
984                self.trace("advanced `desired` ok frontier");
985            }
986        }
987
988        fn advance_desired_err_frontier(&mut self, frontier: Antichain<Timestamp>) {
989            if advance(&mut self.desired_frontiers.err, frontier) {
990                self.apply_desired_frontier_advancement();
991                self.trace("advanced `desired` err frontier");
992            }
993        }
994
995        fn advance_persist_ok_frontier(&mut self, frontier: Antichain<Timestamp>) {
996            if advance(&mut self.persist_frontiers.ok, frontier) {
997                self.apply_persist_frontier_advancement();
998                self.trace("advanced `persist` ok frontier");
999            }
1000        }
1001
1002        fn advance_persist_err_frontier(&mut self, frontier: Antichain<Timestamp>) {
1003            if advance(&mut self.persist_frontiers.err, frontier) {
1004                self.apply_persist_frontier_advancement();
1005                self.trace("advanced `persist` err frontier");
1006            }
1007        }
1008
1009        /// Apply the effects of a previous `desired` frontier advancement.
1010        fn apply_desired_frontier_advancement(&mut self) {
1011            self.maybe_force_consolidation();
1012        }
1013
1014        /// Apply the effects of a previous `persist` frontier advancement.
1015        fn apply_persist_frontier_advancement(&mut self) {
1016            let frontier = self.persist_frontiers.frontier();
1017
1018            // We will only emit times at or after the `persist` frontier, so now is a good time to
1019            // advance the times of stashed updates.
1020            self.corrections.ok.advance_since(frontier.clone());
1021            self.corrections.err.advance_since(frontier.clone());
1022
1023            self.maybe_force_consolidation();
1024        }
1025
1026        /// If the current consolidation request has become applicable, apply it.
1027        fn maybe_force_consolidation(&mut self) {
1028            let Some(request) = &self.force_consolidation_after else {
1029                return;
1030            };
1031
1032            let desired_frontier = self.desired_frontiers.frontier();
1033            let persist_frontier = self.persist_frontiers.frontier();
1034            if PartialOrder::less_than(request, desired_frontier)
1035                && PartialOrder::less_than(request, persist_frontier)
1036            {
1037                self.trace("forcing correction consolidation");
1038                self.corrections.ok.consolidate_at_since();
1039                self.corrections.err.consolidate_at_since();
1040
1041                // Remove the consolidation request, now that we have fulfilled it.
1042                self.force_consolidation_after = None;
1043            }
1044        }
1045
1046        fn absorb_batch_description(&mut self, desc: BatchDescription, cap: Capability<Timestamp>) {
1047            // The incoming batch description is outdated if we already have a batch description
1048            // with a greater `lower`.
1049            //
1050            // Note that we cannot assume a description is outdated based on the comparison of its
1051            // `lower` with the `persist_frontier`. The persist frontier observed by the `write`
1052            // operator is initialized with the shard's read frontier, so it can be greater than
1053            // the shard's write frontier.
1054            if let Some((prev, _)) = &self.batch_description {
1055                if PartialOrder::less_than(&desc.lower, &prev.lower) {
1056                    self.trace(format!("skipping outdated batch description: {desc:?}"));
1057                    return;
1058                }
1059            }
1060
1061            self.batch_description = Some((desc, cap));
1062            self.trace("set batch description");
1063        }
1064
1065        async fn maybe_write_batch(
1066            &mut self,
1067        ) -> Option<(BatchDescription, ProtoBatch, Capability<Timestamp>)> {
1068            let (desc, _cap) = self.batch_description.as_ref()?;
1069
1070            // We can write a new batch if we have seen all `persist` updates before `lower` and
1071            // all `desired` updates up to `upper`.
1072            let persist_complete =
1073                PartialOrder::less_equal(&desc.lower, self.persist_frontiers.frontier());
1074            let desired_complete =
1075                PartialOrder::less_equal(&desc.upper, self.desired_frontiers.frontier());
1076            if !persist_complete || !desired_complete {
1077                return None;
1078            }
1079
1080            let (desc, cap) = self.batch_description.take()?;
1081
1082            let ok_updates = self.corrections.ok.updates_before(&desc.upper);
1083            let err_updates = self.corrections.err.updates_before(&desc.upper);
1084
1085            let oks = ok_updates.map(|(d, t, r)| ((SourceData(Ok(d)), ()), t, r.into_inner()));
1086            let errs = err_updates.map(|(d, t, r)| ((SourceData(Err(d)), ()), t, r.into_inner()));
1087            let mut updates = oks.chain(errs).peekable();
1088
1089            // Don't write empty batches.
1090            if updates.peek().is_none() {
1091                drop(updates);
1092                self.trace("skipping empty batch");
1093                return None;
1094            }
1095
1096            let batch = self
1097                .persist_writer
1098                .batch(updates, desc.lower.clone(), desc.upper.clone())
1099                .await
1100                .expect("valid usage")
1101                .into_transmittable_batch();
1102
1103            self.trace("wrote a batch");
1104            Some((desc, batch, cap))
1105        }
1106    }
1107}
1108
1109/// Implementation of the `append` operator.
1110mod append {
1111    use super::*;
1112
1113    /// Render the `append` operator.
1114    ///
1115    /// The parameters passed in are:
1116    ///  * `sink_id`: The `GlobalId` of the sink export.
1117    ///  * `persist_api`: An object providing access to the output persist shard.
1118    ///  * `descs`: The stream of batch descriptions produced by the `mint` operator.
1119    ///  * `batches`: The stream of written batches produced by the `write` operator.
1120    pub fn render<'s>(
1121        sink_id: GlobalId,
1122        persist_api: PersistApi,
1123        descs: DescsStream<'s>,
1124        batches: BatchesStream<'s>,
1125    ) -> PressOnDropButton {
1126        let scope = descs.scope();
1127        let worker_id = scope.index();
1128
1129        let name = operator_name(sink_id, "append");
1130        let mut op = OperatorBuilder::new(name, scope);
1131
1132        // Broadcast batch descriptions to all workers, regardless of whether or not they are
1133        // responsible for the append, to give them a chance to clean up any outdated state they
1134        // might still hold.
1135        let mut descs_input = op.new_disconnected_input(descs.broadcast(), Pipeline);
1136        let mut batches_input = op.new_disconnected_input(
1137            batches,
1138            Exchange::new(move |(desc, _): &(BatchDescription, _)| {
1139                u64::cast_from(desc.append_worker)
1140            }),
1141        );
1142
1143        let button = op.build(move |_capabilities| async move {
1144            let writer = persist_api.open_writer().await;
1145            let mut state = State::new(sink_id, worker_id, writer);
1146
1147            loop {
1148                // Read from the inputs, absorb batch descriptions and batches. If the `batches`
1149                // frontier advances, or if we receive a new batch description, we might have to
1150                // append a new batch.
1151                tokio::select! {
1152                    Some(event) = descs_input.next() => {
1153                        if let Event::Data(_cap, data) = event {
1154                            for desc in data {
1155                                state.absorb_batch_description(desc).await;
1156                                state.maybe_append_batches().await;
1157                            }
1158                        }
1159                    }
1160                    Some(event) = batches_input.next() => {
1161                        match event {
1162                            Event::Data(_cap, data) => {
1163                                // The batch description is only used for routing and we ignore it
1164                                // here since we already get one from `descs_input`.
1165                                for (_desc, batch) in data {
1166                                    state.absorb_batch(batch).await;
1167                                }
1168                            }
1169                            Event::Progress(frontier) => {
1170                                state.advance_batches_frontier(frontier);
1171                                state.maybe_append_batches().await;
1172                            }
1173                        }
1174                    }
1175                    // All inputs are exhausted, so we can shut down.
1176                    else => return,
1177                }
1178            }
1179        });
1180
1181        button.press_on_drop()
1182    }
1183
1184    /// State maintained by the `append` operator.
1185    struct State {
1186        sink_id: GlobalId,
1187        worker_id: usize,
1188        persist_writer: WriteHandle<SourceData, (), Timestamp, StorageDiff>,
1189        /// The current input frontier of `batches`.
1190        batches_frontier: Antichain<Timestamp>,
1191        /// The greatest observed `lower` from both `descs` and `batches`.
1192        lower: Antichain<Timestamp>,
1193        /// The batch description for `lower`, if any.
1194        batch_description: Option<BatchDescription>,
1195        /// Batches received for `lower`.
1196        batches: Vec<Batch<SourceData, (), Timestamp, StorageDiff>>,
1197    }
1198
1199    impl State {
1200        fn new(
1201            sink_id: GlobalId,
1202            worker_id: usize,
1203            persist_writer: WriteHandle<SourceData, (), Timestamp, StorageDiff>,
1204        ) -> Self {
1205            Self {
1206                sink_id,
1207                worker_id,
1208                persist_writer,
1209                batches_frontier: Antichain::from_elem(Timestamp::MIN),
1210                lower: Antichain::from_elem(Timestamp::MIN),
1211                batch_description: None,
1212                batches: Default::default(),
1213            }
1214        }
1215
1216        fn trace<S: AsRef<str>>(&self, message: S) {
1217            let message = message.as_ref();
1218            trace!(
1219                sink_id = %self.sink_id,
1220                worker = %self.worker_id,
1221                batches_frontier = ?self.batches_frontier.elements(),
1222                lower = ?self.lower.elements(),
1223                batch_description = ?self.batch_description,
1224                message,
1225            );
1226        }
1227
1228        fn advance_batches_frontier(&mut self, frontier: Antichain<Timestamp>) {
1229            if advance(&mut self.batches_frontier, frontier) {
1230                self.trace("advanced `batches` frontier");
1231            }
1232        }
1233
1234        /// Advance the current `lower`.
1235        ///
1236        /// Discards all currently stashed batches and batch descriptions, assuming that they are
1237        /// now invalid.
1238        async fn advance_lower(&mut self, frontier: Antichain<Timestamp>) {
1239            assert!(PartialOrder::less_than(&self.lower, &frontier));
1240
1241            self.lower = frontier;
1242            self.batch_description = None;
1243
1244            // Remove stashed batches, cleaning up those we didn't append.
1245            for batch in self.batches.drain(..) {
1246                batch.delete().await;
1247            }
1248
1249            self.trace("advanced `lower`");
1250        }
1251
1252        /// Absorb the given batch description into the state, provided it is not outdated.
1253        async fn absorb_batch_description(&mut self, desc: BatchDescription) {
1254            if PartialOrder::less_than(&self.lower, &desc.lower) {
1255                self.advance_lower(desc.lower.clone()).await;
1256            } else if &self.lower != &desc.lower {
1257                self.trace(format!("skipping outdated batch description: {desc:?}"));
1258                return;
1259            }
1260
1261            if desc.append_worker == self.worker_id {
1262                self.batch_description = Some(desc);
1263                self.trace("set batch description");
1264            }
1265        }
1266
1267        /// Absorb the given batch into the state, provided it is not outdated.
1268        async fn absorb_batch(&mut self, batch: ProtoBatch) {
1269            let batch = self.persist_writer.batch_from_transmittable_batch(batch);
1270            if PartialOrder::less_than(&self.lower, batch.lower()) {
1271                self.advance_lower(batch.lower().clone()).await;
1272            } else if &self.lower != batch.lower() {
1273                self.trace(format!(
1274                    "skipping outdated batch: ({:?}, {:?})",
1275                    batch.lower().elements(),
1276                    batch.upper().elements(),
1277                ));
1278
1279                // Ensure the batch's data gets properly cleaned up before dropping it.
1280                batch.delete().await;
1281                return;
1282            }
1283
1284            self.batches.push(batch);
1285            self.trace("absorbed a batch");
1286        }
1287
1288        async fn maybe_append_batches(&mut self) {
1289            let batches_complete = PartialOrder::less_than(&self.lower, &self.batches_frontier);
1290            if !batches_complete {
1291                return;
1292            }
1293
1294            let Some(desc) = self.batch_description.take() else {
1295                return;
1296            };
1297
1298            let new_lower = match self.append_batches(desc).await {
1299                Ok(shard_upper) => {
1300                    self.trace("appended a batch");
1301                    shard_upper
1302                }
1303                Err(shard_upper) => {
1304                    // Failing the append is expected in the presence of concurrent replicas. There
1305                    // is nothing special to do here: The self-correcting feedback mechanism
1306                    // ensures that we observe the concurrent changes, compute their consequences,
1307                    // and append them at a future time.
1308                    self.trace(format!(
1309                        "append failed due to `lower` mismatch: {:?}",
1310                        shard_upper.elements(),
1311                    ));
1312                    shard_upper
1313                }
1314            };
1315
1316            self.advance_lower(new_lower).await;
1317        }
1318
1319        /// Append the current `batches` to the output shard.
1320        ///
1321        /// Returns whether the append was successful or not, and the current shard upper in either
1322        /// case.
1323        ///
1324        /// This method advances the shard upper to the batch `lower` if necessary. This is the
1325        /// mechanism that brings the shard upper to the sink as-of when appending the initial
1326        /// batch.
1327        ///
1328        /// An alternative mechanism for bringing the shard upper to the sink as-of would be making
1329        /// a single append at operator startup. The reason we are doing it here instead is that it
1330        /// simplifies the implementation of read-only mode. In read-only mode we have to defer any
1331        /// persist writes, including the initial upper bump. Having only a single place that
1332        /// performs writes makes it easy to ensure we are doing that correctly.
1333        async fn append_batches(
1334            &mut self,
1335            desc: BatchDescription,
1336        ) -> Result<Antichain<Timestamp>, Antichain<Timestamp>> {
1337            let (lower, upper) = (desc.lower, desc.upper);
1338            let mut to_append: Vec<_> = self.batches.iter_mut().collect();
1339
1340            loop {
1341                let result = self
1342                    .persist_writer
1343                    .compare_and_append_batch(&mut to_append, lower.clone(), upper.clone(), true)
1344                    .await
1345                    .expect("valid usage");
1346
1347                match result {
1348                    Ok(()) => return Ok(upper),
1349                    Err(mismatch) if PartialOrder::less_than(&mismatch.current, &lower) => {
1350                        advance_shard_upper(&mut self.persist_writer, lower.clone()).await;
1351
1352                        // At this point the shard's since and upper are likely the same, a state
1353                        // that is likely to hit edge-cases in logic reasoning about frontiers.
1354                        fail::fail_point!("mv_advanced_upper");
1355                    }
1356                    Err(mismatch) => return Err(mismatch.current),
1357                }
1358            }
1359        }
1360    }
1361
1362    /// Advance the frontier of the given writer's shard to at least the given `upper`.
1363    async fn advance_shard_upper(
1364        persist_writer: &mut WriteHandle<SourceData, (), Timestamp, StorageDiff>,
1365        upper: Antichain<Timestamp>,
1366    ) {
1367        let empty_updates: &[((SourceData, ()), Timestamp, StorageDiff)] = &[];
1368        let lower = Antichain::from_elem(Timestamp::MIN);
1369        persist_writer
1370            .append(empty_updates, lower, upper)
1371            .await
1372            .expect("valid usage")
1373            .expect("should always succeed");
1374    }
1375}