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
// 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 tokio tasks (and support machinery) for dealing with the persist handles
//! that the storage controller needs to hold.

use std::collections::{BTreeMap, BTreeSet, VecDeque};
use std::fmt::Debug;
use std::sync::Arc;

use differential_dataflow::lattice::Lattice;
use futures::future::BoxFuture;
use futures::stream::FuturesUnordered;
use futures::{FutureExt, StreamExt};
use itertools::Itertools;
use mz_persist_client::critical::SinceHandle;
use mz_persist_client::stats::SnapshotStats;
use mz_persist_client::write::WriteHandle;
use mz_persist_types::Codec64;
use mz_repr::{Diff, GlobalId, TimestampManipulation};
use mz_storage_types::sources::SourceData;
use timely::progress::{Antichain, Timestamp};
use tokio::sync::mpsc::UnboundedSender;
use tokio::sync::oneshot;
use tracing::Instrument;

use crate::client::{StorageResponse, TimestamplessUpdate, Update};
use crate::controller::{PersistEpoch, StorageError};

/// A wrapper that holds on to backing persist shards/collections that the
/// storage controller is aware of. The handles hold back the since frontier and
/// we need to downgrade them when the read capabilities change.
///
/// Internally, this has an async task and the methods for registering a handle
/// and downgrading sinces add commands to a queue that this task is working
/// off. This makes the methods non-blocking and moves the work outside the main
/// coordinator task, meaning the coordinator is spending less time waiting on
/// persist calls.
#[derive(Debug)]
pub struct PersistReadWorker<T: Timestamp + Lattice + Codec64> {
    tx: UnboundedSender<(tracing::Span, PersistReadWorkerCmd<T>)>,
}

/// Commands for [PersistReadWorker].
#[derive(Debug)]
enum PersistReadWorkerCmd<T: Timestamp + Lattice + Codec64> {
    Register(GlobalId, SinceHandle<SourceData, (), T, Diff, PersistEpoch>),
    Update(GlobalId, SinceHandle<SourceData, (), T, Diff, PersistEpoch>),
    Downgrade(BTreeMap<GlobalId, Antichain<T>>),
    SnapshotStats(GlobalId, Antichain<T>, oneshot::Sender<SnapshotStatsRes<T>>),
}

/// A newtype wrapper to hang a Debug impl off of.
pub(crate) struct SnapshotStatsRes<T>(BoxFuture<'static, Result<SnapshotStats<T>, StorageError>>);

impl<T: Debug> Debug for SnapshotStatsRes<T> {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("SnapshotStatsRes").finish_non_exhaustive()
    }
}

impl<T: Timestamp + Lattice + Codec64> PersistReadWorker<T> {
    pub(crate) fn new() -> Self {
        let (tx, mut rx) = tokio::sync::mpsc::unbounded_channel::<(tracing::Span, _)>();

        mz_ore::task::spawn(|| "PersistWorker", async move {
            let mut since_handles = BTreeMap::new();

            while let Some(cmd) = rx.recv().await {
                // Peel off all available commands.
                // This allows us to catch up if we fall behind on downgrade commands.
                let mut commands = vec![cmd];
                while let Ok(cmd) = rx.try_recv() {
                    commands.push(cmd);
                }
                // Collect all downgrade requests and apply them last.
                let mut downgrades = BTreeMap::default();

                for (span, command) in commands {
                    match command {
                        PersistReadWorkerCmd::Register(id, since_handle) => {
                            let previous = since_handles.insert(id, since_handle);
                            if previous.is_some() {
                                panic!("already registered a SinceHandle for collection {id}");
                            }
                        }
                        PersistReadWorkerCmd::Update(id, since_handle) => {
                            since_handles.insert(id, since_handle).expect("PersistReadWorkerCmd::Update only valid for updating extant since handles");
                        }
                        PersistReadWorkerCmd::Downgrade(since_frontiers) => {
                            for (id, frontier) in since_frontiers {
                                downgrades.insert(id, (span.clone(), frontier));
                            }
                        }
                        PersistReadWorkerCmd::SnapshotStats(id, as_of, tx) => {
                            // NB: The requested as_of could be arbitrarily far in the future. So,
                            // in order to avoid blocking the PersistReadWorker loop until it's
                            // available and the `snapshot_stats` call resolves, instead return the
                            // future to the caller and await it there.
                            let res = match since_handles.get(&id) {
                                Some(x) => {
                                    let fut = x.snapshot_stats(as_of).map(move |x| {
                                        x.map_err(|_| StorageError::ReadBeforeSince(id))
                                    });
                                    SnapshotStatsRes(Box::pin(fut))
                                }
                                None => SnapshotStatsRes(Box::pin(futures::future::ready(Err(
                                    StorageError::IdentifierMissing(id),
                                )))),
                            };
                            // It's fine if the listener hung up.
                            let _ = tx.send(res);
                        }
                    }
                }

                let mut futs = FuturesUnordered::new();

                for (id, (span, since)) in downgrades {
                    let Some(mut since_handle) = since_handles.remove(&id) else {
                        panic!("downgrade command for absent collection {id}");
                    };

                    futs.push(async move {
                        let epoch = since_handle.opaque().clone();

                        let result = if since.is_empty() {
                            // A shard's since reaching the empty frontier is a prereq for being
                            // able to finalize a shard, so the final downgrade should never be
                            // rate-limited.
                            Some(
                                since_handle
                                    .compare_and_downgrade_since(&epoch, (&epoch, &since))
                                    .instrument(span)
                                    .await,
                            )
                        } else {
                            since_handle
                                .maybe_compare_and_downgrade_since(&epoch, (&epoch, &since))
                                .instrument(span)
                                .await
                        };

                        if let Some(Err(other_epoch)) = result {
                            mz_ore::halt!("fenced by envd @ {other_epoch:?}. ours = {epoch:?}");
                        }

                        // If we're not done we put the handle back
                        if !since.is_empty() {
                            Some((id, (since_handle)))
                        } else {
                            None
                        }
                    });
                }

                while let Some(entry) = futs.next().await {
                    since_handles.extend(entry);
                }
            }
            tracing::trace!("shutting down persist since downgrade task");
        });

        Self { tx }
    }

