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

//! Types and traits related to reporting changing collections out of `dataflow`.

use std::borrow::Cow;
use std::fmt::Debug;
use std::time::Duration;

use mz_dyncfg::ConfigSet;
use mz_expr::MirScalarExpr;
use mz_persist_types::ShardId;
use mz_pgcopy::CopyFormatParams;
use mz_proto::{IntoRustIfSome, ProtoType, RustType, TryFromProtoError};
use mz_repr::bytes::ByteSize;
use mz_repr::{CatalogItemId, GlobalId, RelationDesc};
use proptest::prelude::{any, Arbitrary, BoxedStrategy, Strategy};
use proptest_derive::Arbitrary;
use serde::{Deserialize, Serialize};
use timely::progress::frontier::Antichain;
use timely::PartialOrder;

use crate::connections::inline::{
    ConnectionAccess, ConnectionResolver, InlinedConnection, IntoInlineConnection,
    ReferencedConnection,
};
use crate::connections::{ConnectionContext, KafkaConnection, KafkaTopicOptions};
use crate::controller::{AlterError, CollectionMetadata};
use crate::AlterCompatible;

include!(concat!(env!("OUT_DIR"), "/mz_storage_types.sinks.rs"));

/// A sink for updates to a relational collection.
#[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)]
pub struct StorageSinkDesc<S: StorageSinkDescFillState, T = mz_repr::Timestamp> {
    pub from: GlobalId,
    pub from_desc: RelationDesc,
    pub connection: StorageSinkConnection,
    pub partition_strategy: SinkPartitionStrategy,
    pub with_snapshot: bool,
    pub version: u64,
    pub envelope: SinkEnvelope,
    pub as_of: Antichain<T>,
    pub status_id: Option<<S as StorageSinkDescFillState>::StatusId>,
    pub from_storage_metadata: <S as StorageSinkDescFillState>::StorageMetadata,
}

impl<S: Debug + StorageSinkDescFillState + PartialEq, T: Debug + PartialEq + PartialOrder>
    AlterCompatible for StorageSinkDesc<S, T>
{
    /// Determines if `self` is compatible with another `StorageSinkDesc`, in
    /// such a way that it is possible to turn `self` into `other` through a
    /// valid series of transformations.
    ///
    /// Currently, the only "valid transformation" is the passage of time such
    /// that the sink's as ofs may differ. However, this will change once we
    /// support `ALTER CONNECTION` or `ALTER SINK`.
    fn alter_compatible(
        &self,
        id: GlobalId,
        other: &StorageSinkDesc<S, T>,
    ) -> Result<(), AlterError> {
        if self == other {
            return Ok(());
        }
        let StorageSinkDesc {
            from,
            from_desc,
            connection,
            envelope,
            version: _,
            // The as of of the descriptions may differ.
            as_of: _,
            status_id,
            from_storage_metadata,
            partition_strategy,
            with_snapshot,
        } = self;

        let compatibility_checks = [
            (from == &other.from, "from"),
            (from_desc == &other.from_desc, "from_desc"),
            (
                connection.alter_compatible(id, &other.connection).is_ok(),
                "connection",
            ),
            (envelope == &other.envelope, "envelope"),
            (status_id == &other.status_id, "status_id"),
            (with_snapshot == &other.with_snapshot, "with_snapshot"),
            (
                partition_strategy == &other.partition_strategy,
                "partition_strategy",
            ),
            (
                from_storage_metadata == &other.from_storage_metadata,
                "from_storage_metadata",
            ),
        ];

        for (compatible, field) in compatibility_checks {
            if !compatible {
                tracing::warn!(
                    "StorageSinkDesc incompatible at {field}:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );

                return Err(AlterError { id });
            }
        }

        Ok(())
    }
}

pub trait StorageSinkDescFillState {
    type StatusId: Debug + Clone + Serialize + for<'a> Deserialize<'a> + Eq + PartialEq;
    type StorageMetadata: Debug + Clone + Serialize + for<'a> Deserialize<'a> + Eq + PartialEq;
}

#[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)]
pub struct MetadataUnfilled;
impl StorageSinkDescFillState for MetadataUnfilled {
    type StatusId = GlobalId;
    type StorageMetadata = ();
}

#[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)]
pub struct MetadataFilled;
impl StorageSinkDescFillState for MetadataFilled {
    type StatusId = ShardId;
    type StorageMetadata = CollectionMetadata;
}

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

    fn arbitrary_with(_: Self::Parameters) -> Self::Strategy {
        (
            any::<GlobalId>(),
            any::<RelationDesc>(),
            any::<StorageSinkConnection>(),
            any::<SinkEnvelope>(),
            any::<Option<mz_repr::Timestamp>>(),
            any::<Option<ShardId>>(),
            any::<CollectionMetadata>(),
            any::<SinkPartitionStrategy>(),
            any::<bool>(),
            any::<u64>(),
        )
            .prop_map(
                |(
                    from,
                    from_desc,
                    connection,
                    envelope,
                    as_of,
                    status_id,
                    from_storage_metadata,
                    partition_strategy,
                    with_snapshot,
                    version,
                )| {
                    StorageSinkDesc {
                        from,
                        from_desc,
                        connection,
                        envelope,
                        version,
                        as_of: Antichain::from_iter(as_of),
                        status_id,
                        from_storage_metadata,
                        partition_strategy,
                        with_snapshot,
                    }
                },
            )
            .boxed()
    }
}

