1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

//! A dataflow sink that writes input records to a persist shard.
//!
//! This implementation is both parallel and self-correcting.
//!
//!  * parallel: Multiple workers can participate in writing updates for the same times, letting
//!    sink throughput scale with the number of workers allocated to the replica.
//!  * self-correcting: The sink continually compares the contents of the persist shard with the
//!    contents of the input collection and writes down the difference. If the persist shard ends
//!    up with undesired contents for any reason, this is corrected the next time the sink manages
//!    to append to the shard.
//!
//! ### Operators
//!
//! The persist sink consists of a graph of operators.
//!
//!    desired                    persist <---------------.
//!       |                          |                    |
//!       |                          |                    |
//!       |---------------------.    |                    |
//!       |                     |    |                    |
//!       |                     |    |                    |
//!       v                     v    v                    |
//!   +--------+              +--------+              +--------+
//!   |  mint  | --descs-.--> | write  | --batches--> | append |
//!   +--------+          \   +--------+          .-> +--------+
//!                        \_____________________/
//!
//!  * `mint` mints batch descriptions, i.e., `(lower, upper)` bounds of batches that should be
//!    written. The persist API requires that all workers write batches with the same bounds, so
//!    they can be appended as a single logical batch. To ensure this, the `mint` operator only
//!    runs on a single worker that broadcasts minted descriptions to all workers. Batch bounds are
//!    picked based on the frontiers of the `desired` stream and the output persist shard.
//!  * `write` stages batch data in persist, based on the batch descriptions received from the
//!    `mint` operator, but without appending it to the persist shard. This is a multi-worker
//!    operator, with each worker writing batches of the data that arrives at its local inputs. To
//!    do so it reads from the `desired` and `persist` streams and produces the difference between
//!    them to write back out, ensuring that the final contents of the persist shard match
//!    `desired`.
//!  * `append` appends the batches minted by `mint` and written by `write` to the persist shard.
//!    This is again a single-worker operator. It waits for all workers to stage their batches for
//!    a given batch description, then appends all the batches together as a single logical batch.
//!
//! Note that while the above graph suggests that `mint` and `write` both receive copies of the
//! `desired` stream, the actual implementation passes that stream through `mint` and lets `write`
//! read the passed-through stream, to avoid cloning data.
//!
//! The persist sink is written to be robust to the presence of other conflicting instances (e.g.
//! from other replicas) writing to the same persist shard. Each of the three operators needs to be
//! able to handle conflicting writes that unexpectedly change the contents of the output persist
//! shard.
//!
//! ### Frontiers
//!
//! The `desired` frontier tracks the progress of the upstream dataflow, but may be rounded up to
//! the next refresh time for dataflows that follow a refresh schedule other than "on commit".
//!
//! The `persist` frontier tracks the `upper` frontier of the target persist shard, with one
//! exception: When the `persist_source` that reads back the shard is rendered, it will start
//! reading at its `since` frontier. So if the shard's `since` is initially greater than its
//! `upper`, the `persist` frontier too will be in advance of the shard `upper`, until the `upper`
//! has caught up. To avoid getting confused by this edge case, the `mint` operator does not use
//! the `persist` stream to observe the shard frontier but keeps its own `WriteHandle` instead.
//!
//! The `descs` frontier communicates which `lower` bounds may still be emitted in batch
//! descriptions. All future batch descriptions will have a `lower` that is greater or equal to the
//! current `descs` frontier.
//!
//! The `batches` frontier communicates for which `lower` bounds batches may still be written. All
//! batches for descriptions with `lower`s less than the current `batches` frontier have already
//! been written.
//!
//! ### Invariants
//!
//! The implementation upholds several invariants that can be relied upon to simplify the
//! implementation:
//!
//!  1. `lower`s in minted batch descriptions are unique and strictly increasing. That is, the
//!     `mint` operator will never mint the same `lower` twice and a minted `lower` is always
//!     greater than any previously minted ones.
//!  2. `upper`s in minted batch descriptions are monotonically increasing.
//!  3. From (1) follows that there is always at most one "valid" batch description in flight in
//!     the operator graph. "Valid" here means that the described batch can be appended to the
//!     persist shard.
//!
//! The main simplification these invariants allow is that operators only need to keep track of the
//! most recent batch description and/or `lower`. Previous batch descriptions are not valid
//! anymore, so there is no reason to hold any state or perform any work in support of them.
//!
//! ### Read-only Mode
//!
//! The persist sink can optionally be initialized in read-only mode. In this mode it is passive
//! and avoids any writes to persist. Activating the `read_only_rx` transitions the sink into write
//! mode, where it commences normal operation.
//!
//! Read-only mode is implemented by the `mint` operator. To disable writes, the `mint` operator
//! simply avoids minting any batch descriptions. Since both the `write` and the `append` operator
//! require batch descriptions to write/append batches, this suppresses any persist communication.
//! At the same time, the `write` operator still observes changes to the `desired` and `persist`
//! collections, allowing it to keep its correction buffer up-to-date.

use std::any::Any;
use std::cell::RefCell;
use std::pin::pin;
use std::rc::Rc;
use std::sync::Arc;

