mz_storage_client/
client.rs

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
// 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.

#![allow(missing_docs)]
// Tonic generates code that violates clippy lints.
// TODO: Remove this once tonic does not produce this code anymore.
#![allow(clippy::as_conversions, clippy::clone_on_ref_ptr)]

//! The public API of the storage layer.

use std::collections::{BTreeMap, BTreeSet};
use std::fmt::Debug;
use std::iter;

use async_trait::async_trait;
use differential_dataflow::difference::Semigroup;
use differential_dataflow::lattice::Lattice;
use mz_cluster_client::client::{ClusterStartupEpoch, TimelyConfig, TryIntoTimelyConfig};
use mz_ore::assert_none;
use mz_persist_client::batch::{BatchBuilder, ProtoBatch};
use mz_persist_client::write::WriteHandle;
use mz_persist_types::{Codec, Codec64, StepForward};
use mz_proto::{IntoRustIfSome, ProtoType, RustType, TryFromProtoError};
use mz_repr::{Diff, GlobalId, Row, TimestampManipulation};
use mz_service::client::{GenericClient, Partitionable, PartitionedState};
use mz_service::grpc::{GrpcClient, GrpcServer, ProtoServiceTypes, ResponseStream};
use mz_storage_types::controller::CollectionMetadata;
use mz_storage_types::oneshot_sources::OneshotIngestionRequest;
use mz_storage_types::parameters::StorageParameters;
use mz_storage_types::sinks::StorageSinkDesc;
use mz_storage_types::sources::IngestionDescription;
use mz_timely_util::progress::any_antichain;
use proptest::prelude::{any, Arbitrary};
use proptest::strategy::{BoxedStrategy, Strategy, Union};
use serde::{Deserialize, Serialize};
use smallvec::SmallVec;
use timely::progress::frontier::{Antichain, MutableAntichain};
use timely::progress::Timestamp;
use timely::PartialOrder;
use tonic::{Request, Status as TonicStatus, Streaming};
use uuid::Uuid;

use crate::client::proto_storage_server::ProtoStorage;
use crate::metrics::ReplicaMetrics;
use crate::statistics::{SinkStatisticsUpdate, SourceStatisticsUpdate};

include!(concat!(env!("OUT_DIR"), "/mz_storage_client.client.rs"));

/// A client to a storage server.
pub trait StorageClient<T = mz_repr::Timestamp>:
    GenericClient<StorageCommand<T>, StorageResponse<T>>
{
}

impl<C, T> StorageClient<T> for C where C: GenericClient<StorageCommand<T>, StorageResponse<T>> {}

#[async_trait]
impl<T: Send> GenericClient<StorageCommand<T>, StorageResponse<T>> for Box<dyn StorageClient<T>> {
    async fn send(&mut self, cmd: StorageCommand<T>) -> Result<(), anyhow::Error> {
        (**self).send(cmd).await
    }

    /// # Cancel safety
    ///
    /// This method is cancel safe. If `recv` is used as the event in a [`tokio::select!`]
    /// statement and some other branch completes first, it is guaranteed that no messages were
    /// received by this client.
    async fn recv(&mut self) -> Result<Option<StorageResponse<T>>, anyhow::Error> {
        // `GenericClient::recv` is required to be cancel safe.
        (**self).recv().await
    }
}

#[derive(Debug, Clone)]
pub enum StorageProtoServiceTypes {}

impl ProtoServiceTypes for StorageProtoServiceTypes {
    type PC = ProtoStorageCommand;
    type PR = ProtoStorageResponse;
    type STATS = ReplicaMetrics;
    const URL: &'static str = "/mz_storage_client.client.ProtoStorage/CommandResponseStream";
}

pub type StorageGrpcClient = GrpcClient<StorageProtoServiceTypes>;

#[async_trait]
impl<F, G> ProtoStorage for GrpcServer<F>
where
    F: Fn() -> G + Send + Sync + 'static,
    G: StorageClient + 'static,
{
    type CommandResponseStreamStream = ResponseStream<ProtoStorageResponse>;

    async fn command_response_stream(
        &self,
        request: Request<Streaming<ProtoStorageCommand>>,
    ) -> Result<tonic::Response<Self::CommandResponseStreamStream>, TonicStatus> {
        self.forward_bidi_stream(request).await
    }
}