impl RustType<ProtoStorageSinkDesc> for StorageSinkDesc<MetadataFilled, mz_repr::Timestamp> {
    fn into_proto(&self) -> ProtoStorageSinkDesc {
        ProtoStorageSinkDesc {
            connection: Some(self.connection.into_proto()),
            from: Some(self.from.into_proto()),
            from_desc: Some(self.from_desc.into_proto()),
            envelope: Some(self.envelope.into_proto()),
            as_of: Some(self.as_of.into_proto()),
            status_id: self.status_id.into_proto(),
            from_storage_metadata: Some(self.from_storage_metadata.into_proto()),
            partition_strategy: Some(self.partition_strategy.into_proto()),
            with_snapshot: self.with_snapshot,
            version: self.version,
        }
    }

    fn from_proto(proto: ProtoStorageSinkDesc) -> Result<Self, TryFromProtoError> {
        Ok(StorageSinkDesc {
            from: proto.from.into_rust_if_some("ProtoStorageSinkDesc::from")?,
            from_desc: proto
                .from_desc
                .into_rust_if_some("ProtoStorageSinkDesc::from_desc")?,
            connection: proto
                .connection
                .into_rust_if_some("ProtoStorageSinkDesc::connection")?,
            envelope: proto
                .envelope
                .into_rust_if_some("ProtoStorageSinkDesc::envelope")?,
            as_of: proto
                .as_of
                .into_rust_if_some("ProtoStorageSinkDesc::as_of")?,
            status_id: proto.status_id.into_rust()?,
            from_storage_metadata: proto
                .from_storage_metadata
                .into_rust_if_some("ProtoStorageSinkDesc::from_storage_metadata")?,
            partition_strategy: proto
                .partition_strategy
                .into_rust_if_some("ProtoStorageSinkDesc::partition_strategy")?,
            with_snapshot: proto.with_snapshot,
            version: proto.version,
        })
    }
}

#[derive(Arbitrary, Copy, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)]
pub enum SinkEnvelope {
    Debezium,
    Upsert,
}

impl RustType<ProtoSinkEnvelope> for SinkEnvelope {
    fn into_proto(&self) -> ProtoSinkEnvelope {
        use proto_sink_envelope::Kind;
        ProtoSinkEnvelope {
            kind: Some(match self {
                SinkEnvelope::Debezium => Kind::Debezium(()),
                SinkEnvelope::Upsert => Kind::Upsert(()),
            }),
        }
    }

    fn from_proto(proto: ProtoSinkEnvelope) -> Result<Self, TryFromProtoError> {
        use proto_sink_envelope::Kind;
        let kind = proto
            .kind
            .ok_or_else(|| TryFromProtoError::missing_field("ProtoSinkEnvelope::kind"))?;
        Ok(match kind {
            Kind::Debezium(()) => SinkEnvelope::Debezium,
            Kind::Upsert(()) => SinkEnvelope::Upsert,
        })
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)]
pub enum SinkPartitionStrategy {
    /// A partition strategy based on the raw encoded bytes of each row.
    V0,
    /// A partition strategy that maintains the property row1 = row2 => partition(row1) =
    /// partition(row2) even when the encoding of each row changes (e.g due to a new avro schema id
    /// being recorded in the data).
    V1,
}

impl RustType<ProtoSinkPartitionStrategy> for SinkPartitionStrategy {
    fn into_proto(&self) -> ProtoSinkPartitionStrategy {
        use proto_sink_partition_strategy::Kind;
        ProtoSinkPartitionStrategy {
            kind: Some(match self {
                SinkPartitionStrategy::V0 => Kind::V0(()),
                SinkPartitionStrategy::V1 => Kind::V1(()),
            }),
        }
    }

    fn from_proto(proto: ProtoSinkPartitionStrategy) -> Result<Self, TryFromProtoError> {
        use proto_sink_partition_strategy::Kind;
        let kind = proto
            .kind
            .ok_or_else(|| TryFromProtoError::missing_field("ProtoSinkPartitionStrategy::kind"))?;
        Ok(match kind {
            Kind::V0(()) => SinkPartitionStrategy::V0,
            Kind::V1(()) => SinkPartitionStrategy::V1,
        })
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)]
pub enum StorageSinkConnection<C: ConnectionAccess = InlinedConnection> {
    Kafka(KafkaSinkConnection<C>),
}