use differential_dataflow::{Collection, Hashable};
use futures::StreamExt;
use mz_ore::cast::CastFrom;
use mz_persist_client::batch::{Batch, ProtoBatch};
use mz_persist_client::cache::PersistClientCache;
use mz_persist_client::metrics::SinkMetrics;
use mz_persist_client::operators::shard_source::SnapshotMode;
use mz_persist_client::write::WriteHandle;
use mz_persist_client::{Diagnostics, PersistClient};
use mz_persist_types::codec_impls::UnitSchema;
use mz_repr::{Diff, GlobalId, Row, Timestamp};
use mz_storage_types::controller::CollectionMetadata;
use mz_storage_types::errors::DataflowError;
use mz_storage_types::sources::SourceData;
use mz_timely_util::builder_async::PressOnDropButton;
use mz_timely_util::builder_async::{Event, OperatorBuilder};
use serde::{Deserialize, Serialize};
use timely::container::CapacityContainerBuilder;
use timely::dataflow::channels::pact::{Exchange, Pipeline};
use timely::dataflow::operators::{Broadcast, Capability, CapabilitySet};
use timely::dataflow::{Scope, Stream};
use timely::progress::Antichain;
use timely::PartialOrder;
use tokio::sync::watch;
use tracing::trace;

use crate::compute_state::ComputeState;
use crate::render::StartSignal;
use crate::sink::correction::Correction;

/// Type of the `desired` stream, split into `Ok` and `Err` streams.
type DesiredStreams<S> =
    OkErr<Stream<S, (Row, Timestamp, Diff)>, Stream<S, (DataflowError, Timestamp, Diff)>>;

/// Type of the `persist` stream, split into `Ok` and `Err` streams.
type PersistStreams<S> =
    OkErr<Stream<S, (Row, Timestamp, Diff)>, Stream<S, (DataflowError, Timestamp, Diff)>>;

/// Type of the `descs` stream.
type DescsStream<S> = Stream<S, BatchDescription>;

/// Type of the `batches` stream.
type BatchesStream<S> = Stream<S, ProtoBatch>;

/// Type of the shared sink write frontier.
type SharedSinkFrontier = Rc<RefCell<Antichain<Timestamp>>>;

/// Renders an MV sink writing the given desired collection into the `target` persist collection.
pub(super) fn persist_sink<S>(
    sink_id: GlobalId,
    target: &CollectionMetadata,
    ok_collection: Collection<S, Row, Diff>,
    err_collection: Collection<S, DataflowError, Diff>,
    as_of: Antichain<Timestamp>,
    compute_state: &mut ComputeState,
    start_signal: StartSignal,
) -> Rc<dyn Any>
where
    S: Scope<Timestamp = Timestamp>,
{
    let mut scope = ok_collection.scope();
    let desired = OkErr::new(ok_collection.inner, err_collection.inner);

    // Read back the persist shard.
    let (persist, persist_token) = persist_source(
        &mut scope,
        sink_id,
        target.clone(),
        compute_state,
        start_signal,
    );

    // Determine the active worker for single-worker operators.
    let active_worker_id = usize::cast_from(sink_id.hashed()) % scope.peers();

    let persist_api = PersistApi {
        persist_clients: Arc::clone(&compute_state.persist_clients),
        collection: target.clone(),
        shard_name: sink_id.to_string(),
        purpose: format!("MV sink {sink_id}"),
    };

    let (desired, descs, sink_frontier, mint_token) = mint::render(
        sink_id,
        persist_api.clone(),
        as_of.clone(),
        active_worker_id,
        compute_state.read_only_rx.clone(),
        &desired,
    );

    let (batches, write_token) = write::render(
        sink_id,
        persist_api.clone(),
        as_of,
        &desired,
        &persist,
        &descs,
    );

    let append_token = append::render(sink_id, persist_api, active_worker_id, &descs, &batches);

    // Report sink frontier updates to the `ComputeState`.
    let collection = compute_state.expect_collection_mut(sink_id);
    collection.sink_write_frontier = Some(sink_frontier);

    Rc::new((persist_token, mint_token, write_token, append_token))
}

/// Generic wrapper around ok/err pairs (e.g. streams, frontiers), to simplify code dealing with
/// such pairs.
struct OkErr<O, E> {
    ok: O,
    err: E,
}

impl<O, E> OkErr<O, E> {
    fn new(ok: O, err: E) -> Self {
        Self { ok, err }
    }
}

impl OkErr<Antichain<Timestamp>, Antichain<Timestamp>> {
    fn new_frontiers() -> Self {
        Self {
            ok: Antichain::from_elem(Timestamp::MIN),
            err: Antichain::from_elem(Timestamp::MIN),
        }
    }

    /// Return the overall frontier, i.e., the minimum of `ok` and `err`.
    fn frontier(&self) -> &Antichain<Timestamp> {
        if PartialOrder::less_equal(&self.ok, &self.err) {
            &self.ok
        } else {
            &self.err
        }
    }
}

/// Advance the given `frontier` to `new`, if the latter one is greater.
///
/// Returns whether `frontier` was advanced.
fn advance(frontier: &mut Antichain<Timestamp>, new: Antichain<Timestamp>) -> bool {
    if PartialOrder::less_than(frontier, &new) {
        *frontier = new;
        true
    } else {
        false
    }
}

/// A persist API specialized to a single collection.
#[derive(Clone)]
struct PersistApi {
    persist_clients: Arc<PersistClientCache>,
    collection: CollectionMetadata,
    shard_name: String,
    purpose: String,
}

impl PersistApi {
    async fn open_client(&self) -> PersistClient {
        self.persist_clients
            .open(self.collection.persist_location.clone())
            .await
            .unwrap_or_else(|error| panic!("error opening persist client: {error}"))
    }