    pub(crate) fn register(
        &self,
        id: GlobalId,
        since_handle: SinceHandle<SourceData, (), T, Diff, PersistEpoch>,
    ) {
        self.send(PersistReadWorkerCmd::Register(id, since_handle))
    }

    /// Update the existing since handle associated with `id` to `since_handle`.
    ///
    /// Note that this should only be called when updating a since handle; to
    /// initially associate an `id` to a since handle, use [`Self::register`].
    ///
    /// # Panics
    /// - If `id` is not currently associated with any since handle.
    #[allow(dead_code)]
    pub(crate) fn update(
        &self,
        id: GlobalId,
        since_handle: SinceHandle<SourceData, (), T, Diff, PersistEpoch>,
    ) {
        self.send(PersistReadWorkerCmd::Update(id, since_handle))
    }

    pub(crate) fn downgrade(&self, frontiers: BTreeMap<GlobalId, Antichain<T>>) {
        self.send(PersistReadWorkerCmd::Downgrade(frontiers))
    }

    pub(crate) async fn snapshot_stats(
        &self,
        id: GlobalId,
        as_of: Antichain<T>,
    ) -> Result<SnapshotStats<T>, StorageError> {
        // TODO: Pull this out of PersistReadWorker. Unlike the other methods,
        // the caller of this one drives it to completion.
        //
        // We'd need to either share the critical handle somehow or maybe have
        // two instances around, one in the worker and one in the controller.
        let (tx, rx) = oneshot::channel();
        self.send(PersistReadWorkerCmd::SnapshotStats(id, as_of, tx));
        rx.await.expect("PersistReadWorker should be live").0.await
    }

    fn send(&self, cmd: PersistReadWorkerCmd<T>) {
        self.tx
            .send((tracing::Span::current(), cmd))
            .expect("persist worker exited while its handle was alive")
    }
}

#[derive(Debug, Clone)]
pub struct PersistTableWriteWorker<T: Timestamp + Lattice + Codec64 + TimestampManipulation> {
    inner: Arc<PersistTableWriteWorkerInner<T>>,
}

/// Commands for [PersistTableWriteWorker].
#[derive(Debug)]
enum PersistTableWriteCmd<T: Timestamp + Lattice + Codec64> {
    Register(GlobalId, WriteHandle<SourceData, (), T, Diff>),
    Update(GlobalId, WriteHandle<SourceData, (), T, Diff>),
    DropHandle(GlobalId),
    Append(
        Vec<(GlobalId, Vec<Update<T>>, T)>,
        tokio::sync::oneshot::Sender<Result<(), StorageError>>,
    ),
    Shutdown,
}