impl<C: ConnectionAccess> StorageSinkConnection<C> {
    /// Determines if `self` is compatible with another `StorageSinkConnection`,
    /// in such a way that it is possible to turn `self` into `other` through a
    /// valid series of transformations (e.g. no transformation or `ALTER
    /// CONNECTION`).
    pub fn alter_compatible(
        &self,
        id: GlobalId,
        other: &StorageSinkConnection<C>,
    ) -> Result<(), AlterError> {
        if self == other {
            return Ok(());
        }
        match (self, other) {
            (StorageSinkConnection::Kafka(s), StorageSinkConnection::Kafka(o)) => {
                s.alter_compatible(id, o)?
            }
        }

        Ok(())
    }
}

impl<R: ConnectionResolver> IntoInlineConnection<StorageSinkConnection, R>
    for StorageSinkConnection<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> StorageSinkConnection {
        match self {
            Self::Kafka(conn) => StorageSinkConnection::Kafka(conn.into_inline_connection(r)),
        }
    }
}

impl RustType<ProtoStorageSinkConnection> for StorageSinkConnection {
    fn into_proto(&self) -> ProtoStorageSinkConnection {
        use proto_storage_sink_connection::Kind::*;

        ProtoStorageSinkConnection {
            kind: Some(match self {
                Self::Kafka(conn) => KafkaV2(conn.into_proto()),
            }),
        }
    }
    fn from_proto(proto: ProtoStorageSinkConnection) -> Result<Self, TryFromProtoError> {
        use proto_storage_sink_connection::Kind::*;

        let kind = proto
            .kind
            .ok_or_else(|| TryFromProtoError::missing_field("ProtoStorageSinkConnection::kind"))?;

        Ok(match kind {
            KafkaV2(proto) => Self::Kafka(proto.into_rust()?),
        })
    }
}

impl<C: ConnectionAccess> StorageSinkConnection<C> {
    /// returns an option to not constrain ourselves in the future
    pub fn connection_id(&self) -> Option<CatalogItemId> {
        use StorageSinkConnection::*;
        match self {
            Kafka(KafkaSinkConnection { connection_id, .. }) => Some(*connection_id),
        }
    }

    /// Returns the name of the sink connection.
    pub fn name(&self) -> &'static str {
        use StorageSinkConnection::*;
        match self {
            Kafka(_) => "kafka",
        }
    }
}

impl RustType<proto_kafka_sink_connection_v2::ProtoKeyDescAndIndices>
    for (RelationDesc, Vec<usize>)
{
    fn into_proto(&self) -> proto_kafka_sink_connection_v2::ProtoKeyDescAndIndices {
        proto_kafka_sink_connection_v2::ProtoKeyDescAndIndices {
            desc: Some(self.0.into_proto()),
            indices: self.1.into_proto(),
        }
    }

    fn from_proto(
        proto: proto_kafka_sink_connection_v2::ProtoKeyDescAndIndices,
    ) -> Result<Self, TryFromProtoError> {
        Ok((
            proto
                .desc
                .into_rust_if_some("ProtoKeyDescAndIndices::desc")?,
            proto.indices.into_rust()?,
        ))
    }
}

impl RustType<proto_kafka_sink_connection_v2::ProtoRelationKeyIndicesVec> for Vec<usize> {
    fn into_proto(&self) -> proto_kafka_sink_connection_v2::ProtoRelationKeyIndicesVec {
        proto_kafka_sink_connection_v2::ProtoRelationKeyIndicesVec {
            relation_key_indices: self.into_proto(),
        }
    }

    fn from_proto(
        proto: proto_kafka_sink_connection_v2::ProtoRelationKeyIndicesVec,
    ) -> Result<Self, TryFromProtoError> {
        proto.relation_key_indices.into_rust()
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)]
pub enum KafkaSinkCompressionType {
    None,
    Gzip,
    Snappy,
    Lz4,
    Zstd,
}