    async fn open_writer(&self) -> WriteHandle<SourceData, (), Timestamp, Diff> {
        self.open_client()
            .await
            .open_writer(
                self.collection.data_shard,
                Arc::new(self.collection.relation_desc.clone()),
                Arc::new(UnitSchema),
                Diagnostics {
                    shard_name: self.shard_name.clone(),
                    handle_purpose: self.purpose.clone(),
                },
            )
            .await
            .unwrap_or_else(|error| panic!("error opening persist writer: {error}"))
    }

    async fn open_metrics(&self) -> SinkMetrics {
        let client = self.open_client().await;
        client.metrics().sink.clone()
    }
}

/// Instantiate a persist source reading back the `target` collection.
fn persist_source<S>(
    scope: &mut S,
    sink_id: GlobalId,
    target: CollectionMetadata,
    compute_state: &ComputeState,
    start_signal: StartSignal,
) -> (PersistStreams<S>, Vec<PressOnDropButton>)
where
    S: Scope<Timestamp = Timestamp>,
{
    // There is no guarantee that the sink as-of is beyond the persist shard's since. If it isn't,
    // instantiating a `persist_source` with it would panic. So instead we leave it to
    // `persist_source` to select an appropriate as-of. We only care about times beyond the current
    // shard upper anyway.
    //
    // TODO(teskje): Ideally we would select the as-of as `join(sink_as_of, since, upper)`, to
    // allow `persist_source` to omit as much historical detail as possible. However, we don't know
    // the shard frontiers and we cannot get them here as that requires an `async` context. We
    // should consider extending the `persist_source` API to allow as-of selection based on the
    // shard's current frontiers.
    let as_of = None;

    let until = Antichain::new();
    let map_filter_project = None;

    let (ok_stream, err_stream, token) = mz_storage_operators::persist_source::persist_source(
        scope,
        sink_id,
        Arc::clone(&compute_state.persist_clients),
        &compute_state.txns_ctx,
        &compute_state.worker_config,
        target,
        as_of,
        SnapshotMode::Include,
        until,
        map_filter_project,
        compute_state.dataflow_max_inflight_bytes(),
        start_signal,
        |error| panic!("compute_persist_sink: {error}"),
    );

    let streams = OkErr::new(ok_stream, err_stream);
    (streams, token)
}

/// A description for a batch of updates to be written.
///
/// Batch descriptions are produced by the `mint` operator and consumed by the `write` and `append`
/// operators, where they inform which batches should be written or appended, respectively.
#[derive(Clone, Serialize, Deserialize)]
struct BatchDescription {
    lower: Antichain<Timestamp>,
    upper: Antichain<Timestamp>,
}

impl BatchDescription {
    fn new(lower: Antichain<Timestamp>, upper: Antichain<Timestamp>) -> Self {
        assert!(PartialOrder::less_than(&lower, &upper));
        Self { lower, upper }
    }
}

impl std::fmt::Debug for BatchDescription {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        write!(
            f,
            "({:?}, {:?})",
            self.lower.elements(),
            self.upper.elements()
        )
    }
}

/// Construct a name for the given sub-operator.
fn operator_name(sink_id: GlobalId, sub_operator: &str) -> String {
    format!("mv_sink({sink_id})::{sub_operator}")
}

/// Implementation of the `mint` operator.
mod mint {
    use super::*;