// TODO(txn): This impl is duplicative of PersistMonotonicWriteWorker, but it's
// an intermediate state. The impl of this one will be completely replaced by
// #20954.
impl<T: Timestamp + Lattice + Codec64 + TimestampManipulation> PersistTableWriteWorker<T> {
    pub(crate) fn new(
        mut frontier_responses: tokio::sync::mpsc::UnboundedSender<StorageResponse<T>>,
    ) -> Self {
        let (tx, mut rx) = tokio::sync::mpsc::unbounded_channel::<(tracing::Span, _)>();

        mz_ore::task::spawn(|| "PersistTableWriteWorker", async move {
            let mut write_handles =
                BTreeMap::<GlobalId, WriteHandle<SourceData, (), T, Diff>>::new();

            let mut shutdown = false;
            while !shutdown {
                tokio::select! {
                    cmd = rx.recv() => {
                        if let Some(cmd) = cmd {
                            // Peel off all available commands.
                            // We do this in case we can consolidate commands.
                            // It would be surprising to receive multiple concurrent `Append` commands,
                            // but we might receive multiple *empty* `Append` commands.
                            let mut commands = VecDeque::new();
                            commands.push_back(cmd);
                            while let Ok(cmd) = rx.try_recv() {
                                commands.push_back(cmd);
                            }

                            // Accumulated updates and upper frontier.
                            let mut all_updates = BTreeMap::default();
                            let mut all_responses = Vec::default();

                            while let Some((span, command)) = commands.pop_front() {
                                match command {
                                    PersistTableWriteCmd::Register(id, write_handle) => {
                                        let previous = write_handles.insert(id, write_handle);
                                        if previous.is_some() {
                                            panic!(
                                                "already registered a WriteHandle for collection {:?}",
                                                id
                                            );
                                        }
                                    }
                                    PersistTableWriteCmd::Update(id, write_handle) => {
                                        write_handles.insert(id, write_handle).expect("PersistTableWriteCmd::Update only valid for updating extant write handles");
                                    },
                                    PersistTableWriteCmd::DropHandle(id) => {
                                        // n.b. this should only remove the
                                        // handle from the persist worker and
                                        // not take any additional action such
                                        // as closing the shard it's connected
                                        // to because dataflows might still be
                                        // using it.
                                        write_handles.remove(&id);
                                    }
                                    PersistTableWriteCmd::Append(updates, response) => {
                                        let mut ids = BTreeSet::new();
                                        for (id, update, upper) in updates {
                                            ids.insert(id);
                                            let (old_span, updates, old_upper) =
                                                all_updates.entry(id).or_insert_with(|| {
                                                    (
                                                        span.clone(),
                                                        Vec::default(),
                                                        Antichain::from_elem(T::minimum()),
                                                    )
                                                });

                                            if old_span.id() != span.id() {
                                                // Link in any spans for `Append`
                                                // operations that we lump together by
                                                // doing this. This is not ideal,
                                                // because we only have a true tracing
                                                // history for the "first" span that we
                                                // process, but it's better than
                                                // nothing.
                                                old_span.follows_from(span.id());
                                            }
                                            updates.extend(update);
                                            old_upper.join_assign(&Antichain::from_elem(upper));
                                        }
                                        all_responses.push((ids, response));
                                    }
                                    PersistTableWriteCmd::Shutdown => {
                                        shutdown = true;
                                    }
                                }
                            }

                            async fn append_work<T2: Timestamp + Lattice + Codec64>(
                                frontier_responses: &mut tokio::sync::mpsc::UnboundedSender<
                                    StorageResponse<T2>,
                                >,
                                write_handles: &mut BTreeMap<
                                    GlobalId,
                                    WriteHandle<SourceData, (), T2, Diff>,
                                >,
                                mut commands: BTreeMap<
                                    GlobalId,
                                    (tracing::Span, Vec<Update<T2>>, Antichain<T2>),
                                >,
                            ) -> Result<(), Vec<GlobalId>> {
                                let futs = FuturesUnordered::new();

                                // We cannot iterate through the updates and then set off a persist call
                                // on the write handle because we cannot mutably borrow the write handle
                                // multiple times.
                                //
                                // Instead, we first group the update by ID above and then iterate
                                // through all available write handles and see if there are any updates
                                // for it. If yes, we send them all in one go.
                                for (id, write) in write_handles.iter_mut() {
                                    if let Some((span, updates, new_upper)) = commands.remove(id) {
                                        let persist_upper = write.upper().clone();
                                        let updates = updates
                                            .into_iter()
                                            .map(|u| ((SourceData(Ok(u.row)), ()), u.timestamp, u.diff));

                                        futs.push(async move {
                                            let persist_upper = persist_upper.clone();
                                            write
                                                .compare_and_append(
                                                    updates.clone(),
                                                    persist_upper.clone(),
                                                    new_upper.clone(),
                                                )
                                                .instrument(span.clone())
                                                .await
                                                .expect("cannot append updates")
                                                .or(Err(*id))?;

                                            Ok::<_, GlobalId>((*id, new_upper))
                                        })
                                    }
                                }

                                use futures::StreamExt;
                                // Ensure all futures run to completion, and track status of each of them individually
                                let (new_uppers, failed_appends): (Vec<_>, Vec<_>) = futs
                                    .collect::<Vec<_>>()
                                    .await
                                    .into_iter()
                                    .partition_result();

                                // It is not strictly an error for the controller to hang up.
                                let _ =
                                    frontier_responses.send(StorageResponse::FrontierUppers(new_uppers));

                                if failed_appends.is_empty() {
                                    Ok(())
                                } else {
                                    Err(failed_appends)
                                }
                            }

                            let result =
                                append_work(&mut frontier_responses, &mut write_handles, all_updates).await;

                            for (ids, response) in all_responses {
                                let result = match &result {
                                    Err(bad_ids) => {
                                        let filtered: Vec<_> = bad_ids.iter().filter(|id| ids.contains(id)).copied().collect();
                                        if filtered.is_empty() {
                                            Ok(())
                                        } else {
                                            Err(StorageError::InvalidUppers(filtered))
                                        }
                                    }
                                    Ok(()) => Ok(()),
                                };
                                // It is not an error for the other end to hang up.
                                let _ = response.send(result);
                            }

                            if shutdown {
                                tracing::trace!("shutting down persist write append task");
                                break;
                            }
                        } else {
                            shutdown = true;
                        }
                    }
                }
            }

            tracing::info!("PersistTableWriteWorker shutting down");
        });

        Self {
            inner: Arc::new(PersistTableWriteWorkerInner::new(tx)),
        }
    }