impl KafkaSinkCompressionType {
    /// Format the compression type as expected by `compression.type` librdkafka
    /// setting.
    pub fn to_librdkafka_option(&self) -> &'static str {
        match self {
            KafkaSinkCompressionType::None => "none",
            KafkaSinkCompressionType::Gzip => "gzip",
            KafkaSinkCompressionType::Snappy => "snappy",
            KafkaSinkCompressionType::Lz4 => "lz4",
            KafkaSinkCompressionType::Zstd => "zstd",
        }
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)]
pub struct KafkaSinkConnection<C: ConnectionAccess = InlinedConnection> {
    pub connection_id: CatalogItemId,
    pub connection: C::Kafka,
    pub format: KafkaSinkFormat<C>,
    /// A natural key of the sinked relation (view or source).
    pub relation_key_indices: Option<Vec<usize>>,
    /// The user-specified key for the sink.
    pub key_desc_and_indices: Option<(RelationDesc, Vec<usize>)>,
    /// The index of the column containing message headers value, if any.
    pub headers_index: Option<usize>,
    pub value_desc: RelationDesc,
    /// An expression that, if present, computes a hash value that should be
    /// used to determine the partition for each message.
    pub partition_by: Option<MirScalarExpr>,
    pub topic: String,
    /// Options to use when creating the topic if it doesn't already exist.
    pub topic_options: KafkaTopicOptions,
    pub compression_type: KafkaSinkCompressionType,
    pub progress_group_id: KafkaIdStyle,
    pub transactional_id: KafkaIdStyle,
    pub topic_metadata_refresh_interval: Duration,
}

impl KafkaSinkConnection {
    /// Returns the client ID to register with librdkafka with.
    ///
    /// The caller is responsible for providing the sink ID as it is not known
    /// to `KafkaSinkConnection`.
    pub fn client_id(
        &self,
        configs: &ConfigSet,
        connection_context: &ConnectionContext,
        sink_id: GlobalId,
    ) -> String {
        let mut client_id =
            KafkaConnection::id_base(connection_context, self.connection_id, sink_id);
        self.connection.enrich_client_id(configs, &mut client_id);
        client_id
    }

    /// Returns the name of the progress topic to use for the sink.
    pub fn progress_topic(&self, connection_context: &ConnectionContext) -> Cow<str> {
        self.connection
            .progress_topic(connection_context, self.connection_id)
    }

    /// Returns the ID for the consumer group the sink will use to read the
    /// progress topic on resumption.
    ///
    /// The caller is responsible for providing the sink ID as it is not known
    /// to `KafkaSinkConnection`.
    pub fn progress_group_id(
        &self,
        connection_context: &ConnectionContext,
        sink_id: GlobalId,
    ) -> String {
        match self.progress_group_id {
            KafkaIdStyle::Prefix(ref prefix) => format!(
                "{}{}",
                prefix.as_deref().unwrap_or(""),
                KafkaConnection::id_base(connection_context, self.connection_id, sink_id),
            ),
            KafkaIdStyle::Legacy => format!("materialize-bootstrap-sink-{sink_id}"),
        }
    }

    /// Returns the transactional ID to use for the sink.
    ///
    /// The caller is responsible for providing the sink ID as it is not known
    /// to `KafkaSinkConnection`.
    pub fn transactional_id(
        &self,
        connection_context: &ConnectionContext,
        sink_id: GlobalId,
    ) -> String {
        match self.transactional_id {
            KafkaIdStyle::Prefix(ref prefix) => format!(
                "{}{}",
                prefix.as_deref().unwrap_or(""),
                KafkaConnection::id_base(connection_context, self.connection_id, sink_id)
            ),
            KafkaIdStyle::Legacy => format!("mz-producer-{sink_id}-0"),
        }
    }
}

impl<C: ConnectionAccess> KafkaSinkConnection<C> {
    /// Determines if `self` is compatible with another `StorageSinkConnection`,
    /// in such a way that it is possible to turn `self` into `other` through a
    /// valid series of transformations (e.g. no transformation or `ALTER
    /// CONNECTION`).
    pub fn alter_compatible(
        &self,
        id: GlobalId,
        other: &KafkaSinkConnection<C>,
    ) -> Result<(), AlterError> {
        if self == other {
            return Ok(());
        }
        let KafkaSinkConnection {
            connection_id,
            connection,
            format,
            relation_key_indices,
            key_desc_and_indices,
            headers_index,
            value_desc,
            partition_by,
            topic,
            compression_type,
            progress_group_id,
            transactional_id,
            topic_options,
            topic_metadata_refresh_interval,
        } = self;

        let compatibility_checks = [
            (connection_id == &other.connection_id, "connection_id"),
            (
                connection.alter_compatible(id, &other.connection).is_ok(),
                "connection",
            ),
            (format.alter_compatible(id, &other.format).is_ok(), "format"),
            (
                relation_key_indices == &other.relation_key_indices,
                "relation_key_indices",
            ),
            (
                key_desc_and_indices == &other.key_desc_and_indices,
                "key_desc_and_indices",
            ),
            (headers_index == &other.headers_index, "headers_index"),
            (value_desc == &other.value_desc, "value_desc"),
            (partition_by == &other.partition_by, "partition_by"),
            (topic == &other.topic, "topic"),
            (
                compression_type == &other.compression_type,
                "compression_type",
            ),
            (
                progress_group_id == &other.progress_group_id,
                "progress_group_id",
            ),
            (
                transactional_id == &other.transactional_id,
                "transactional_id",
            ),
            (topic_options == &other.topic_options, "topic_config"),
            (
                topic_metadata_refresh_interval == &other.topic_metadata_refresh_interval,
                "topic_metadata_refresh_interval",
            ),
        ];
        for (compatible, field) in compatibility_checks {
            if !compatible {
                tracing::warn!(
                    "KafkaSinkConnection incompatible at {field}:\nself:\n{:#?}\n\nother\n{:#?}",
                    self,
                    other
                );

                return Err(AlterError { id });
            }
        }

        Ok(())
    }
}