    /// Render the `mint` operator.
    ///
    /// The parameters passed in are:
    ///  * `sink_id`: The `GlobalId` of the sink export.
    ///  * `persist_api`: An object providing access to the output persist shard.
    ///  * `as_of`: The first time for which the sink may produce output.
    ///  * `active_worker_id`: The ID of the worker that runs this (single-threaded) operator.
    ///  * `read_only_tx`: A receiver that reports the sink is in read-only mode.
    ///  * `desired`: The ok/err streams that should be sinked to persist.
    pub fn render<S>(
        sink_id: GlobalId,
        persist_api: PersistApi,
        as_of: Antichain<Timestamp>,
        active_worker_id: usize,
        mut read_only_rx: watch::Receiver<bool>,
        desired: &DesiredStreams<S>,
    ) -> (
        DesiredStreams<S>,
        DescsStream<S>,
        SharedSinkFrontier,
        PressOnDropButton,
    )
    where
        S: Scope<Timestamp = Timestamp>,
    {
        let scope = desired.ok.scope();
        let worker_id = scope.index();

        let sink_frontier = Rc::new(RefCell::new(Antichain::from_elem(Timestamp::MIN)));
        let shared_frontier = Rc::clone(&sink_frontier);

        let name = operator_name(sink_id, "mint");
        let mut op = OperatorBuilder::new(name, scope);

        let (ok_output, ok_stream) = op.new_output::<CapacityContainerBuilder<_>>();
        let (err_output, err_stream) = op.new_output::<CapacityContainerBuilder<_>>();
        let desired_outputs = OkErr::new(ok_output, err_output);
        let desired_output_streams = OkErr::new(ok_stream, err_stream);

        let (desc_output, desc_output_stream) = op.new_output();

        let mut desired_inputs = OkErr {
            ok: op.new_input_for(&desired.ok, Pipeline, &desired_outputs.ok),
            err: op.new_input_for(&desired.err, Pipeline, &desired_outputs.err),
        };

        let button = op.build(move |capabilities| async move {
            // Passing through the `desired` streams only requires data capabilities, so we can
            // immediately drop their initial capabilities here.
            let [_, _, desc_cap]: [_; 3] =
                capabilities.try_into().expect("one capability per output");

            // Non-active workers just pass the `desired` and `persist` data through.
            if worker_id != active_worker_id {
                drop(desc_cap);
                shared_frontier.borrow_mut().clear();

                loop {
                    tokio::select! {
                        Some(event) = desired_inputs.ok.next() => {
                            if let Event::Data(cap, mut data) = event {
                                desired_outputs.ok.give_container(&cap, &mut data);
                            }
                        }
                        Some(event) = desired_inputs.err.next() => {
                            if let Event::Data(cap, mut data) = event {
                                desired_outputs.err.give_container(&cap, &mut data);
                            }
                        }
                        // All inputs are exhausted, so we can shut down.
                        else => return,
                    }
                }
            }

            let mut cap_set = CapabilitySet::from_elem(desc_cap);

            let read_only = *read_only_rx.borrow_and_update();
            let mut state = State::new(sink_id, as_of, read_only);

            // Create a stream that reports advancements of the target shard's frontier and updates
            // the shared sink frontier.
            //
            // We collect the persist frontier from a write handle directly, rather than inspecting
            // the `persist` stream, because the latter has two annoying glitches:
            //  (a) It starts at the shard's read frontier, not its write frontier.
            //  (b) It can lag behind if there are spikes in ingested data.
            let mut persist_frontiers = pin!(async_stream::stream! {
                let mut writer = persist_api.open_writer().await;
                let mut frontier = Antichain::from_elem(Timestamp::MIN);
                while !frontier.is_empty() {
                    writer.wait_for_upper_past(&frontier).await;
                    frontier = writer.upper().clone();
                    shared_frontier.borrow_mut().clone_from(&frontier);
                    yield frontier.clone();
                }
            });

            loop {
                // Read from the inputs, pass through all data to the respective outputs, and keep
                // track of the input frontiers. When a frontier advances we might have to mint a
                // new batch description.
                let maybe_desc = tokio::select! {
                    Some(event) = desired_inputs.ok.next() => {
                        match event {
                            Event::Data(cap, mut data) => {
                                desired_outputs.ok.give_container(&cap, &mut data);
                                None
                            }
                            Event::Progress(frontier) => {
                                state.advance_desired_ok_frontier(frontier);
                                state.maybe_mint_batch_description()
                            }
                        }
                    }
                    Some(event) = desired_inputs.err.next() => {
                        match event {
                            Event::Data(cap, mut data) => {
                                desired_outputs.err.give_container(&cap, &mut data);
                                None
                            }
                            Event::Progress(frontier) => {
                                state.advance_desired_err_frontier(frontier);
                                state.maybe_mint_batch_description()
                            }
                        }
                    }
                    Some(frontier) = persist_frontiers.next() => {
                        state.advance_persist_frontier(frontier);
                        state.maybe_mint_batch_description()
                    }
                    Ok(()) = read_only_rx.changed(), if read_only => {
                        state.allow_writes();
                        state.maybe_mint_batch_description()
                    }
                    // All inputs are exhausted, so we can shut down.
                    else => return,
                };

                if let Some(desc) = maybe_desc {
                    let lower_ts = *desc.lower.as_option().expect("not empty");
                    let cap = cap_set.delayed(&lower_ts);
                    desc_output.give(&cap, desc);

                    // We only emit strictly increasing `lower`s, so we can let our output frontier
                    // advance beyond the current `lower`.
                    cap_set.downgrade([lower_ts.step_forward()]);
                } else {
                    // The next emitted `lower` will be at least the `persist` frontier, so we can
                    // advance our output frontier as far.
                    let _ = cap_set.try_downgrade(state.persist_frontier.iter());
                }
            }
        });

        (
            desired_output_streams,
            desc_output_stream,
            sink_frontier,
            button.press_on_drop(),
        )
    }

    /// State maintained by the `mint` operator.
    struct State {
        sink_id: GlobalId,
        /// The frontiers of the `desired` inputs.
        desired_frontiers: OkErr<Antichain<Timestamp>, Antichain<Timestamp>>,
        /// The frontier of the target persist shard.
        persist_frontier: Antichain<Timestamp>,
        /// The last `lower` we have emitted in a batch description, if any. Whenever the
        /// `persist_frontier` moves beyond this frontier, we need to mint a new description.
        last_lower: Option<Antichain<Timestamp>>,
        /// Whether we are operating in read-only mode.
        ///
        /// In read-only mode, minting of batch descriptions is disabled.
        read_only: bool,
    }

    impl State {
        fn new(sink_id: GlobalId, as_of: Antichain<Timestamp>, read_only: bool) -> Self {
            // Initializing `persist_frontier` to the `as_of` ensures that the first minted batch
            // description will have a `lower` of `as_of` or beyond, and thus that we don't spend
            // work needlessly writing batches at previous times.
            let persist_frontier = as_of;

            Self {
                sink_id,
                desired_frontiers: OkErr::new_frontiers(),
                persist_frontier,
                last_lower: None,
                read_only,
            }
        }

        fn trace<S: AsRef<str>>(&self, message: S) {
            let message = message.as_ref();
            trace!(
                sink_id = %self.sink_id,
                desired_frontier = ?self.desired_frontiers.frontier().elements(),
                persist_frontier = ?self.persist_frontier.elements(),
                last_lower = ?self.last_lower.as_ref().map(|f| f.elements()),
                message,
            );
        }

        fn advance_desired_ok_frontier(&mut self, frontier: Antichain<Timestamp>) {
            if advance(&mut self.desired_frontiers.ok, frontier) {
                self.trace("advanced `desired` ok frontier");
            }
        }

        fn advance_desired_err_frontier(&mut self, frontier: Antichain<Timestamp>) {
            if advance(&mut self.desired_frontiers.err, frontier) {
                self.trace("advanced `desired` err frontier");
            }
        }