    pub(crate) fn register(
        &self,
        id: GlobalId,
        write_handle: WriteHandle<SourceData, (), T, Diff>,
    ) {
        self.send(PersistTableWriteCmd::Register(id, write_handle))
    }

    /// Update the existing write handle associated with `id` to `write_handle`.
    ///
    /// Note that this should only be called when updating a write handle; to
    /// initially associate an `id` to a write handle, use [`Self::register`].
    ///
    /// # Panics
    /// - If `id` is not currently associated with any write handle.
    #[allow(dead_code)]
    pub(crate) fn update(&self, id: GlobalId, write_handle: WriteHandle<SourceData, (), T, Diff>) {
        self.send(PersistTableWriteCmd::Update(id, write_handle))
    }

    pub(crate) fn append(
        &self,
        updates: Vec<(GlobalId, Vec<Update<T>>, T)>,
    ) -> tokio::sync::oneshot::Receiver<Result<(), StorageError>> {
        let (tx, rx) = tokio::sync::oneshot::channel();
        if updates.is_empty() {
            tx.send(Ok(()))
                .expect("rx has not been dropped at this point");
            rx
        } else {
            self.send(PersistTableWriteCmd::Append(updates, tx));
            rx
        }
    }

    /// Drops the handle associated with `id` from this worker.
    ///
    /// Note that this does not perform any other cleanup, such as finalizing
    /// the handle's shard.
    pub(crate) fn drop_handle(&self, id: GlobalId) {
        self.send(PersistTableWriteCmd::DropHandle(id))
    }

    fn send(&self, cmd: PersistTableWriteCmd<T>) {
        self.inner.send(cmd);
    }
}

/// Contains the components necessary for sending commands to a `PersistTableWriteWorker`.
///
/// When `Drop`-ed sends a shutdown command, as such this should _never_ implement `Clone` because
/// if one clone is dropped, the other clones will be unable to send commands. If you need this
/// to be `Clone`-able, wrap it in an `Arc` or `Rc` first.
///
/// #[derive(Clone)] <-- do not do this.
///
#[derive(Debug)]
struct PersistTableWriteWorkerInner<T: Timestamp + Lattice + Codec64 + TimestampManipulation> {
    /// Sending side of a channel that we can use to send commands.
    tx: UnboundedSender<(tracing::Span, PersistTableWriteCmd<T>)>,
}