impl<R: ConnectionResolver> IntoInlineConnection<KafkaSinkConnection, R>
    for KafkaSinkConnection<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> KafkaSinkConnection {
        let KafkaSinkConnection {
            connection_id,
            connection,
            format,
            relation_key_indices,
            key_desc_and_indices,
            headers_index,
            value_desc,
            partition_by,
            topic,
            compression_type,
            progress_group_id,
            transactional_id,
            topic_options,
            topic_metadata_refresh_interval,
        } = self;
        KafkaSinkConnection {
            connection_id,
            connection: r.resolve_connection(connection).unwrap_kafka(),
            format: format.into_inline_connection(r),
            relation_key_indices,
            key_desc_and_indices,
            headers_index,
            value_desc,
            partition_by,
            topic,
            compression_type,
            progress_group_id,
            transactional_id,
            topic_options,
            topic_metadata_refresh_interval,
        }
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)]
pub enum KafkaIdStyle {
    /// A new-style id that is optionally prefixed.
    Prefix(Option<String>),
    /// A legacy style id.
    Legacy,
}

impl RustType<ProtoKafkaIdStyle> for KafkaIdStyle {
    fn into_proto(&self) -> ProtoKafkaIdStyle {
        use crate::sinks::proto_kafka_id_style::Kind::*;
        use crate::sinks::proto_kafka_id_style::ProtoKafkaIdStylePrefix;

        ProtoKafkaIdStyle {
            kind: Some(match self {
                Self::Prefix(prefix) => Prefix(ProtoKafkaIdStylePrefix {
                    prefix: prefix.into_proto(),
                }),
                Self::Legacy => Legacy(()),
            }),
        }
    }
    fn from_proto(proto: ProtoKafkaIdStyle) -> Result<Self, TryFromProtoError> {
        use crate::sinks::proto_kafka_id_style::Kind::*;

        let kind = proto
            .kind
            .ok_or_else(|| TryFromProtoError::missing_field("ProtoKafkaIdStyle::kind"))?;

        Ok(match kind {
            Prefix(prefix) => Self::Prefix(prefix.prefix.into_rust()?),
            Legacy(()) => Self::Legacy,
        })
    }
}

impl RustType<ProtoKafkaSinkConnectionV2> for KafkaSinkConnection {
    fn into_proto(&self) -> ProtoKafkaSinkConnectionV2 {
        use crate::sinks::proto_kafka_sink_connection_v2::CompressionType;
        ProtoKafkaSinkConnectionV2 {
            connection_id: Some(self.connection_id.into_proto()),
            connection: Some(self.connection.into_proto()),
            format: Some(self.format.into_proto()),
            key_desc_and_indices: self.key_desc_and_indices.into_proto(),
            relation_key_indices: self.relation_key_indices.into_proto(),
            headers_index: self.headers_index.into_proto(),
            value_desc: Some(self.value_desc.into_proto()),
            partition_by: self.partition_by.into_proto(),
            topic: self.topic.clone(),
            compression_type: Some(match self.compression_type {
                KafkaSinkCompressionType::None => CompressionType::None(()),
                KafkaSinkCompressionType::Gzip => CompressionType::Gzip(()),
                KafkaSinkCompressionType::Snappy => CompressionType::Snappy(()),
                KafkaSinkCompressionType::Lz4 => CompressionType::Lz4(()),
                KafkaSinkCompressionType::Zstd => CompressionType::Zstd(()),
            }),
            progress_group_id: Some(self.progress_group_id.into_proto()),
            transactional_id: Some(self.transactional_id.into_proto()),
            topic_options: Some(self.topic_options.into_proto()),
            topic_metadata_refresh_interval: Some(
                self.topic_metadata_refresh_interval.into_proto(),
            ),
        }
    }