/// Commands related to the ingress and egress of collections.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub enum StorageCommand<T = mz_repr::Timestamp> {
    /// Specifies to the storage server(s) the shape of the timely cluster
    /// we want created, before other commands are sent.
    CreateTimely {
        config: TimelyConfig,
        epoch: ClusterStartupEpoch,
    },
    /// Indicates that the controller has sent all commands reflecting its
    /// initial state.
    InitializationComplete,
    /// `AllowWrites` informs the replica that it can transition out of the
    /// read-only stage and into the read-write computation stage.
    /// It is now allowed to affect changes to external systems (writes).
    ///
    /// See `ComputeCommand::AllowWrites` for details. This command works
    /// analogously to the compute version.
    AllowWrites,
    /// Update storage instance configuration.
    UpdateConfiguration(StorageParameters),
    /// Run the enumerated sources, each associated with its identifier.
    RunIngestions(Vec<RunIngestionCommand>),
    /// Enable compaction in storage-managed collections.
    ///
    /// Each entry in the vector names a collection and provides a frontier after which
    /// accumulations must be correct.
    AllowCompaction(Vec<(GlobalId, Antichain<T>)>),
    RunSinks(Vec<RunSinkCommand<T>>),
    /// Run a dataflow which will ingest data from an external source and only __stage__ it in
    /// Persist.
    ///
    /// Unlike regular ingestions/sources, some other component (e.g. `environmentd`) is
    /// responsible for linking the staged data into a shard.
    RunOneshotIngestion(Vec<RunOneshotIngestion>),
    /// `CancelOneshotIngestion` instructs the replica to cancel the identified oneshot ingestions.
    ///
    /// It is invalid to send a [`CancelOneshotIngestion`] command that references a oneshot
    /// ingestion that was not created by a corresponding [`RunOneshotIngestion`] command before.
    /// Doing so may cause the replica to exhibit undefined behavior.
    ///
    /// [`CancelOneshotIngestion`]: crate::client::StorageCommand::CancelOneshotIngestion
    /// [`RunOneshotIngestion`]: crate::client::StorageCommand::RunOneshotIngestion
    CancelOneshotIngestion {
        ingestions: Vec<Uuid>,
    },
}

impl<T> StorageCommand<T> {
    /// Returns whether this command instructs the installation of storage objects.
    pub fn installs_objects(&self) -> bool {
        use StorageCommand::*;
        match self {
            CreateTimely { .. }
            | InitializationComplete
            | AllowWrites
            | UpdateConfiguration(_)
            | AllowCompaction(_)
            | CancelOneshotIngestion { .. } => false,
            // TODO(cf2): multi-replica oneshot ingestions. At the moment returning
            // true here means we can't run `COPY FROM` on multi-replica clusters, this
            // should be easy enough to support though.
            RunIngestions(_) | RunSinks(_) | RunOneshotIngestion(_) => true,
        }
    }
}

/// A command that starts ingesting the given ingestion description
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub struct RunIngestionCommand {
    /// The id of the storage collection being ingested.
    pub id: GlobalId,
    /// The description of what source type should be ingested and what post-processing steps must
    /// be applied to the data before writing them down into the storage collection
    pub description: IngestionDescription<CollectionMetadata>,
}

impl Arbitrary for RunIngestionCommand {
    type Strategy = BoxedStrategy<Self>;
    type Parameters = ();

    fn arbitrary_with(_: Self::Parameters) -> Self::Strategy {
        (
            any::<GlobalId>(),
            any::<IngestionDescription<CollectionMetadata>>(),
        )
            .prop_map(|(id, description)| Self { id, description })
            .boxed()
    }
}

impl RustType<ProtoRunIngestionCommand> for RunIngestionCommand {
    fn into_proto(&self) -> ProtoRunIngestionCommand {
        ProtoRunIngestionCommand {
            id: Some(self.id.into_proto()),
            description: Some(self.description.into_proto()),
        }
    }

    fn from_proto(proto: ProtoRunIngestionCommand) -> Result<Self, TryFromProtoError> {
        Ok(RunIngestionCommand {
            id: proto.id.into_rust_if_some("ProtoRunIngestionCommand::id")?,
            description: proto
                .description
                .into_rust_if_some("ProtoRunIngestionCommand::description")?,
        })
    }
}

/// A command that starts ingesting the given ingestion description
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub struct RunOneshotIngestion {
    /// The ID of the ingestion dataflow.
    pub ingestion_id: uuid::Uuid,
    /// The ID of collection we'll stage batches for.
    pub collection_id: GlobalId,
    /// Metadata for the collection we'll stage batches for.
    pub collection_meta: CollectionMetadata,
    /// Details for the oneshot ingestion.
    pub request: OneshotIngestionRequest,
}

impl RustType<ProtoRunOneshotIngestion> for RunOneshotIngestion {
    fn into_proto(&self) -> ProtoRunOneshotIngestion {
        ProtoRunOneshotIngestion {
            ingestion_id: Some(self.ingestion_id.into_proto()),
            collection_id: Some(self.collection_id.into_proto()),
            storage_metadata: Some(self.collection_meta.into_proto()),
            request: Some(self.request.into_proto()),
        }
    }

    fn from_proto(proto: ProtoRunOneshotIngestion) -> Result<Self, TryFromProtoError> {
        Ok(RunOneshotIngestion {
            ingestion_id: proto
                .ingestion_id
                .into_rust_if_some("ProtoRunOneshotIngestion::ingestion_id")?,
            collection_id: proto
                .collection_id
                .into_rust_if_some("ProtoRunOneshotIngestion::collection_id")?,
            collection_meta: proto
                .storage_metadata
                .into_rust_if_some("ProtoRunOneshotIngestion::storage_metadata")?,
            request: proto
                .request
                .into_rust_if_some("ProtoRunOneshotIngestion::request")?,
        })
    }
}

impl RustType<ProtoRunSinkCommand> for RunSinkCommand<mz_repr::Timestamp> {
    fn into_proto(&self) -> ProtoRunSinkCommand {
        ProtoRunSinkCommand {
            id: Some(self.id.into_proto()),
            description: Some(self.description.into_proto()),
        }
    }

    fn from_proto(proto: ProtoRunSinkCommand) -> Result<Self, TryFromProtoError> {
        Ok(RunSinkCommand {
            id: proto.id.into_rust_if_some("ProtoRunSinkCommand::id")?,
            description: proto
                .description
                .into_rust_if_some("ProtoRunSinkCommand::description")?,
        })
    }
}