        fn advance_persist_frontier(&mut self, frontier: Antichain<Timestamp>) {
            if advance(&mut self.persist_frontier, frontier) {
                self.trace("advanced `persist` frontier");
            }
        }

        fn allow_writes(&mut self) {
            if self.read_only {
                self.read_only = false;
                self.trace("disabled read-only mode");
            }
        }

        fn maybe_mint_batch_description(&mut self) -> Option<BatchDescription> {
            let desired_frontier = self.desired_frontiers.frontier();
            let persist_frontier = &self.persist_frontier;

            // We only mint new batch descriptions when:
            //  1. We are _not_ in read-only mode.
            //  2. The `desired` frontier is ahead of the `persist` frontier.
            //  3. The `persist` frontier advanced since we last emitted a batch description.
            let desired_ahead = PartialOrder::less_than(persist_frontier, desired_frontier);
            let persist_advanced = self.last_lower.as_ref().map_or(true, |lower| {
                PartialOrder::less_than(lower, persist_frontier)
            });

            if self.read_only || !desired_ahead || !persist_advanced {
                return None;
            }

            let lower = persist_frontier.clone();
            let upper = desired_frontier.clone();
            let desc = BatchDescription::new(lower, upper);

            self.last_lower = Some(desc.lower.clone());

            self.trace(format!("minted batch description: {desc:?}"));
            Some(desc)
        }
    }
}

/// Implementation of the `write` operator.
mod write {
    use super::*;

    /// Render the `write` operator.
    ///
    /// The parameters passed in are:
    ///  * `sink_id`: The `GlobalId` of the sink export.
    ///  * `persist_api`: An object providing access to the output persist shard.
    ///  * `as_of`: The first time for which the sink may produce output.
    ///  * `desired`: The ok/err streams that should be sinked to persist.
    ///  * `persist`: The ok/err streams read back from the output persist shard.
    ///  * `descs`: The stream of batch descriptions produced by the `mint` operator.
    pub fn render<S>(
        sink_id: GlobalId,
        persist_api: PersistApi,
        as_of: Antichain<Timestamp>,
        desired: &DesiredStreams<S>,
        persist: &PersistStreams<S>,
        descs: &Stream<S, BatchDescription>,
    ) -> (BatchesStream<S>, PressOnDropButton)
    where
        S: Scope<Timestamp = Timestamp>,
    {
        let scope = desired.ok.scope();
        let worker_id = scope.index();

        let name = operator_name(sink_id, "write");
        let mut op = OperatorBuilder::new(name, scope);

        let (batches_output, batches_output_stream) = op.new_output();

        // It is important that we exchange the `desired` and `persist` data the same way, so
        // updates that cancel each other out end up on the same worker.
        let exchange_ok = |(d, _, _): &(Row, Timestamp, Diff)| d.hashed();
        let exchange_err = |(d, _, _): &(DataflowError, Timestamp, Diff)| d.hashed();

        let mut desired_inputs = OkErr::new(
            op.new_disconnected_input(&desired.ok, Exchange::new(exchange_ok)),
            op.new_disconnected_input(&desired.err, Exchange::new(exchange_err)),
        );
        let mut persist_inputs = OkErr::new(
            op.new_disconnected_input(&persist.ok, Exchange::new(exchange_ok)),
            op.new_disconnected_input(&persist.err, Exchange::new(exchange_err)),
        );
        let mut descs_input = op.new_input_for(&descs.broadcast(), Pipeline, &batches_output);

        let button = op.build(move |capabilities| async move {
            // We will use the data capabilities from the `descs` input to produce output, so no
            // need to hold onto the initial capabilities.
            drop(capabilities);

            let writer = persist_api.open_writer().await;
            let sink_metrics = persist_api.open_metrics().await;
            let mut state = State::new(sink_id, worker_id, writer, sink_metrics, as_of);

            loop {
                // Read from the inputs, extract `desired` updates as positive contributions to
                // `correction` and `persist` updates as negative contributions. If either the
                // `desired` or `persist` frontier advances, or if we receive a new batch description,
                // we might have to write a new batch.
                let maybe_batch = tokio::select! {
                    Some(event) = desired_inputs.ok.next() => {
                        match event {
                            Event::Data(_cap, data) => {
                                state.corrections.ok.insert(data);
                                None
                            }
                            Event::Progress(frontier) => {
                                state.advance_desired_ok_frontier(frontier);
                                state.maybe_write_batch().await
                            }
                        }
                    }
                    Some(event) = desired_inputs.err.next() => {
                        match event {
                            Event::Data(_cap, data) => {
                                state.corrections.err.insert(data);
                                None
                            }
                            Event::Progress(frontier) => {
                                state.advance_desired_err_frontier(frontier);
                                state.maybe_write_batch().await
                            }
                        }
                    }
                    Some(event) = persist_inputs.ok.next() => {
                        match event {
                            Event::Data(_cap, data) => {
                                state.corrections.ok.insert_negated(data);
                                None
                            }
                            Event::Progress(frontier) => {
                                state.advance_persist_ok_frontier(frontier);
                                state.maybe_write_batch().await
                            }
                        }
                    }
                    Some(event) = persist_inputs.err.next() => {
                        match event {
                            Event::Data(_cap, data) => {
                                state.corrections.err.insert_negated(data);
                                None
                            }
                            Event::Progress(frontier) => {
                                state.advance_persist_err_frontier(frontier);
                                state.maybe_write_batch().await
                            }
                        }
                    }
                    Some(event) = descs_input.next() => {
                        match event {
                            Event::Data(cap, data) => {
                                for desc in data {
                                    state.absorb_batch_description(desc, cap.clone());
                                }
                                state.maybe_write_batch().await
                            }
                            Event::Progress(_frontier) => None,
                        }
                    }
                    // All inputs are exhausted, so we can shut down.
                    else => return,
                };

                if let Some((batch, cap)) = maybe_batch {
                    batches_output.give(&cap, batch);
                }
            }
        });

        (batches_output_stream, button.press_on_drop())
    }