    fn from_proto(proto: ProtoKafkaSinkConnectionV2) -> Result<Self, TryFromProtoError> {
        use crate::sinks::proto_kafka_sink_connection_v2::CompressionType;
        Ok(KafkaSinkConnection {
            connection_id: proto
                .connection_id
                .into_rust_if_some("ProtoKafkaSinkConnectionV2::connection_id")?,
            connection: proto
                .connection
                .into_rust_if_some("ProtoKafkaSinkConnectionV2::connection")?,
            format: proto
                .format
                .into_rust_if_some("ProtoKafkaSinkConnectionV2::format")?,
            key_desc_and_indices: proto.key_desc_and_indices.into_rust()?,
            relation_key_indices: proto.relation_key_indices.into_rust()?,
            headers_index: proto.headers_index.into_rust()?,
            value_desc: proto
                .value_desc
                .into_rust_if_some("ProtoKafkaSinkConnectionV2::value_desc")?,
            partition_by: proto.partition_by.into_rust()?,
            topic: proto.topic,
            compression_type: match proto.compression_type {
                Some(CompressionType::None(())) => KafkaSinkCompressionType::None,
                Some(CompressionType::Gzip(())) => KafkaSinkCompressionType::Gzip,
                Some(CompressionType::Snappy(())) => KafkaSinkCompressionType::Snappy,
                Some(CompressionType::Lz4(())) => KafkaSinkCompressionType::Lz4,
                Some(CompressionType::Zstd(())) => KafkaSinkCompressionType::Zstd,
                None => {
                    return Err(TryFromProtoError::missing_field(
                        "ProtoKafkaSinkConnectionV2::compression_type",
                    ))
                }
            },
            progress_group_id: proto
                .progress_group_id
                .into_rust_if_some("ProtoKafkaSinkConnectionV2::progress_group_id")?,
            transactional_id: proto
                .transactional_id
                .into_rust_if_some("ProtoKafkaSinkConnectionV2::transactional_id")?,
            topic_options: match proto.topic_options {
                Some(topic_options) => topic_options.into_rust()?,
                None => Default::default(),
            },
            topic_metadata_refresh_interval: proto
                .topic_metadata_refresh_interval
                .into_rust_if_some("ProtoKafkaSinkConnectionV2::topic_metadata_refresh_interval")?,
        })
    }
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)]
pub struct KafkaSinkFormat<C: ConnectionAccess = InlinedConnection> {
    pub key_format: Option<KafkaSinkFormatType<C>>,
    pub value_format: KafkaSinkFormatType<C>,
}

#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)]
pub enum KafkaSinkFormatType<C: ConnectionAccess = InlinedConnection> {
    Avro {
        schema: String,
        compatibility_level: Option<mz_ccsr::CompatibilityLevel>,
        csr_connection: C::Csr,
    },
    Json,
    Text,
    Bytes,
}

impl<C: ConnectionAccess> KafkaSinkFormatType<C> {
    pub fn get_format_name(&self) -> &str {
        match self {
            Self::Avro { .. } => "avro",
            Self::Json => "json",
            Self::Text => "text",
            Self::Bytes => "bytes",
        }
    }
}

impl<C: ConnectionAccess> KafkaSinkFormat<C> {
    pub fn get_format_name<'a>(&'a self) -> Cow<'a, str> {
        // For legacy reasons, if the key-format is none or the key & value formats are
        // both the same (either avro or json), we return the value format name,
        // otherwise we return a composite name.
        match &self.key_format {
            None => self.value_format.get_format_name().into(),
            Some(key_format) => match (key_format, &self.value_format) {
                (KafkaSinkFormatType::Avro { .. }, KafkaSinkFormatType::Avro { .. }) => {
                    "avro".into()
                }
                (KafkaSinkFormatType::Json, KafkaSinkFormatType::Json) => "json".into(),
                (keyf, valuef) => format!(
                    "key-{}-value-{}",
                    keyf.get_format_name(),
                    valuef.get_format_name()
                )
                .into(),
            },
        }
    }

    fn alter_compatible(&self, id: GlobalId, other: &Self) -> Result<(), AlterError> {
        if self == other {
            return Ok(());
        }

        match (&self.value_format, &other.value_format) {
            (
                KafkaSinkFormatType::Avro {
                    schema,
                    compatibility_level: _,
                    csr_connection,
                },
                KafkaSinkFormatType::Avro {
                    schema: other_schema,
                    compatibility_level: _,
                    csr_connection: other_csr_connection,
                },
            ) => {
                if schema != other_schema
                    || csr_connection
                        .alter_compatible(id, other_csr_connection)
                        .is_err()
                {
                    tracing::warn!(
                        "KafkaSinkFormat::Avro incompatible at value_format:\nself:\n{:#?}\n\nother\n{:#?}",
                        self,
                        other
                    );

                    return Err(AlterError { id });
                }
            }
            (s, o) => {
                if s != o {
                    tracing::warn!(
                        "KafkaSinkFormat incompatible at value_format:\nself:\n{:#?}\n\nother:{:#?}",
                        s,
                        o
                    );
                    return Err(AlterError { id });
                }
            }
        }

        match (&self.key_format, &other.key_format) {
            (
                Some(KafkaSinkFormatType::Avro {
                    schema,
                    compatibility_level: _,
                    csr_connection,
                }),
                Some(KafkaSinkFormatType::Avro {
                    schema: other_schema,
                    compatibility_level: _,
                    csr_connection: other_csr_connection,
                }),
            ) => {
                if schema != other_schema
                    || csr_connection
                        .alter_compatible(id, other_csr_connection)
                        .is_err()
                {
                    tracing::warn!(
                        "KafkaSinkFormat::Avro incompatible at key_format:\nself:\n{:#?}\n\nother\n{:#?}",
                        self,
                        other
                    );

                    return Err(AlterError { id });
                }
            }
            (s, o) => {
                if s != o {
                    tracing::warn!(
                        "KafkaSinkFormat incompatible at key_format\nself:\n{:#?}\n\nother:{:#?}",
                        s,
                        o
                    );
                    return Err(AlterError { id });
                }
            }
        }

        Ok(())
    }
}