/// A command that starts exporting the given sink description
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub struct RunSinkCommand<T> {
    pub id: GlobalId,
    pub description: StorageSinkDesc<CollectionMetadata, T>,
}

impl Arbitrary for RunSinkCommand<mz_repr::Timestamp> {
    type Strategy = BoxedStrategy<Self>;
    type Parameters = ();

    fn arbitrary_with(_: Self::Parameters) -> Self::Strategy {
        (
            any::<GlobalId>(),
            any::<StorageSinkDesc<CollectionMetadata, mz_repr::Timestamp>>(),
        )
            .prop_map(|(id, description)| Self { id, description })
            .boxed()
    }
}

impl RustType<ProtoStorageCommand> for StorageCommand<mz_repr::Timestamp> {
    fn into_proto(&self) -> ProtoStorageCommand {
        use proto_storage_command::Kind::*;
        use proto_storage_command::*;
        ProtoStorageCommand {
            kind: Some(match self {
                StorageCommand::CreateTimely { config, epoch } => CreateTimely(ProtoCreateTimely {
                    config: Some(config.into_proto()),
                    epoch: Some(epoch.into_proto()),
                }),
                StorageCommand::InitializationComplete => InitializationComplete(()),
                StorageCommand::AllowWrites => AllowWrites(()),
                StorageCommand::UpdateConfiguration(params) => {
                    UpdateConfiguration(params.into_proto())
                }
                StorageCommand::AllowCompaction(collections) => {
                    AllowCompaction(ProtoAllowCompaction {
                        collections: collections.into_proto(),
                    })
                }
                StorageCommand::RunIngestions(sources) => CreateSources(ProtoCreateSources {
                    sources: sources.into_proto(),
                }),
                StorageCommand::RunSinks(sinks) => RunSinks(ProtoRunSinks {
                    sinks: sinks.into_proto(),
                }),
                StorageCommand::RunOneshotIngestion(ingestions) => {
                    RunOneshotIngestions(ProtoRunOneshotIngestionsCommand {
                        ingestions: ingestions.iter().map(|cmd| cmd.into_proto()).collect(),
                    })
                }
                StorageCommand::CancelOneshotIngestion { ingestions } => {
                    CancelOneshotIngestions(ProtoCancelOneshotIngestionsCommand {
                        ingestions: ingestions.iter().map(|uuid| uuid.into_proto()).collect(),
                    })
                }
            }),
        }
    }

    fn from_proto(proto: ProtoStorageCommand) -> Result<Self, TryFromProtoError> {
        use proto_storage_command::Kind::*;
        use proto_storage_command::*;
        match proto.kind {
            Some(CreateTimely(ProtoCreateTimely { config, epoch })) => {
                Ok(StorageCommand::CreateTimely {
                    config: config.into_rust_if_some("ProtoCreateTimely::config")?,
                    epoch: epoch.into_rust_if_some("ProtoCreateTimely::epoch")?,
                })
            }
            Some(InitializationComplete(())) => Ok(StorageCommand::InitializationComplete),
            Some(AllowWrites(())) => Ok(StorageCommand::AllowWrites),
            Some(UpdateConfiguration(params)) => {
                Ok(StorageCommand::UpdateConfiguration(params.into_rust()?))
            }
            Some(CreateSources(ProtoCreateSources { sources })) => {
                Ok(StorageCommand::RunIngestions(sources.into_rust()?))
            }
            Some(AllowCompaction(ProtoAllowCompaction { collections })) => {
                Ok(StorageCommand::AllowCompaction(collections.into_rust()?))
            }
            Some(RunSinks(ProtoRunSinks { sinks })) => {
                Ok(StorageCommand::RunSinks(sinks.into_rust()?))
            }
            Some(RunOneshotIngestions(oneshot)) => {
                let ingestions = oneshot
                    .ingestions
                    .into_iter()
                    .map(|cmd| cmd.into_rust())
                    .collect::<Result<_, _>>()?;
                Ok(StorageCommand::RunOneshotIngestion(ingestions))
            }
            Some(CancelOneshotIngestions(oneshot)) => {
                let ingestions = oneshot
                    .ingestions
                    .into_iter()
                    .map(|uuid| uuid.into_rust())
                    .collect::<Result<_, _>>()?;
                Ok(StorageCommand::CancelOneshotIngestion { ingestions })
            }
            None => Err(TryFromProtoError::missing_field(
                "ProtoStorageCommand::kind",
            )),
        }
    }
}

impl Arbitrary for StorageCommand<mz_repr::Timestamp> {
    type Strategy = Union<BoxedStrategy<Self>>;
    type Parameters = ();

    fn arbitrary_with(_: Self::Parameters) -> Self::Strategy {
        Union::new(vec![
            // TODO(guswynn): cluster-unification: also test `CreateTimely` here.
            proptest::collection::vec(any::<RunIngestionCommand>(), 1..4)
                .prop_map(StorageCommand::RunIngestions)
                .boxed(),
            proptest::collection::vec(any::<RunSinkCommand<mz_repr::Timestamp>>(), 1..4)
                .prop_map(StorageCommand::RunSinks)
                .boxed(),
            proptest::collection::vec(
                (
                    any::<GlobalId>(),
                    proptest::collection::vec(any::<mz_repr::Timestamp>(), 1..4),
                ),
                1..4,
            )
            .prop_map(|collections| {
                StorageCommand::AllowCompaction(
                    collections
                        .into_iter()
                        .map(|(id, frontier_vec)| (id, Antichain::from(frontier_vec)))
                        .collect(),
                )
            })
            .boxed(),
        ])
    }
}