impl<T> Drop for PersistTableWriteWorkerInner<T>
where
    T: Timestamp + Lattice + Codec64 + TimestampManipulation,
{
    fn drop(&mut self) {
        self.send(PersistTableWriteCmd::Shutdown);
        // TODO: Can't easily block on shutdown occurring.
    }
}

impl<T> PersistTableWriteWorkerInner<T>
where
    T: Timestamp + Lattice + Codec64 + TimestampManipulation,
{
    fn new(tx: UnboundedSender<(tracing::Span, PersistTableWriteCmd<T>)>) -> Self {
        PersistTableWriteWorkerInner { tx }
    }

    fn send(&self, cmd: PersistTableWriteCmd<T>) {
        match self.tx.send((tracing::Span::current(), cmd)) {
            Ok(()) => (), // All good!
            Err(e) => {
                tracing::trace!("could not forward command: {:?}", e);
            }
        }
    }
}

#[derive(Debug, Clone)]
pub struct PersistMonotonicWriteWorker<T: Timestamp + Lattice + Codec64 + TimestampManipulation> {
    inner: Arc<PersistMonotonicWriteWorkerInner<T>>,
}

/// Commands for [PersistMonotonicWriteWorker].
#[derive(Debug)]
enum PersistMonotonicWriteCmd<T: Timestamp + Lattice + Codec64> {
    Register(GlobalId, WriteHandle<SourceData, (), T, Diff>),
    Update(GlobalId, WriteHandle<SourceData, (), T, Diff>),
    DropHandle(GlobalId),
    Append(
        Vec<(GlobalId, Vec<Update<T>>, T)>,
        tokio::sync::oneshot::Sender<Result<(), StorageError>>,
    ),
    /// Appends `Vec<TimelessUpdate>` to `GlobalId` at, essentially,
    /// `max(write_frontier, T)`.
    MonotonicAppend(
        Vec<(GlobalId, Vec<TimestamplessUpdate>, T)>,
        tokio::sync::oneshot::Sender<Result<(), StorageError>>,
    ),
    Shutdown,
}