impl<R: ConnectionResolver> IntoInlineConnection<KafkaSinkFormat, R>
    for KafkaSinkFormat<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> KafkaSinkFormat {
        KafkaSinkFormat {
            key_format: self.key_format.map(|f| f.into_inline_connection(&r)),
            value_format: self.value_format.into_inline_connection(&r),
        }
    }
}

impl<R: ConnectionResolver> IntoInlineConnection<KafkaSinkFormatType, R>
    for KafkaSinkFormatType<ReferencedConnection>
{
    fn into_inline_connection(self, r: R) -> KafkaSinkFormatType {
        match self {
            KafkaSinkFormatType::Avro {
                schema,
                compatibility_level,
                csr_connection,
            } => KafkaSinkFormatType::Avro {
                schema,
                compatibility_level,
                csr_connection: r.resolve_connection(csr_connection).unwrap_csr(),
            },
            KafkaSinkFormatType::Json => KafkaSinkFormatType::Json,
            KafkaSinkFormatType::Text => KafkaSinkFormatType::Text,
            KafkaSinkFormatType::Bytes => KafkaSinkFormatType::Bytes,
        }
    }
}

impl RustType<ProtoKafkaSinkFormatType> for KafkaSinkFormatType {
    fn into_proto(&self) -> ProtoKafkaSinkFormatType {
        use proto_kafka_sink_format_type::Type;
        ProtoKafkaSinkFormatType {
            r#type: Some(match self {
                Self::Avro {
                    schema,
                    compatibility_level,
                    csr_connection,
                } => Type::Avro(proto_kafka_sink_format_type::ProtoKafkaSinkAvroFormat {
                    schema: schema.clone(),
                    compatibility_level: csr_compat_level_to_proto(compatibility_level),
                    csr_connection: Some(csr_connection.into_proto()),
                }),
                Self::Json => Type::Json(()),
                Self::Text => Type::Text(()),
                Self::Bytes => Type::Bytes(()),
            }),
        }
    }

    fn from_proto(proto: ProtoKafkaSinkFormatType) -> Result<Self, TryFromProtoError> {
        use proto_kafka_sink_format_type::Type;
        let r#type = proto
            .r#type
            .ok_or_else(|| TryFromProtoError::missing_field("ProtoKafkaSinkFormatType::type"))?;

        Ok(match r#type {
            Type::Avro(proto) => Self::Avro {
                schema: proto.schema,
                compatibility_level: csr_compat_level_from_proto(proto.compatibility_level),
                csr_connection: proto
                    .csr_connection
                    .into_rust_if_some("ProtoKafkaSinkFormatType::csr_connection")?,
            },
            Type::Json(()) => Self::Json,
            Type::Text(()) => Self::Text,
            Type::Bytes(()) => Self::Bytes,
        })
    }
}

impl RustType<ProtoKafkaSinkFormat> for KafkaSinkFormat {
    fn into_proto(&self) -> ProtoKafkaSinkFormat {
        ProtoKafkaSinkFormat {
            key_format: self.key_format.as_ref().map(|f| f.into_proto()),
            value_format: Some(self.value_format.into_proto()),
        }
    }

    fn from_proto(proto: ProtoKafkaSinkFormat) -> Result<Self, TryFromProtoError> {
        Ok(KafkaSinkFormat {
            key_format: proto.key_format.into_rust()?,
            value_format: proto
                .value_format
                .into_rust_if_some("ProtoKafkaSinkFormat::value_format")?,
        })
    }
}