/// A "kind" enum for statuses tracked by the health operator
#[derive(Copy, Clone, Debug, Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord)]
pub enum Status {
    Starting,
    Running,
    Paused,
    Stalled,
    /// This status is currently unused.
    // re-design the ceased status
    Ceased,
    Dropped,
}

impl std::str::FromStr for Status {
    type Err = anyhow::Error;
    /// Keep in sync with [`Status::to_str`].
    fn from_str(s: &str) -> Result<Self, Self::Err> {
        Ok(match s {
            "starting" => Status::Starting,
            "running" => Status::Running,
            "paused" => Status::Paused,
            "stalled" => Status::Stalled,
            "ceased" => Status::Ceased,
            "dropped" => Status::Dropped,
            s => return Err(anyhow::anyhow!("{} is not a valid status", s)),
        })
    }
}

impl Status {
    /// Keep in sync with `Status::from_str`.
    pub fn to_str(&self) -> &'static str {
        match self {
            Status::Starting => "starting",
            Status::Running => "running",
            Status::Paused => "paused",
            Status::Stalled => "stalled",
            Status::Ceased => "ceased",
            Status::Dropped => "dropped",
        }
    }

    /// Determines if a new status should be produced in context of a previous
    /// status.
    pub fn superseded_by(self, new: Status) -> bool {
        match (self, new) {
            (_, Status::Dropped) => true,
            (Status::Dropped, _) => false,
            // Don't re-mark that object as paused.
            (Status::Paused, Status::Paused) => false,
            // De-duplication of other statuses is currently managed by the
            // `health_operator`.
            _ => true,
        }
    }
}

/// A source or sink status update.
///
/// Represents a status update for a given object type. The inner value for each
/// variant should be able to be packed into a status row that conforms to the schema
/// for the object's status history relation.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub struct StatusUpdate {
    pub id: GlobalId,
    pub status: Status,
    pub timestamp: chrono::DateTime<chrono::Utc>,
    pub error: Option<String>,
    pub hints: BTreeSet<String>,
    pub namespaced_errors: BTreeMap<String, String>,
}

impl StatusUpdate {
    pub fn new(
        id: GlobalId,
        timestamp: chrono::DateTime<chrono::Utc>,
        status: Status,
    ) -> StatusUpdate {
        StatusUpdate {
            id,
            timestamp,
            status,
            error: None,
            hints: Default::default(),
            namespaced_errors: Default::default(),
        }
    }
}

impl From<StatusUpdate> for Row {
    fn from(update: StatusUpdate) -> Self {
        use mz_repr::Datum;

        let timestamp = Datum::TimestampTz(update.timestamp.try_into().expect("must fit"));
        let id = update.id.to_string();
        let id = Datum::String(&id);
        let status = Datum::String(update.status.to_str());
        let error = update.error.as_deref().into();

        let mut row = Row::default();
        let mut packer = row.packer();
        packer.extend([timestamp, id, status, error]);

        if !update.hints.is_empty() || !update.namespaced_errors.is_empty() {
            packer.push_dict_with(|dict_packer| {
                // `hint` and `namespaced` are ordered,
                // as well as the BTree's they each contain.
                if !update.hints.is_empty() {
                    dict_packer.push(Datum::String("hints"));
                    dict_packer.push_list(update.hints.iter().map(|s| Datum::String(s)));
                }
                if !update.namespaced_errors.is_empty() {
                    dict_packer.push(Datum::String("namespaced"));
                    dict_packer.push_dict(
                        update
                            .namespaced_errors
                            .iter()
                            .map(|(k, v)| (k.as_str(), Datum::String(v))),
                    );
                }
            });
        } else {
            packer.push(Datum::Null);
        }

        row
    }
}

impl RustType<proto_storage_response::ProtoStatus> for Status {
    fn into_proto(&self) -> proto_storage_response::ProtoStatus {
        use proto_storage_response::proto_status::*;

        proto_storage_response::ProtoStatus {
            kind: Some(match self {
                Status::Starting => Kind::Starting(()),
                Status::Running => Kind::Running(()),
                Status::Paused => Kind::Paused(()),
                Status::Stalled => Kind::Stalled(()),
                Status::Ceased => Kind::Ceased(()),
                Status::Dropped => Kind::Dropped(()),
            }),
        }
    }

    fn from_proto(proto: proto_storage_response::ProtoStatus) -> Result<Self, TryFromProtoError> {
        use proto_storage_response::proto_status::*;
        let kind = proto
            .kind
            .ok_or_else(|| TryFromProtoError::missing_field("ProtoStatus::kind"))?;

        Ok(match kind {
            Kind::Starting(()) => Status::Starting,
            Kind::Running(()) => Status::Running,
            Kind::Paused(()) => Status::Paused,
            Kind::Stalled(()) => Status::Stalled,
            Kind::Ceased(()) => Status::Ceased,
            Kind::Dropped(()) => Status::Dropped,
        })
    }
}