// TODO(txn): This impl is duplicative of PersistTableWriteWorker, but it's
// an intermediate state. The impl of the table one will be completely replaced
// by #20954.
impl<T: Timestamp + Lattice + Codec64 + TimestampManipulation> PersistMonotonicWriteWorker<T> {
    pub(crate) fn new(
        mut frontier_responses: tokio::sync::mpsc::UnboundedSender<StorageResponse<T>>,
    ) -> Self {
        let (tx, mut rx) = tokio::sync::mpsc::unbounded_channel::<(tracing::Span, _)>();

        mz_ore::task::spawn(|| "PersistMonotonicWriteWorker", async move {
            let mut write_handles =
                BTreeMap::<GlobalId, WriteHandle<SourceData, (), T, Diff>>::new();

            let mut shutdown = false;
            while !shutdown {
                tokio::select! {
                    cmd = rx.recv() => {
                        if let Some(cmd) = cmd {
                            // Peel off all available commands.
                            // We do this in case we can consolidate commands.
                            // It would be surprising to receive multiple concurrent `Append` commands,
                            // but we might receive multiple *empty* `Append` commands.
                            let mut commands = VecDeque::new();
                            commands.push_back(cmd);
                            while let Ok(cmd) = rx.try_recv() {
                                commands.push_back(cmd);
                            }

                            // Accumulated updates and upper frontier.
                            let mut all_updates = BTreeMap::default();
                            let mut all_responses = Vec::default();

                            while let Some((span, command)) = commands.pop_front() {
                                match command {
                                    PersistMonotonicWriteCmd::Register(id, write_handle) => {
                                        let previous = write_handles.insert(id, write_handle);
                                        if previous.is_some() {
                                            panic!(
                                                "already registered a WriteHandle for collection {:?}",
                                                id
                                            );
                                        }
                                    }
                                    PersistMonotonicWriteCmd::Update(id, write_handle) => {
                                        write_handles.insert(id, write_handle).expect("PersistMonotonicWriteCmd::Update only valid for updating extant write handles");
                                    },
                                    PersistMonotonicWriteCmd::DropHandle(id) => {
                                        // n.b. this should only remove the
                                        // handle from the persist worker and
                                        // not take any additional action such
                                        // as closing the shard it's connected
                                        // to because dataflows might still be
                                        // using it.
                                        write_handles.remove(&id);
                                    }
                                    PersistMonotonicWriteCmd::Append(updates, response) => {
                                        let mut ids = BTreeSet::new();
                                        for (id, update, upper) in updates {
                                            ids.insert(id);
                                            let (old_span, updates, old_upper) =
                                                all_updates.entry(id).or_insert_with(|| {
                                                    (
                                                        span.clone(),
                                                        Vec::default(),
                                                        Antichain::from_elem(T::minimum()),
                                                    )
                                                });

                                            if old_span.id() != span.id() {
                                                // Link in any spans for `Append`
                                                // operations that we lump together by
                                                // doing this. This is not ideal,
                                                // because we only have a true tracing
                                                // history for the "first" span that we
                                                // process, but it's better than
                                                // nothing.
                                                old_span.follows_from(span.id());
                                            }
                                            updates.extend(update);
                                            old_upper.join_assign(&Antichain::from_elem(upper));
                                        }
                                        all_responses.push((ids, response));
                                    }
                                    PersistMonotonicWriteCmd::MonotonicAppend(updates, response) => {
                                        let mut updates_outer = Vec::with_capacity(updates.len());
                                        for (id, update, at_least) in updates {
                                            let current_upper = write_handles[&id].upper().clone();
                                            if update.is_empty() && current_upper.is_empty() {
                                                // Ignore timestamp advancement for
                                                // closed collections. TODO? Make this a
                                                // correctable error
                                                continue;
                                            }

                                            let lower = if current_upper.less_than(&at_least) {
                                                at_least
                                            } else {
                                                current_upper.into_option().expect("cannot append data to closed collection")
                                            };

                                            let upper = lower.step_forward();
                                            let update = update
                                                .into_iter()
                                                .map(|TimestamplessUpdate { row, diff }| Update {
                                                    row,
                                                    diff,
                                                    timestamp: lower.clone(),
                                                })
                                                .collect::<Vec<_>>();

                                            updates_outer.push((id, update, upper));
                                        }
                                        commands.push_front((
                                            span,
                                            PersistMonotonicWriteCmd::Append(updates_outer, response),
                                        ));
                                    }
                                    PersistMonotonicWriteCmd::Shutdown => {
                                        shutdown = true;
                                    }
                                }
                            }

                            async fn append_work<T2: Timestamp + Lattice + Codec64>(
                                frontier_responses: &mut tokio::sync::mpsc::UnboundedSender<
                                    StorageResponse<T2>,
                                >,
                                write_handles: &mut BTreeMap<
                                    GlobalId,
                                    WriteHandle<SourceData, (), T2, Diff>,
                                >,
                                mut commands: BTreeMap<
                                    GlobalId,
                                    (tracing::Span, Vec<Update<T2>>, Antichain<T2>),
                                >,
                            ) -> Result<(), Vec<GlobalId>> {
                                let futs = FuturesUnordered::new();

                                // We cannot iterate through the updates and then set off a persist call
                                // on the write handle because we cannot mutably borrow the write handle
                                // multiple times.
                                //
                                // Instead, we first group the update by ID above and then iterate
                                // through all available write handles and see if there are any updates
                                // for it. If yes, we send them all in one go.
                                for (id, write) in write_handles.iter_mut() {
                                    if let Some((span, updates, new_upper)) = commands.remove(id) {
                                        let persist_upper = write.upper().clone();
                                        let updates = updates
                                            .into_iter()
                                            .map(|u| ((SourceData(Ok(u.row)), ()), u.timestamp, u.diff));

                                        futs.push(async move {
                                            let persist_upper = persist_upper.clone();
                                            write
                                                .compare_and_append(
                                                    updates.clone(),
                                                    persist_upper.clone(),
                                                    new_upper.clone(),
                                                )
                                                .instrument(span.clone())
                                                .await
                                                .expect("cannot append updates")
                                                .or(Err(*id))?;

                                            Ok::<_, GlobalId>((*id, new_upper))
                                        })
                                    }
                                }

                                use futures::StreamExt;
                                // Ensure all futures run to completion, and track status of each of them individually
                                let (new_uppers, failed_appends): (Vec<_>, Vec<_>) = futs
                                    .collect::<Vec<_>>()
                                    .await
                                    .into_iter()
                                    .partition_result();

                                // It is not strictly an error for the controller to hang up.
                                let _ =
                                    frontier_responses.send(StorageResponse::FrontierUppers(new_uppers));

                                if failed_appends.is_empty() {
                                    Ok(())
                                } else {
                                    Err(failed_appends)
                                }
                            }

                            let result =
                                append_work(&mut frontier_responses, &mut write_handles, all_updates).await;

                            for (ids, response) in all_responses {
                                let result = match &result {
                                    Err(bad_ids) => {
                                        let filtered: Vec<_> = bad_ids.iter().filter(|id| ids.contains(id)).copied().collect();
                                        if filtered.is_empty() {
                                            Ok(())
                                        } else {
                                            Err(StorageError::InvalidUppers(filtered))
                                        }
                                    }
                                    Ok(()) => Ok(()),
                                };
                                // It is not an error for the other end to hang up.
                                let _ = response.send(result);
                            }

                            if shutdown {
                                tracing::trace!("shutting down persist write append task");
                                break;
                            }
                        } else {
                            shutdown = true;
                        }
                    }
                }
            }

            tracing::info!("PersistMonotonicWriteWorker shutting down");
        });

        Self {
            inner: Arc::new(PersistMonotonicWriteWorkerInner::new(tx)),
        }
    }