    /// State maintained by the `write` operator.
    struct State {
        sink_id: GlobalId,
        worker_id: usize,
        persist_writer: WriteHandle<SourceData, (), Timestamp, Diff>,
        /// Contains `desired - persist`, reflecting the updates we would like to commit to
        /// `persist` in order to "correct" it to track `desired`. This collection is only modified
        /// by updates received from either the `desired` or `persist` inputs.
        corrections: OkErr<Correction<Row>, Correction<DataflowError>>,
        /// The frontiers of the `desired` inputs.
        desired_frontiers: OkErr<Antichain<Timestamp>, Antichain<Timestamp>>,
        /// The frontiers of the `persist` inputs.
        persist_frontiers: OkErr<Antichain<Timestamp>, Antichain<Timestamp>>,
        /// The current valid batch description and associated output capability, if any.
        ///
        /// Note that "valid" here implies that if a batch description is set, it must be true that
        /// its `lower` is >= the `persist_frontier`. Otherwise the described batch couldn't be
        /// appended anymore, rendering the batch description invalid.
        batch_description: Option<(BatchDescription, Capability<Timestamp>)>,
        /// A request to force a consolidation of `corrections` once both `desired_frontiers` and
        /// `persist_frontiers` become greater than the given frontier.
        ///
        /// Normally we force a consolidation whenever we write a batch, but there are periods
        /// (like read-only mode) when that doesn't happen, and we need to manually force
        /// consolidation instead. Currently this is only used to ensure we quickly get rid of the
        /// snapshot updates.
        force_consolidation_after: Option<Antichain<Timestamp>>,
    }

    impl State {
        fn new(
            sink_id: GlobalId,
            worker_id: usize,
            persist_writer: WriteHandle<SourceData, (), Timestamp, Diff>,
            metrics: SinkMetrics,
            as_of: Antichain<Timestamp>,
        ) -> Self {
            let worker_metrics = metrics.for_worker(worker_id);

            // Force a consolidation of `corrections` after the snapshot updates have been fully
            // processed, to ensure we get rid of those as quickly as possible.
            let force_consolidation_after = Some(as_of);

            Self {
                sink_id,
                worker_id,
                persist_writer,
                corrections: OkErr::new(
                    Correction::new(metrics.clone(), worker_metrics.clone()),
                    Correction::new(metrics, worker_metrics),
                ),
                desired_frontiers: OkErr::new_frontiers(),
                persist_frontiers: OkErr::new_frontiers(),
                batch_description: None,
                force_consolidation_after,
            }
        }

        fn trace<S: AsRef<str>>(&self, message: S) {
            let message = message.as_ref();
            trace!(
                sink_id = %self.sink_id,
                worker = %self.worker_id,
                desired_frontier = ?self.desired_frontiers.frontier().elements(),
                persist_frontier = ?self.persist_frontiers.frontier().elements(),
                batch_description = ?self.batch_description.as_ref().map(|(d, _)| d),
                message,
            );
        }

        fn advance_desired_ok_frontier(&mut self, frontier: Antichain<Timestamp>) {
            if advance(&mut self.desired_frontiers.ok, frontier) {
                self.apply_desired_frontier_advancement();
                self.trace("advanced `desired` ok frontier");
            }
        }

        fn advance_desired_err_frontier(&mut self, frontier: Antichain<Timestamp>) {
            if advance(&mut self.desired_frontiers.err, frontier) {
                self.apply_desired_frontier_advancement();
                self.trace("advanced `desired` err frontier");
            }
        }

        fn advance_persist_ok_frontier(&mut self, frontier: Antichain<Timestamp>) {
            if advance(&mut self.persist_frontiers.ok, frontier) {
                self.apply_persist_frontier_advancement();
                self.trace("advanced `persist` ok frontier");
            }
        }

        fn advance_persist_err_frontier(&mut self, frontier: Antichain<Timestamp>) {
            if advance(&mut self.persist_frontiers.err, frontier) {
                self.apply_persist_frontier_advancement();
                self.trace("advanced `persist` err frontier");
            }
        }

        /// Apply the effects of a previous `desired` frontier advancement.
        fn apply_desired_frontier_advancement(&mut self) {
            self.maybe_force_consolidation();
        }

        /// Apply the effects of a previous `persist` frontier advancement.
        fn apply_persist_frontier_advancement(&mut self) {
            let frontier = self.persist_frontiers.frontier();

            // We will only emit times at or after the `persist` frontier, so now is a good time to
            // advance the times of stashed updates.
            self.corrections.ok.advance_since(frontier.clone());
            self.corrections.err.advance_since(frontier.clone());

            // If the `persist` frontier is greater than the `lower` of the current batch
            // description, we won't be able to append the batch, so the batch description is not
            // valid anymore.
            if let Some((desc, _)) = &self.batch_description {
                if PartialOrder::less_than(&desc.lower, frontier) {
                    self.batch_description = None;
                }
            }

            self.maybe_force_consolidation();
        }