impl RustType<proto_storage_response::ProtoStatusUpdate> for StatusUpdate {
    fn into_proto(&self) -> proto_storage_response::ProtoStatusUpdate {
        proto_storage_response::ProtoStatusUpdate {
            id: Some(self.id.into_proto()),
            status: Some(self.status.into_proto()),
            timestamp: Some(self.timestamp.into_proto()),
            error: self.error.clone(),
            hints: self.hints.iter().cloned().collect(),
            namespaced_errors: self.namespaced_errors.clone(),
        }
    }

    fn from_proto(
        proto: proto_storage_response::ProtoStatusUpdate,
    ) -> Result<Self, TryFromProtoError> {
        Ok(StatusUpdate {
            id: proto.id.into_rust_if_some("ProtoStatusUpdate::id")?,
            timestamp: proto
                .timestamp
                .into_rust_if_some("ProtoStatusUpdate::timestamp")?,
            status: proto
                .status
                .into_rust_if_some("ProtoStatusUpdate::status")?,
            error: proto.error,
            hints: proto.hints.into_iter().collect(),
            namespaced_errors: proto.namespaced_errors,
        })
    }
}

/// An update to an append only collection.
pub enum AppendOnlyUpdate {
    Row((Row, Diff)),
    Status(StatusUpdate),
}

impl AppendOnlyUpdate {
    pub fn into_row(self) -> (Row, Diff) {
        match self {
            AppendOnlyUpdate::Row((row, diff)) => (row, diff),
            AppendOnlyUpdate::Status(status) => (Row::from(status), 1),
        }
    }
}

impl From<(Row, Diff)> for AppendOnlyUpdate {
    fn from((row, diff): (Row, Diff)) -> Self {
        Self::Row((row, diff))
    }
}

impl From<StatusUpdate> for AppendOnlyUpdate {
    fn from(update: StatusUpdate) -> Self {
        Self::Status(update)
    }
}

/// Responses that the storage nature of a worker/dataflow can provide back to the coordinator.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub enum StorageResponse<T = mz_repr::Timestamp> {
    /// A list of identifiers of traces, with new upper frontiers.
    FrontierUppers(Vec<(GlobalId, Antichain<T>)>),
    /// Punctuation indicates that no more responses will be transmitted for the specified id
    DroppedId(GlobalId),
    /// Batches that have been staged in Persist and maybe will be linked into a shard.
    StagedBatches(BTreeMap<uuid::Uuid, Vec<Result<ProtoBatch, String>>>),

    /// A list of statistics updates, currently only for sources.
    StatisticsUpdates(Vec<SourceStatisticsUpdate>, Vec<SinkStatisticsUpdate>),
    /// A list of status updates for sources and sinks. Periodically sent from
    /// storage workers to convey the latest status information about an object.
    StatusUpdates(Vec<StatusUpdate>),
}

impl RustType<ProtoStorageResponse> for StorageResponse<mz_repr::Timestamp> {
    fn into_proto(&self) -> ProtoStorageResponse {
        use proto_storage_response::Kind::*;
        use proto_storage_response::{
            ProtoDroppedId, ProtoStagedBatches, ProtoStatisticsUpdates, ProtoStatusUpdates,
        };
        ProtoStorageResponse {
            kind: Some(match self {
                StorageResponse::FrontierUppers(traces) => FrontierUppers(traces.into_proto()),
                StorageResponse::DroppedId(id) => DroppedId(ProtoDroppedId {
                    id: Some(id.into_proto()),
                }),
                StorageResponse::StatisticsUpdates(source_stats, sink_stats) => {
                    Stats(ProtoStatisticsUpdates {
                        source_updates: source_stats
                            .iter()
                            .map(|update| update.into_proto())
                            .collect(),
                        sink_updates: sink_stats
                            .iter()
                            .map(|update| update.into_proto())
                            .collect(),
                    })
                }
                StorageResponse::StatusUpdates(updates) => StatusUpdates(ProtoStatusUpdates {
                    updates: updates.into_proto(),
                }),
                StorageResponse::StagedBatches(staged) => {
                    let batches = staged
                        .into_iter()
                        .map(|(collection_id, batches)| {
                            let batches = batches
                                .into_iter()
                                .map(|result| {
                                    use proto_storage_response::proto_staged_batches::batch_result::Value;
                                    let value = match result {
                                        Ok(batch) => Value::Batch(batch.clone()),
                                        Err(err) => Value::Error(err.clone()),
                                    };
                                    proto_storage_response::proto_staged_batches::BatchResult { value: Some(value) }
                                })
                                .collect();
                            proto_storage_response::proto_staged_batches::Inner {
                                id: Some(collection_id.into_proto()),
                                batches,
                            }
                        })
                        .collect();
                    StagedBatches(ProtoStagedBatches { batches })
                }
            }),
        }
    }