    pub(crate) fn register(
        &self,
        id: GlobalId,
        write_handle: WriteHandle<SourceData, (), T, Diff>,
    ) {
        self.send(PersistMonotonicWriteCmd::Register(id, write_handle))
    }

    /// Update the existing write handle associated with `id` to `write_handle`.
    ///
    /// Note that this should only be called when updating a write handle; to
    /// initially associate an `id` to a write handle, use [`Self::register`].
    ///
    /// # Panics
    /// - If `id` is not currently associated with any write handle.
    #[allow(dead_code)]
    pub(crate) fn update(&self, id: GlobalId, write_handle: WriteHandle<SourceData, (), T, Diff>) {
        self.send(PersistMonotonicWriteCmd::Update(id, write_handle))
    }

    /// Appends values to collections associated with `GlobalId`, but lets
    /// the persist worker chose timestamps guaranteed to be monotonic and
    /// that the time will be at least `T`.
    ///
    /// This lets the writer influence how far forward the timestamp will be
    /// advanced, while still guaranteeing that it will advance.
    ///
    /// Note it is still possible for the append operation to fail in the
    /// face of contention from other writers.
    ///
    /// # Panics
    /// - If appending non-empty `TimelessUpdate` to closed collections
    ///   (i.e. those with empty uppers), whose uppers cannot be
    ///   monotonically increased.
    ///
    ///   Collections with empty uppers can continue receiving empty
    ///   updates, i.e. those used soley to advance collections' uppers.
    pub(crate) fn monotonic_append(
        &self,
        updates: Vec<(GlobalId, Vec<TimestamplessUpdate>, T)>,
    ) -> tokio::sync::oneshot::Receiver<Result<(), StorageError>> {
        let (tx, rx) = tokio::sync::oneshot::channel();
        if updates.is_empty() {
            tx.send(Ok(()))
                .expect("rx has not been dropped at this point");
            rx
        } else {
            self.send(PersistMonotonicWriteCmd::MonotonicAppend(updates, tx));
            rx
        }
    }

    /// Drops the handle associated with `id` from this worker.
    ///
    /// Note that this does not perform any other cleanup, such as finalizing
    /// the handle's shard.
    pub(crate) fn drop_handle(&self, id: GlobalId) {
        self.send(PersistMonotonicWriteCmd::DropHandle(id))
    }

    fn send(&self, cmd: PersistMonotonicWriteCmd<T>) {
        self.inner.send(cmd);
    }
}

/// Contains the components necessary for sending commands to a `PersistMonotonicWriteWorker`.
///
/// When `Drop`-ed sends a shutdown command, as such this should _never_ implement `Clone` because
/// if one clone is dropped, the other clones will be unable to send commands. If you need this
/// to be `Clone`-able, wrap it in an `Arc` or `Rc` first.
///
/// #[derive(Clone)] <-- do not do this.
///
#[derive(Debug)]
struct PersistMonotonicWriteWorkerInner<T: Timestamp + Lattice + Codec64 + TimestampManipulation> {
    /// Sending side of a channel that we can use to send commands.
    tx: UnboundedSender<(tracing::Span, PersistMonotonicWriteCmd<T>)>,
}