        /// If the current consolidation request has become applicable, apply it.
        fn maybe_force_consolidation(&mut self) {
            let Some(request) = &self.force_consolidation_after else {
                return;
            };

            let desired_frontier = self.desired_frontiers.frontier();
            let persist_frontier = self.persist_frontiers.frontier();
            if PartialOrder::less_than(request, desired_frontier)
                && PartialOrder::less_than(request, persist_frontier)
            {
                self.trace("forcing correction consolidation");
                self.corrections.ok.consolidate_at_since();
                self.corrections.err.consolidate_at_since();

                // Remove the consolidation request, now that we have fulfilled it.
                self.force_consolidation_after = None;
            }
        }

        fn absorb_batch_description(&mut self, desc: BatchDescription, cap: Capability<Timestamp>) {
            // The incoming batch description is outdated if either:
            // * we already have a batch description with a greater `lower`, or
            // * its `lower` is less than the persist frontier
            let validity_frontier = match &self.batch_description {
                Some((prev, _)) => &prev.lower,
                None => self.persist_frontiers.frontier(),
            };
            if PartialOrder::less_than(&desc.lower, validity_frontier) {
                self.trace(format!("skipping outdated batch description: {desc:?}"));
                return;
            }

            self.batch_description = Some((desc, cap));
            self.trace("set batch description");
        }

        async fn maybe_write_batch(&mut self) -> Option<(ProtoBatch, Capability<Timestamp>)> {
            let (desc, _cap) = self.batch_description.as_ref()?;

            // We can write a new batch if we have seen all `persist` updates before `lower` and
            // all `desired` updates up to `upper`.
            let persist_complete = desc.lower == *self.persist_frontiers.frontier();
            let desired_complete =
                PartialOrder::less_equal(&desc.upper, self.desired_frontiers.frontier());
            if !persist_complete || !desired_complete {
                return None;
            }

            let (desc, cap) = self.batch_description.take()?;

            assert_eq!(desc.lower, *self.corrections.ok.since());
            assert_eq!(desc.lower, *self.corrections.err.since());

            let ok_updates = self.corrections.ok.updates_before(&desc.upper);
            let err_updates = self.corrections.err.updates_before(&desc.upper);

            let oks = ok_updates.map(|(d, t, r)| ((SourceData(Ok(d)), ()), t, r));
            let errs = err_updates.map(|(d, t, r)| ((SourceData(Err(d)), ()), t, r));
            let mut updates = oks.chain(errs).peekable();

            // Don't write empty batches.
            if updates.peek().is_none() {
                drop(updates);
                self.trace("skipping empty batch");
                return None;
            }

            let batch = self
                .persist_writer
                .batch(updates, desc.lower, desc.upper)
                .await
                .expect("valid usage")
                .into_transmittable_batch();

            self.trace("wrote a batch");
            Some((batch, cap))
        }
    }
}

/// Implementation of the `append` operator.
mod append {
    use super::*;

    /// Render the `append` operator.
    ///
    /// The parameters passed in are:
    ///  * `sink_id`: The `GlobalId` of the sink export.
    ///  * `persist_api`: An object providing access to the output persist shard.
    ///  * `descs`: The stream of batch descriptions produced by the `mint` operator.
    ///  * `batches`: The stream of written batches produced by the `write` operator.
    pub fn render<S>(
        sink_id: GlobalId,
        persist_api: PersistApi,
        active_worker_id: usize,
        descs: &DescsStream<S>,
        batches: &BatchesStream<S>,
    ) -> PressOnDropButton
    where
        S: Scope<Timestamp = Timestamp>,
    {
        let scope = descs.scope();
        let worker_id = scope.index();

        let name = operator_name(sink_id, "append");
        let mut op = OperatorBuilder::new(name, scope);

        let mut descs_input = op.new_disconnected_input(descs, Pipeline);
        let mut batches_input = op.new_disconnected_input(
            batches,
            Exchange::new(move |_| u64::cast_from(active_worker_id)),
        );

        let button = op.build(move |_capabilities| async move {
            if worker_id != active_worker_id {
                return;
            }

            let writer = persist_api.open_writer().await;
            let mut state = State::new(sink_id, writer);

            loop {
                // Read from the inputs, absorb batch descriptions and batches. If the `batches`
                // frontier advances, or if we receive a new batch description, we might have to
                // append a new batch.
                tokio::select! {
                    Some(event) = descs_input.next() => {
                        if let Event::Data(_cap, data) = event {
                            for desc in data {
                                state.absorb_batch_description(desc).await;
                                state.maybe_append_batches().await;
                            }
                        }
                    }
                    Some(event) = batches_input.next() => {
                        match event {
                            Event::Data(_cap, data) => {
                                for batch in data {
                                    state.absorb_batch(batch).await;
                                }
                            }
                            Event::Progress(frontier) => {
                                state.advance_batches_frontier(frontier);
                                state.maybe_append_batches().await;
                            }
                        }
                    }
                    // All inputs are exhausted, so we can shut down.
                    else => return,
                }
            }
        });

        button.press_on_drop()
    }

    /// State maintained by the `append` operator.
    struct State {
        sink_id: GlobalId,
        persist_writer: WriteHandle<SourceData, (), Timestamp, Diff>,
        /// The current input frontier of `batches`.
        batches_frontier: Antichain<Timestamp>,
        /// The greatest observed `lower` from both `descs` and `batches`.
        lower: Antichain<Timestamp>,
        /// The batch description for `lower`, if any.
        batch_description: Option<BatchDescription>,
        /// Batches received for `lower`.
        batches: Vec<Batch<SourceData, (), Timestamp, Diff>>,
    }