    fn from_proto(proto: ProtoStorageResponse) -> Result<Self, TryFromProtoError> {
        use proto_storage_response::Kind::*;
        use proto_storage_response::{ProtoDroppedId, ProtoStatusUpdates};
        match proto.kind {
            Some(DroppedId(ProtoDroppedId { id })) => Ok(StorageResponse::DroppedId(
                id.into_rust_if_some("ProtoDroppedId::id")?,
            )),
            Some(FrontierUppers(traces)) => {
                Ok(StorageResponse::FrontierUppers(traces.into_rust()?))
            }
            Some(Stats(stats)) => Ok(StorageResponse::StatisticsUpdates(
                stats
                    .source_updates
                    .into_iter()
                    .map(|update| update.into_rust())
                    .collect::<Result<Vec<_>, TryFromProtoError>>()?,
                stats
                    .sink_updates
                    .into_iter()
                    .map(|update| update.into_rust())
                    .collect::<Result<Vec<_>, TryFromProtoError>>()?,
            )),
            Some(StatusUpdates(ProtoStatusUpdates { updates })) => {
                Ok(StorageResponse::StatusUpdates(updates.into_rust()?))
            }
            Some(StagedBatches(staged)) => {
                let batches: BTreeMap<_, _> = staged
                    .batches
                    .into_iter()
                    .map(|inner| {
                        let id = inner
                            .id
                            .into_rust_if_some("ProtoStagedBatches::Inner::id")?;

                        let mut batches = Vec::with_capacity(inner.batches.len());
                        for maybe_batch in inner.batches {
                            use proto_storage_response::proto_staged_batches::batch_result::Value;

                            let value = maybe_batch.value.ok_or_else(|| {
                                TryFromProtoError::missing_field("BatchResult::value")
                            })?;
                            let batch = match value {
                                Value::Batch(batch) => Ok(batch),
                                Value::Error(err) => Err(err),
                            };
                            batches.push(batch);
                        }

                        Ok::<_, TryFromProtoError>((id, batches))
                    })
                    .collect::<Result<_, _>>()?;

                Ok(StorageResponse::StagedBatches(batches))
            }
            None => Err(TryFromProtoError::missing_field(
                "ProtoStorageResponse::kind",
            )),
        }
    }
}

impl Arbitrary for StorageResponse<mz_repr::Timestamp> {
    type Strategy = Union<BoxedStrategy<Self>>;
    type Parameters = ();

    fn arbitrary_with(_: Self::Parameters) -> Self::Strategy {
        // TODO(guswynn): test `SourceStatisticsUpdates`
        Union::new(vec![proptest::collection::vec(
            (any::<GlobalId>(), any_antichain()),
            1..4,
        )
        .prop_map(StorageResponse::FrontierUppers)
        .boxed()])
    }
}

/// Maintained state for partitioned storage clients.
///
/// This helper type unifies the responses of multiple partitioned
/// workers in order to present as a single worker.
#[derive(Debug)]
pub struct PartitionedStorageState<T> {
    /// Number of partitions the state machine represents.
    parts: usize,
    /// Upper frontiers for sources and sinks, both unioned across all partitions and from each
    /// individual partition.
    uppers: BTreeMap<GlobalId, (MutableAntichain<T>, Vec<Option<Antichain<T>>>)>,
    /// Staged batches from oneshot sources that will get appended by `environmentd`.
    oneshot_source_responses:
        BTreeMap<uuid::Uuid, BTreeMap<usize, Vec<Result<ProtoBatch, String>>>>,
}

impl<T> Partitionable<StorageCommand<T>, StorageResponse<T>>
    for (StorageCommand<T>, StorageResponse<T>)
where
    T: timely::progress::Timestamp + Lattice,
{
    type PartitionedState = PartitionedStorageState<T>;

    fn new(parts: usize) -> PartitionedStorageState<T> {
        PartitionedStorageState {
            parts,
            uppers: BTreeMap::new(),
            oneshot_source_responses: BTreeMap::new(),
        }
    }
}

impl<T> PartitionedStorageState<T>
where
    T: timely::progress::Timestamp,
{
    fn observe_command(&mut self, command: &StorageCommand<T>) {
        // Note that `observe_command` is quite different in `mz_compute_client`.
        // Compute (currently) only sends the command to 1 process,
        // but storage fans out to all workers, allowing the storage processes
        // to self-coordinate how commands and internal commands are ordered.
        //
        // TODO(guswynn): cluster-unification: consolidate this with compute.
        let _ = match command {
            StorageCommand::CreateTimely { .. } => {
                // Similarly, we don't reset state here like compute, because,
                // until we are required to manage multiple replicas, we can handle
                // keeping track of state across restarts of storage server(s).
            }
            StorageCommand::RunIngestions(ingestions) => ingestions
                .iter()
                .for_each(|i| self.insert_new_uppers(i.description.collection_ids())),
            StorageCommand::RunSinks(exports) => {
                exports.iter().for_each(|e| self.insert_new_uppers([e.id]))
            }
            StorageCommand::InitializationComplete
            | StorageCommand::AllowWrites
            | StorageCommand::UpdateConfiguration(_)
            | StorageCommand::AllowCompaction(_)
            | StorageCommand::RunOneshotIngestion(_)
            | StorageCommand::CancelOneshotIngestion { .. } => {}
        };
    }

    /// Shared implementation for commands that install uppers with controllable behavior with
    /// encountering existing uppers.
    ///
    /// If any ID was previously tracked in `self` and `skip_existing` is `false`, we return the ID
    /// as an error.
    fn insert_new_uppers<I: IntoIterator<Item = GlobalId>>(&mut self, ids: I) {
        for id in ids {
            self.uppers.entry(id).or_insert_with(|| {
                let mut frontier = MutableAntichain::new();
                // TODO(guswynn): cluster-unification: fix this dangerous use of `as`, by
                // merging the types that compute and storage use.
                #[allow(clippy::as_conversions)]
                frontier.update_iter(iter::once((T::minimum(), self.parts as i64)));
                let part_frontiers = vec![Some(Antichain::from_elem(T::minimum())); self.parts];

                (frontier, part_frontiers)
            });
        }
    }
}