impl<T> Drop for PersistMonotonicWriteWorkerInner<T>
where
    T: Timestamp + Lattice + Codec64 + TimestampManipulation,
{
    fn drop(&mut self) {
        self.send(PersistMonotonicWriteCmd::Shutdown);
        // TODO: Can't easily block on shutdown occurring.
    }
}

impl<T> PersistMonotonicWriteWorkerInner<T>
where
    T: Timestamp + Lattice + Codec64 + TimestampManipulation,
{
    fn new(tx: UnboundedSender<(tracing::Span, PersistMonotonicWriteCmd<T>)>) -> Self {
        PersistMonotonicWriteWorkerInner { tx }
    }

    fn send(&self, cmd: PersistMonotonicWriteCmd<T>) {
        match self.tx.send((tracing::Span::current(), cmd)) {
            Ok(()) => (), // All good!
            Err(e) => {
                tracing::trace!("could not forward command: {:?}", e);
            }
        }
    }
}

#[cfg(test)]
mod tests {
    use std::sync::Arc;

    use mz_build_info::DUMMY_BUILD_INFO;
    use mz_ore::metrics::MetricsRegistry;
    use mz_ore::now::SYSTEM_TIME;
    use mz_persist_client::cache::PersistClientCache;
    use mz_persist_client::cfg::PersistConfig;
    use mz_persist_client::rpc::PubSubClientConnection;
    use mz_persist_client::{Diagnostics, PersistClient, PersistLocation, ShardId};
    use mz_persist_types::codec_impls::UnitSchema;
    use mz_repr::{RelationDesc, Row};

    use super::*;

    #[mz_ore::test(tokio::test)]
    #[cfg_attr(miri, ignore)] // unsupported operation: integer-to-pointer casts and `ptr::from_exposed_addr`
    async fn snapshot_stats(&self) {
        let client = PersistClientCache::new(
            PersistConfig::new(&DUMMY_BUILD_INFO, SYSTEM_TIME.clone()),
            &MetricsRegistry::new(),
            |_, _| PubSubClientConnection::noop(),
        )
        .open(PersistLocation {
            blob_uri: "mem://".to_owned(),
            consensus_uri: "mem://".to_owned(),
        })
        .await
        .unwrap();
        let shard_id = ShardId::new();
        let since_handle = client
            .open_critical_since(
                shard_id,
                PersistClient::CONTROLLER_CRITICAL_SINCE,
                Diagnostics::for_tests(),
            )
            .await
            .unwrap();
        let mut write_handle = client
            .open_writer::<SourceData, (), u64, i64>(
                shard_id,
                Arc::new(RelationDesc::empty()),
                Arc::new(UnitSchema),
                Diagnostics::for_tests(),
            )
            .await
            .unwrap();

        let worker = PersistReadWorker::<u64>::new();
        worker.register(GlobalId::User(1), since_handle);

        // No stats for unknown GlobalId.
        let stats = worker
            .snapshot_stats(GlobalId::User(2), Antichain::from_elem(0))
            .await;
        assert!(stats.is_err());

        // Stats don't resolve for as_of past the upper.
        let stats_fut = worker.snapshot_stats(GlobalId::User(1), Antichain::from_elem(1));
        assert!(stats_fut.now_or_never().is_none());
        // Call it again because now_or_never consumed our future and it's not clone-able.
        let stats_ts1_fut = worker.snapshot_stats(GlobalId::User(1), Antichain::from_elem(1));

        // Write some data.
        let data = ((SourceData(Ok(Row::default())), ()), 0u64, 1i64);
        let () = write_handle
            .compare_and_append(&[data], Antichain::from_elem(0), Antichain::from_elem(1))
            .await
            .unwrap()
            .unwrap();

        // Verify that we can resolve stats for ts 0 while the ts 1 stats call is outstanding.
        let stats = worker
            .snapshot_stats(GlobalId::User(1), Antichain::from_elem(0))
            .await
            .unwrap();
        assert_eq!(stats.num_updates, 1);

        // Write more data and unblock the ts 1 call
        let data = ((SourceData(Ok(Row::default())), ()), 1u64, 1i64);
        let () = write_handle
            .compare_and_append(&[data], Antichain::from_elem(1), Antichain::from_elem(2))
            .await
            .unwrap()
            .unwrap();
        let stats = stats_ts1_fut.await.unwrap();
        assert_eq!(stats.num_updates, 2);
    }
}