fn csr_compat_level_to_proto(compatibility_level: &Option<mz_ccsr::CompatibilityLevel>) -> i32 {
    use proto_kafka_sink_format_type::proto_kafka_sink_avro_format::CompatibilityLevel as ProtoCompatLevel;
    match compatibility_level {
        Some(level) => match level {
            mz_ccsr::CompatibilityLevel::Backward => ProtoCompatLevel::Backward,
            mz_ccsr::CompatibilityLevel::BackwardTransitive => ProtoCompatLevel::BackwardTransitive,
            mz_ccsr::CompatibilityLevel::Forward => ProtoCompatLevel::Forward,
            mz_ccsr::CompatibilityLevel::ForwardTransitive => ProtoCompatLevel::ForwardTransitive,
            mz_ccsr::CompatibilityLevel::Full => ProtoCompatLevel::Full,
            mz_ccsr::CompatibilityLevel::FullTransitive => ProtoCompatLevel::FullTransitive,
            mz_ccsr::CompatibilityLevel::None => ProtoCompatLevel::None,
        },
        None => ProtoCompatLevel::Unset,
    }
    .into()
}

fn csr_compat_level_from_proto(val: i32) -> Option<mz_ccsr::CompatibilityLevel> {
    use proto_kafka_sink_format_type::proto_kafka_sink_avro_format::CompatibilityLevel as ProtoCompatLevel;
    match ProtoCompatLevel::try_from(val) {
        Ok(ProtoCompatLevel::Backward) => Some(mz_ccsr::CompatibilityLevel::Backward),
        Ok(ProtoCompatLevel::BackwardTransitive) => {
            Some(mz_ccsr::CompatibilityLevel::BackwardTransitive)
        }
        Ok(ProtoCompatLevel::Forward) => Some(mz_ccsr::CompatibilityLevel::Forward),
        Ok(ProtoCompatLevel::ForwardTransitive) => {
            Some(mz_ccsr::CompatibilityLevel::ForwardTransitive)
        }
        Ok(ProtoCompatLevel::Full) => Some(mz_ccsr::CompatibilityLevel::Full),
        Ok(ProtoCompatLevel::FullTransitive) => Some(mz_ccsr::CompatibilityLevel::FullTransitive),
        Ok(ProtoCompatLevel::None) => Some(mz_ccsr::CompatibilityLevel::None),
        Ok(ProtoCompatLevel::Unset) => None,
        Err(_) => None,
    }
}

#[derive(Arbitrary, Clone, Debug, Serialize, Deserialize, Eq, PartialEq)]
pub enum S3SinkFormat {
    /// Encoded using the PG `COPY` protocol, with one of its supported formats.
    PgCopy(CopyFormatParams<'static>),
    /// Encoded as Parquet.
    Parquet,
}

impl RustType<ProtoS3SinkFormat> for S3SinkFormat {
    fn into_proto(&self) -> ProtoS3SinkFormat {
        use proto_s3_sink_format::Kind;
        ProtoS3SinkFormat {
            kind: Some(match self {
                Self::PgCopy(params) => Kind::PgCopy(params.into_proto()),
                Self::Parquet => Kind::Parquet(()),
            }),
        }
    }

    fn from_proto(proto: ProtoS3SinkFormat) -> Result<Self, TryFromProtoError> {
        use proto_s3_sink_format::Kind;
        let kind = proto
            .kind
            .ok_or_else(|| TryFromProtoError::missing_field("ProtoS3SinkFormat::kind"))?;

        Ok(match kind {
            Kind::PgCopy(proto) => Self::PgCopy(proto.into_rust()?),
            Kind::Parquet(_) => Self::Parquet,
        })
    }
}

/// Info required to copy the data to s3.
#[derive(Arbitrary, Clone, Debug, Serialize, Deserialize, Eq, PartialEq)]
pub struct S3UploadInfo {
    /// The s3 uri path to write the data to.
    pub uri: String,
    /// The max file size of each file uploaded to S3.
    pub max_file_size: u64,
    /// The relation desc of the data to be uploaded to S3.
    pub desc: RelationDesc,
    /// The selected sink format.
    pub format: S3SinkFormat,
}

impl RustType<ProtoS3UploadInfo> for S3UploadInfo {
    fn into_proto(&self) -> ProtoS3UploadInfo {
        ProtoS3UploadInfo {
            uri: self.uri.clone(),
            max_file_size: self.max_file_size,
            desc: Some(self.desc.into_proto()),
            format: Some(self.format.into_proto()),
        }
    }

    fn from_proto(proto: ProtoS3UploadInfo) -> Result<Self, TryFromProtoError> {
        Ok(S3UploadInfo {
            uri: proto.uri,
            max_file_size: proto.max_file_size,
            desc: proto.desc.into_rust_if_some("ProtoS3UploadInfo::desc")?,
            format: proto
                .format
                .into_rust_if_some("ProtoS3UploadInfo::format")?,
        })
    }
}

pub const MIN_S3_SINK_FILE_SIZE: ByteSize = ByteSize::mb(16);
pub const MAX_S3_SINK_FILE_SIZE: ByteSize = ByteSize::gb(4);