impl<T> PartitionedState<StorageCommand<T>, StorageResponse<T>> for PartitionedStorageState<T>
where
    T: timely::progress::Timestamp + Lattice,
{
    fn split_command(&mut self, command: StorageCommand<T>) -> Vec<Option<StorageCommand<T>>> {
        self.observe_command(&command);

        match command {
            StorageCommand::CreateTimely { config, epoch } => {
                let timely_cmds = config.split_command(self.parts);

                let timely_cmds = timely_cmds
                    .into_iter()
                    .map(|config| Some(StorageCommand::CreateTimely { config, epoch }))
                    .collect();
                timely_cmds
            }
            command => {
                // Fan out to all processes (which will fan out to all workers).
                // StorageState manages ordering of commands internally.
                vec![Some(command); self.parts]
            }
        }
    }

    fn absorb_response(
        &mut self,
        shard_id: usize,
        response: StorageResponse<T>,
    ) -> Option<Result<StorageResponse<T>, anyhow::Error>> {
        match response {
            // Avoid multiple retractions of minimum time, to present as updates from one worker.
            StorageResponse::FrontierUppers(list) => {
                let mut new_uppers = Vec::new();

                for (id, new_shard_upper) in list {
                    let (frontier, shard_frontiers) = match self.uppers.get_mut(&id) {
                        Some(value) => value,
                        None => panic!("Reference to absent collection: {id}"),
                    };
                    let old_upper = frontier.frontier().to_owned();
                    let shard_upper = match &mut shard_frontiers[shard_id] {
                        Some(shard_upper) => shard_upper,
                        None => panic!("Reference to absent shard {shard_id} for collection {id}"),
                    };
                    frontier.update_iter(shard_upper.iter().map(|t| (t.clone(), -1)));
                    frontier.update_iter(new_shard_upper.iter().map(|t| (t.clone(), 1)));
                    shard_upper.join_assign(&new_shard_upper);

                    let new_upper = frontier.frontier();
                    if PartialOrder::less_than(&old_upper.borrow(), &new_upper) {
                        new_uppers.push((id, new_upper.to_owned()));
                    }
                }

                if new_uppers.is_empty() {
                    None
                } else {
                    Some(Ok(StorageResponse::FrontierUppers(new_uppers)))
                }
            }
            StorageResponse::DroppedId(id) => {
                let (_, shard_frontiers) = match self.uppers.get_mut(&id) {
                    Some(value) => value,
                    None => panic!("Reference to absent collection: {id}"),
                };
                let prev = shard_frontiers[shard_id].take();
                assert!(
                    prev.is_some(),
                    "got double drop for {id} from shard {shard_id}"
                );

                if shard_frontiers.iter().all(Option::is_none) {
                    self.uppers.remove(&id);
                    Some(Ok(StorageResponse::DroppedId(id)))
                } else {
                    None
                }
            }
            StorageResponse::StatisticsUpdates(source_stats, sink_stats) => {
                // Just forward it along; the `worker_id` should have been set in `storage_state`.
                // We _could_ consolidate across worker_id's, here, but each worker only produces
                // responses periodically, so we avoid that complexity.
                Some(Ok(StorageResponse::StatisticsUpdates(
                    source_stats,
                    sink_stats,
                )))
            }
            StorageResponse::StatusUpdates(updates) => {
                Some(Ok(StorageResponse::StatusUpdates(updates)))
            }
            StorageResponse::StagedBatches(batches) => {
                let mut finished_batches = BTreeMap::new();

                for (collection_id, batches) in batches {
                    tracing::info!(%shard_id, %collection_id, "got batch");

                    let entry = self
                        .oneshot_source_responses
                        .entry(collection_id)
                        .or_default();
                    let novel = entry.insert(shard_id, batches);
                    assert_none!(novel, "Duplicate oneshot source response");

                    // Check if we've received responses from all shards.
                    if entry.len() == self.parts {
                        let entry = self
                            .oneshot_source_responses
                            .remove(&collection_id)
                            .expect("checked above");
                        let all_batches: Vec<_> = entry.into_values().flatten().collect();

                        finished_batches.insert(collection_id, all_batches);
                    }
                }

                if !finished_batches.is_empty() {
                    Some(Ok(StorageResponse::StagedBatches(finished_batches)))
                } else {
                    None
                }
            }
        }
    }
}

#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)]
/// A batch of updates to be fed to a local input
pub struct Update<T = mz_repr::Timestamp> {
    pub row: Row,
    pub timestamp: T,
    pub diff: Diff,
}

#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)]
/// A batch of updates to be fed to a local input; however, the input must
/// determine the most appropriate timestamps to use.
///
/// TODO(cf2): Can we remove this and use only on [`TableData`].
pub struct TimestamplessUpdate {
    pub row: Row,
    pub diff: Diff,
}