    impl State {
        fn new(
            sink_id: GlobalId,
            persist_writer: WriteHandle<SourceData, (), Timestamp, Diff>,
        ) -> Self {
            Self {
                sink_id,
                persist_writer,
                batches_frontier: Antichain::from_elem(Timestamp::MIN),
                lower: Antichain::from_elem(Timestamp::MIN),
                batch_description: None,
                batches: Default::default(),
            }
        }

        fn trace<S: AsRef<str>>(&self, message: S) {
            let message = message.as_ref();
            trace!(
                sink_id = %self.sink_id,
                batches_frontier = ?self.batches_frontier.elements(),
                lower = ?self.lower.elements(),
                batch_description = ?self.batch_description,
                message,
            );
        }

        fn advance_batches_frontier(&mut self, frontier: Antichain<Timestamp>) {
            if advance(&mut self.batches_frontier, frontier) {
                self.trace("advanced `batches` frontier");
            }
        }

        /// Advance the current `lower`.
        ///
        /// Discards all currently stashed batches and batch descriptions, assuming that they are
        /// now invalid.
        async fn advance_lower(&mut self, frontier: Antichain<Timestamp>) {
            assert!(PartialOrder::less_than(&self.lower, &frontier));

            self.lower = frontier;
            self.batch_description = None;

            // Remove stashed batches, cleaning up those we didn't append.
            for batch in self.batches.drain(..) {
                batch.delete().await;
            }

            self.trace("advanced `lower`");
        }

        /// Absorb the given batch description into the state, provided it is not outdated.
        async fn absorb_batch_description(&mut self, desc: BatchDescription) {
            if PartialOrder::less_than(&self.lower, &desc.lower) {
                self.advance_lower(desc.lower.clone()).await;
            } else if &self.lower != &desc.lower {
                self.trace(format!("skipping outdated batch description: {desc:?}"));
                return;
            }

            self.batch_description = Some(desc);
            self.trace("set batch description");
        }

        /// Absorb the given batch into the state, provided it is not outdated.
        async fn absorb_batch(&mut self, batch: ProtoBatch) {
            let batch = self.persist_writer.batch_from_transmittable_batch(batch);
            if PartialOrder::less_than(&self.lower, batch.lower()) {
                self.advance_lower(batch.lower().clone()).await;
            } else if &self.lower != batch.lower() {
                self.trace(format!(
                    "skipping outdated batch: ({:?}, {:?})",
                    batch.lower().elements(),
                    batch.upper().elements(),
                ));

                // Ensure the batch's data gets properly cleaned up before dropping it.
                batch.delete().await;
                return;
            }

            self.batches.push(batch);
            self.trace("absorbed a batch");
        }

        async fn maybe_append_batches(&mut self) {
            let batches_complete = PartialOrder::less_than(&self.lower, &self.batches_frontier);
            if !batches_complete {
                return;
            }

            let Some(desc) = self.batch_description.take() else {
                return;
            };

            let new_lower = match self.append_batches(desc).await {
                Ok(shard_upper) => {
                    self.trace("appended a batch");
                    shard_upper
                }
                Err(shard_upper) => {
                    // Failing the append is expected in the presence of concurrent replicas. There
                    // is nothing special to do here: The self-correcting feedback mechanism
                    // ensures that we observe the concurrent changes, compute their consequences,
                    // and append them at a future time.
                    self.trace(format!(
                        "append failed due to `lower` mismatch: {:?}",
                        shard_upper.elements(),
                    ));
                    shard_upper
                }
            };

            self.advance_lower(new_lower).await;
        }

        /// Append the current `batches` to the output shard.
        ///
        /// Returns whether the append was successful or not, and the current shard upper in either
        /// case.
        ///
        /// This method advances the shard upper to the batch `lower` if necessary. This is the
        /// mechanism that brings the shard upper to the sink as-of when appending the initial
        /// batch.
        ///
        /// An alternative mechanism for bringing the shard upper to the sink as-of would be making
        /// a single append at operator startup. The reason we are doing it here instead is that it
        /// simplifies the implementation of read-only mode. In read-only mode we have to defer any
        /// persist writes, including the initial upper bump. Having only a single place that
        /// performs writes makes it easy to ensure we are doing that correctly.
        async fn append_batches(
            &mut self,
            desc: BatchDescription,
        ) -> Result<Antichain<Timestamp>, Antichain<Timestamp>> {
            let (lower, upper) = (desc.lower, desc.upper);
            let mut to_append: Vec<_> = self.batches.iter_mut().collect();

            loop {
                let result = self
                    .persist_writer
                    .compare_and_append_batch(&mut to_append, lower.clone(), upper.clone())
                    .await
                    .expect("valid usage");

                match result {
                    Ok(()) => return Ok(upper),
                    Err(mismatch) if PartialOrder::less_than(&mismatch.current, &lower) => {
                        advance_shard_upper(&mut self.persist_writer, lower.clone()).await;
                    }
                    Err(mismatch) => return Err(mismatch.current),
                }
            }
        }
    }

    /// Advance the frontier of the given writer's shard to at least the given `upper`.
    async fn advance_shard_upper(
        persist_writer: &mut WriteHandle<SourceData, (), Timestamp, Diff>,
        upper: Antichain<Timestamp>,
    ) {
        let empty_updates: &[((SourceData, ()), Timestamp, Diff)] = &[];
        let lower = Antichain::from_elem(Timestamp::MIN);
        persist_writer
            .append(empty_updates, lower, upper)
            .await
            .expect("valid usage")
            .expect("should always succeed");
    }
}