#[derive(Debug, Clone, PartialEq)]
pub enum TableData {
    /// Rows that still need to be persisted and appended.
    ///
    /// The contained [`Row`]s are _not_ consolidated.
    Rows(Vec<(Row, Diff)>),
    /// Batches already staged in Persist ready to be appended.
    Batches(SmallVec<[ProtoBatch; 1]>),
}

impl TableData {
    pub fn is_empty(&self) -> bool {
        match self {
            TableData::Rows(rows) => rows.is_empty(),
            TableData::Batches(batches) => batches.is_empty(),
        }
    }
}

/// A collection of timestamp-less updates. As updates are added to the builder
/// they are automatically spilled to blob storage.
pub struct TimestamplessUpdateBuilder<K, V, T, D>
where
    K: Codec,
    V: Codec,
    T: Timestamp + Lattice + Codec64,
    D: Codec64,
{
    builder: BatchBuilder<K, V, T, D>,
    initial_ts: T,
}

impl<K, V, T, D> TimestamplessUpdateBuilder<K, V, T, D>
where
    K: Debug + Codec,
    V: Debug + Codec,
    T: TimestampManipulation + Lattice + Codec64 + Sync,
    D: Semigroup + Ord + Codec64 + Send + Sync,
{
    /// Create a new [`TimestamplessUpdateBuilder`] for the shard associated
    /// with the provided [`WriteHandle`].
    pub fn new(handle: &WriteHandle<K, V, T, D>) -> Self {
        let initial_ts = T::minimum();
        let builder = handle.builder(Antichain::from_elem(initial_ts.clone()));
        TimestamplessUpdateBuilder {
            builder,
            initial_ts,
        }
    }

    /// Add a `(K, V, D)` to the staged batch.
    pub async fn add(&mut self, k: &K, v: &V, d: &D) {
        self.builder
            .add(k, v, &self.initial_ts, d)
            .await
            .expect("invalid Persist usage");
    }

    /// Finish the builder and return a [`ProtoBatch`] which can later be linked into a shard.
    ///
    /// The returned batch has nonsensical lower and upper bounds and must be re-written before
    /// appending into the destination shard.
    pub async fn finish(self) -> ProtoBatch {
        let finish_ts = StepForward::step_forward(&self.initial_ts);
        let batch = self
            .builder
            .finish(Antichain::from_elem(finish_ts))
            .await
            .expect("invalid Persist usage");

        batch.into_transmittable_batch()
    }
}

impl RustType<ProtoTrace> for (GlobalId, Antichain<mz_repr::Timestamp>) {
    fn into_proto(&self) -> ProtoTrace {
        ProtoTrace {
            id: Some(self.0.into_proto()),
            upper: Some(self.1.into_proto()),
        }
    }

    fn from_proto(proto: ProtoTrace) -> Result<Self, TryFromProtoError> {
        Ok((
            proto.id.into_rust_if_some("ProtoTrace::id")?,
            proto.upper.into_rust_if_some("ProtoTrace::upper")?,
        ))
    }
}

impl RustType<ProtoFrontierUppersKind> for Vec<(GlobalId, Antichain<mz_repr::Timestamp>)> {
    fn into_proto(&self) -> ProtoFrontierUppersKind {
        ProtoFrontierUppersKind {
            traces: self.into_proto(),
        }
    }

    fn from_proto(proto: ProtoFrontierUppersKind) -> Result<Self, TryFromProtoError> {
        proto.traces.into_rust()
    }
}

impl RustType<ProtoCompaction> for (GlobalId, Antichain<mz_repr::Timestamp>) {
    fn into_proto(&self) -> ProtoCompaction {
        ProtoCompaction {
            id: Some(self.0.into_proto()),
            frontier: Some(self.1.into_proto()),
        }
    }

    fn from_proto(proto: ProtoCompaction) -> Result<Self, TryFromProtoError> {
        Ok((
            proto.id.into_rust_if_some("ProtoCompaction::id")?,
            proto
                .frontier
                .into_rust_if_some("ProtoCompaction::frontier")?,
        ))
    }
}

impl TryIntoTimelyConfig for StorageCommand {
    fn try_into_timely_config(self) -> Result<(TimelyConfig, ClusterStartupEpoch), Self> {
        match self {
            StorageCommand::CreateTimely { config, epoch } => Ok((config, epoch)),
            cmd => Err(cmd),
        }
    }
}

#[cfg(test)]
mod tests {
    use mz_ore::assert_ok;
    use mz_proto::protobuf_roundtrip;
    use proptest::prelude::ProptestConfig;
    use proptest::proptest;

    use super::*;

    proptest! {
        #![proptest_config(ProptestConfig::with_cases(32))]

        #[mz_ore::test]
        #[cfg_attr(miri, ignore)] // too slow
        fn storage_command_protobuf_roundtrip(expect in any::<StorageCommand<mz_repr::Timestamp>>() ) {
            let actual = protobuf_roundtrip::<_, ProtoStorageCommand>(&expect);
            assert_ok!(actual);
            assert_eq!(actual.unwrap(), expect);
        }

        #[mz_ore::test]
        #[cfg_attr(miri, ignore)] // too slow
        fn storage_response_protobuf_roundtrip(expect in any::<StorageResponse<mz_repr::Timestamp>>() ) {
            let actual = protobuf_roundtrip::<_, ProtoStorageResponse>(&expect);
            assert_ok!(actual);
            assert_eq!(actual.unwrap(), expect);
        }
    }
}