krafka 0.9.0

A pure Rust, async-native Apache Kafka client
Documentation
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
use bytes::{Buf, BufMut, Bytes};

use super::{VersionedDecode, VersionedEncode};
use crate::error::{ErrorCode, KrafkaError, ProtocolErrorKind, Result};
use crate::protocol::primitives::{
    Decode, Encode, KafkaBytes, KafkaString, TaggedFields, TryEncode,
};
use crate::protocol::{check_compact_array_len, encode_compact_array_len};

// ============================================================================
// ShareGroupHeartbeat (API Key 76) — KIP-932
// ============================================================================

/// Request for share group heartbeat (KIP-932).
#[derive(Debug, Clone)]
pub struct ShareGroupHeartbeatRequest {
    /// Group identifier.
    pub group_id: String,
    /// Member ID (generated by the consumer).
    pub member_id: String,
    /// Current member epoch: 0 to join, -1 to leave.
    pub member_epoch: i32,
    /// Rack ID of the consumer, or None if unchanged.
    pub rack_id: Option<String>,
    /// Subscribed topic names, or None if unchanged.
    pub subscribed_topic_names: Option<Vec<String>>,
}

impl ShareGroupHeartbeatRequest {
    /// Encode for version 1 (stable, KIP-932).
    pub fn encode_v1(&self, buf: &mut impl BufMut) -> Result<()> {
        KafkaString::new(&self.group_id).try_encode_compact(buf)?;
        KafkaString::new(&self.member_id).try_encode_compact(buf)?;
        self.member_epoch.encode(buf);
        // RackId: nullable compact string
        KafkaString(self.rack_id.clone()).try_encode_compact(buf)?;
        // SubscribedTopicNames: nullable compact array of compact strings
        if let Some(ref topics) = self.subscribed_topic_names {
            let len = u32::try_from(topics.len().saturating_add(1)).map_err(|_| {
                KrafkaError::protocol_kind(
                    ProtocolErrorKind::InvalidLength,
                    "subscribed topics array too large",
                )
            })?;
            crate::util::varint::encode_unsigned_varint(len, buf);
            for topic in topics {
                KafkaString::new(topic).try_encode_compact(buf)?;
            }
        } else {
            // null array: varint 0
            crate::util::varint::encode_unsigned_varint(0, buf);
        }
        TaggedFields::default().try_encode(buf)?;
        Ok(())
    }
}

/// Response from share group heartbeat (KIP-932).
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareGroupHeartbeatResponse {
    /// Throttle time in milliseconds.
    pub throttle_time_ms: i32,
    /// Error code.
    pub error_code: ErrorCode,
    /// Error message, or None.
    pub error_message: Option<String>,
    /// Member ID.
    pub member_id: Option<String>,
    /// Member epoch.
    pub member_epoch: i32,
    /// Heartbeat interval in milliseconds.
    pub heartbeat_interval_ms: i32,
    /// Assignment (topic partitions assigned to this member), or None.
    pub assignment: Option<Vec<ShareGroupTopicPartitions>>,
}

/// Topic-partition assignment in share group responses.
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareGroupTopicPartitions {
    /// Topic ID.
    pub topic_id: [u8; 16],
    /// Partition indices.
    pub partitions: Vec<i32>,
}

impl ShareGroupHeartbeatResponse {
    /// Decode from version 1.
    pub fn decode_v1(buf: &mut impl Buf) -> Result<Self> {
        let throttle_time_ms = i32::decode(buf)?;
        let error_code = ErrorCode::from_i16(i16::decode(buf)?);
        let error_message = KafkaString::decode_compact(buf)?.0;
        let member_id = KafkaString::decode_compact(buf)?.0;
        let member_epoch = i32::decode(buf)?;
        let heartbeat_interval_ms = i32::decode(buf)?;

        // Assignment — nullable struct.
        // Non-tagged nullable structs in flexible versions use a single byte
        // as presence marker: `(byte) -1` (`0xff`) = null, `(byte) 1` = present.
        let assignment = Self::decode_assignment(buf)?;

        let _ = TaggedFields::decode(buf)?; // top-level tagged fields
        Ok(Self {
            throttle_time_ms,
            error_code,
            error_message,
            member_id,
            member_epoch,
            heartbeat_interval_ms,
            assignment,
        })
    }

    /// Decode the nullable Assignment struct.
    fn decode_assignment(buf: &mut impl Buf) -> Result<Option<Vec<ShareGroupTopicPartitions>>> {
        if buf.remaining() < 1 {
            return Err(KrafkaError::protocol_kind(
                ProtocolErrorKind::TruncatedFrame,
                "not enough bytes for assignment presence tag",
            ));
        }
        let presence = buf.get_i8();
        if presence < 0 {
            return Ok(None);
        }
        if presence != 1 {
            return Err(KrafkaError::protocol_kind(
                ProtocolErrorKind::Malformed,
                format!(
                    "invalid assignment presence tag: expected negative for null or 1 for present, got {presence}"
                ),
            ));
        }

        // Struct is present — decode TopicPartitions compact array.
        let tp_count = check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
        let mut topic_partitions = Vec::with_capacity(tp_count);
        for _ in 0..tp_count {
            let mut topic_id = [0u8; 16];
            if buf.remaining() < 16 {
                return Err(KrafkaError::protocol_kind(
                    ProtocolErrorKind::TruncatedFrame,
                    "not enough bytes for topic_id UUID",
                ));
            }
            buf.copy_to_slice(&mut topic_id);
            let p_count =
                check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
            let mut partitions = Vec::with_capacity(p_count);
            for _ in 0..p_count {
                partitions.push(i32::decode(buf)?);
            }
            let _ = TaggedFields::decode(buf)?;
            topic_partitions.push(ShareGroupTopicPartitions {
                topic_id,
                partitions,
            });
        }
        let _ = TaggedFields::decode(buf)?; // assignment tagged fields
        Ok(Some(topic_partitions))
    }
}

// ============================================================================
// ShareGroupDescribe (API Key 77) — KIP-932
// ============================================================================

/// Request to describe share groups (KIP-932).
#[derive(Debug, Clone)]
pub struct ShareGroupDescribeRequest {
    /// Group IDs to describe.
    pub group_ids: Vec<String>,
    /// Whether to include authorized operations.
    pub include_authorized_operations: bool,
}

impl ShareGroupDescribeRequest {
    /// Encode for version 1 (stable, KIP-932).
    pub fn encode_v1(&self, buf: &mut impl BufMut) -> Result<()> {
        let len = u32::try_from(self.group_ids.len().saturating_add(1)).map_err(|_| {
            KrafkaError::protocol_kind(
                ProtocolErrorKind::InvalidLength,
                "group_ids array too large",
            )
        })?;
        crate::util::varint::encode_unsigned_varint(len, buf);
        for gid in &self.group_ids {
            KafkaString::new(gid).try_encode_compact(buf)?;
        }
        buf.put_u8(u8::from(self.include_authorized_operations));
        TaggedFields::default().try_encode(buf)?;
        Ok(())
    }
}

/// Response from share group describe (KIP-932).
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareGroupDescribeResponse {
    /// Throttle time in milliseconds.
    pub throttle_time_ms: i32,
    /// Described groups.
    pub groups: Vec<ShareGroupDescription>,
}

/// A single described share group (KIP-932).
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareGroupDescription {
    /// Error code for this group.
    pub error_code: ErrorCode,
    /// Error message, or None.
    pub error_message: Option<String>,
    /// Group ID.
    pub group_id: String,
    /// Current group state.
    pub group_state: String,
    /// Group epoch.
    pub group_epoch: i32,
    /// Assignment epoch.
    pub assignment_epoch: i32,
    /// Selected assignor.
    pub assignor_name: String,
    /// Members of the group.
    pub members: Vec<ShareGroupMember>,
    /// Authorized operations bitfield.
    pub authorized_operations: i32,
}

/// A member of a share group (KIP-932).
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareGroupMember {
    /// Member ID.
    pub member_id: String,
    /// Rack ID, or None.
    pub rack_id: Option<String>,
    /// Current member epoch.
    pub member_epoch: i32,
    /// Client ID.
    pub client_id: String,
    /// Client host.
    pub client_host: String,
    /// Subscribed topic names.
    pub subscribed_topic_names: Vec<String>,
    /// Current assignment (topic partitions).
    pub assignment: Vec<ShareGroupDescribeTopicPartitions>,
}

/// Topic partition info in a share group describe response, with topic name (KIP-932).
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareGroupDescribeTopicPartitions {
    /// Topic ID.
    pub topic_id: [u8; 16],
    /// Topic name.
    pub topic_name: String,
    /// Partition indices.
    pub partitions: Vec<i32>,
}

impl ShareGroupDescribeResponse {
    /// Decode from version 1.
    pub fn decode_v1(buf: &mut impl Buf) -> Result<Self> {
        let throttle_time_ms = i32::decode(buf)?;
        let group_count =
            check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
        let mut groups = Vec::with_capacity(group_count);
        for _ in 0..group_count {
            let error_code = ErrorCode::from_i16(i16::decode(buf)?);
            let error_message = KafkaString::decode_compact(buf)?.0;
            let group_id =
                super::non_nullable_string("group_id", KafkaString::decode_compact(buf)?.0)?;
            let group_state =
                super::non_nullable_string("group_state", KafkaString::decode_compact(buf)?.0)?;
            let group_epoch = i32::decode(buf)?;
            let assignment_epoch = i32::decode(buf)?;
            let assignor_name =
                super::non_nullable_string("assignor_name", KafkaString::decode_compact(buf)?.0)?;
            // Members
            let member_count =
                check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
            let mut members = Vec::with_capacity(member_count);
            for _ in 0..member_count {
                let member_id =
                    super::non_nullable_string("member_id", KafkaString::decode_compact(buf)?.0)?;
                let rack_id = KafkaString::decode_compact(buf)?.0;
                let member_epoch = i32::decode(buf)?;
                let client_id =
                    super::non_nullable_string("client_id", KafkaString::decode_compact(buf)?.0)?;
                let client_host =
                    super::non_nullable_string("client_host", KafkaString::decode_compact(buf)?.0)?;
                // SubscribedTopicNames
                let stn_count =
                    check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
                let mut subscribed_topic_names = Vec::with_capacity(stn_count);
                for _ in 0..stn_count {
                    subscribed_topic_names.push(super::non_nullable_string(
                        "subscribed_topic_name",
                        KafkaString::decode_compact(buf)?.0,
                    )?);
                }
                // Assignment struct (non-nullable)
                let tp_count =
                    check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
                let mut assignment = Vec::with_capacity(tp_count);
                for _ in 0..tp_count {
                    let mut topic_id = [0u8; 16];
                    if buf.remaining() < 16 {
                        return Err(KrafkaError::protocol_kind(
                            ProtocolErrorKind::TruncatedFrame,
                            "not enough bytes for topic_id",
                        ));
                    }
                    buf.copy_to_slice(&mut topic_id);
                    let topic_name = super::non_nullable_string(
                        "topic_name",
                        KafkaString::decode_compact(buf)?.0,
                    )?;
                    let p_count =
                        check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
                    let mut partitions = Vec::with_capacity(p_count);
                    for _ in 0..p_count {
                        partitions.push(i32::decode(buf)?);
                    }
                    let _ = TaggedFields::decode(buf)?;
                    assignment.push(ShareGroupDescribeTopicPartitions {
                        topic_id,
                        topic_name,
                        partitions,
                    });
                }
                let _ = TaggedFields::decode(buf)?; // assignment tagged fields
                let _ = TaggedFields::decode(buf)?; // member tagged fields
                members.push(ShareGroupMember {
                    member_id,
                    rack_id,
                    member_epoch,
                    client_id,
                    client_host,
                    subscribed_topic_names,
                    assignment,
                });
            }
            let authorized_operations = i32::decode(buf)?;
            let _ = TaggedFields::decode(buf)?; // group tagged fields
            groups.push(ShareGroupDescription {
                error_code,
                error_message,
                group_id,
                group_state,
                group_epoch,
                assignment_epoch,
                assignor_name,
                members,
                authorized_operations,
            });
        }
        let _ = TaggedFields::decode(buf)?; // top-level tagged fields
        Ok(Self {
            throttle_time_ms,
            groups,
        })
    }
}

// ============================================================================
// ShareFetch (API Key 78) — KIP-932
// ============================================================================

/// Acknowledgement batch for share fetch/acknowledge requests (KIP-932).
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareAcknowledgementBatch {
    /// First offset of the batch.
    pub first_offset: i64,
    /// Last offset (inclusive) of the batch.
    pub last_offset: i64,
    /// Acknowledge types per record: 0=Gap, 1=Accept, 2=Release, 3=Reject, 4=Renew.
    pub acknowledge_types: Vec<i8>,
}

/// Request to fetch from share groups (KIP-932).
#[derive(Debug, Clone)]
pub struct ShareFetchRequest {
    /// Group identifier.
    pub group_id: Option<String>,
    /// Member ID.
    pub member_id: Option<String>,
    /// Share session epoch: 0 to open, -1 to close.
    pub share_session_epoch: i32,
    /// Max wait time in milliseconds.
    pub max_wait_ms: i32,
    /// Min bytes to accumulate.
    pub min_bytes: i32,
    /// Max bytes to fetch.
    pub max_bytes: i32,
    /// Max number of records to fetch (v1+).
    pub max_records: i32,
    /// Optimal batch size for acquired records (v1+).
    pub batch_size: i32,
    /// Topics to fetch.
    pub topics: Vec<ShareFetchTopic>,
    /// Topics/partitions to remove from the session.
    pub forgotten_topics: Vec<ShareFetchForgottenTopic>,
}

/// Topic within a share fetch request.
#[derive(Debug, Clone)]
pub struct ShareFetchTopic {
    /// Topic ID.
    pub topic_id: [u8; 16],
    /// Partitions to fetch.
    pub partitions: Vec<ShareFetchPartition>,
}

/// Partition within a share fetch topic.
#[derive(Debug, Clone)]
pub struct ShareFetchPartition {
    /// Partition index.
    pub partition_index: i32,
    /// Acknowledgement batches.
    pub acknowledgement_batches: Vec<ShareAcknowledgementBatch>,
}

/// Forgotten topic in a share fetch request.
#[derive(Debug, Clone)]
pub struct ShareFetchForgottenTopic {
    /// Topic ID.
    pub topic_id: [u8; 16],
    /// Partition indices to forget.
    pub partitions: Vec<i32>,
}

impl ShareFetchRequest {
    /// Encode topics and forgotten topics arrays (shared by v1 and v2).
    fn encode_topics_and_forgotten(&self, buf: &mut impl BufMut) -> Result<()> {
        // Topics compact array
        encode_compact_array_len(self.topics.len(), buf)?;
        for topic in &self.topics {
            buf.put_slice(&topic.topic_id);
            encode_compact_array_len(topic.partitions.len(), buf)?;
            for partition in &topic.partitions {
                partition.partition_index.encode(buf);
                // AcknowledgementBatches compact array
                encode_compact_array_len(partition.acknowledgement_batches.len(), buf)?;
                for batch in &partition.acknowledgement_batches {
                    batch.first_offset.encode(buf);
                    batch.last_offset.encode(buf);
                    encode_compact_array_len(batch.acknowledge_types.len(), buf)?;
                    for &t in &batch.acknowledge_types {
                        t.encode(buf);
                    }
                    TaggedFields::default().try_encode(buf)?;
                }
                TaggedFields::default().try_encode(buf)?;
            }
            TaggedFields::default().try_encode(buf)?;
        }
        // ForgottenTopicsData compact array
        encode_compact_array_len(self.forgotten_topics.len(), buf)?;
        for forgotten in &self.forgotten_topics {
            buf.put_slice(&forgotten.topic_id);
            encode_compact_array_len(forgotten.partitions.len(), buf)?;
            for &p in &forgotten.partitions {
                p.encode(buf);
            }
            TaggedFields::default().try_encode(buf)?;
        }
        TaggedFields::default().try_encode(buf)?;
        Ok(())
    }

    /// Encode for version 1 (stable, KIP-932).
    pub fn encode_v1(&self, buf: &mut impl BufMut) -> Result<()> {
        KafkaString(self.group_id.clone()).try_encode_compact(buf)?;
        KafkaString(self.member_id.clone()).try_encode_compact(buf)?;
        self.share_session_epoch.encode(buf);
        self.max_wait_ms.encode(buf);
        self.min_bytes.encode(buf);
        self.max_bytes.encode(buf);
        self.max_records.encode(buf);
        self.batch_size.encode(buf);
        self.encode_topics_and_forgotten(buf)
    }

    /// Encode for version 2 (KIP-1206 + KIP-1222: ShareAcquireMode + IsRenewAck).
    pub fn encode_v2(
        &self,
        buf: &mut impl BufMut,
        share_acquire_mode: i8,
        is_renew_ack: bool,
    ) -> Result<()> {
        KafkaString(self.group_id.clone()).try_encode_compact(buf)?;
        KafkaString(self.member_id.clone()).try_encode_compact(buf)?;
        self.share_session_epoch.encode(buf);
        self.max_wait_ms.encode(buf);
        self.min_bytes.encode(buf);
        self.max_bytes.encode(buf);
        self.max_records.encode(buf);
        self.batch_size.encode(buf);
        share_acquire_mode.encode(buf);
        buf.put_u8(u8::from(is_renew_ack));
        self.encode_topics_and_forgotten(buf)
    }
}

/// Leader info in share responses.
#[derive(Debug, Clone, Default)]
pub struct ShareLeaderIdAndEpoch {
    /// Leader broker ID, or -1.
    pub leader_id: i32,
    /// Leader epoch.
    pub leader_epoch: i32,
}

/// Acquired records batch in share fetch response.
#[derive(Debug, Clone)]
pub struct ShareAcquiredRecords {
    /// First offset.
    pub first_offset: i64,
    /// Last offset.
    pub last_offset: i64,
    /// Delivery count.
    pub delivery_count: i16,
}

/// Node endpoint in share responses.
#[derive(Debug, Clone)]
pub struct ShareNodeEndpoint {
    /// Node ID.
    pub node_id: i32,
    /// Hostname.
    pub host: String,
    /// Port.
    pub port: i32,
    /// Rack, or None.
    pub rack: Option<String>,
}

/// Response from share fetch (KIP-932).
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareFetchResponse {
    /// Throttle time in milliseconds.
    pub throttle_time_ms: i32,
    /// Error code.
    pub error_code: ErrorCode,
    /// Error message, or None.
    pub error_message: Option<String>,
    /// Acquisition lock timeout in milliseconds (v1+).
    pub acquisition_lock_timeout_ms: i32,
    /// Topic responses.
    pub responses: Vec<ShareFetchTopicResponse>,
    /// Node endpoints for current leaders.
    pub node_endpoints: Vec<ShareNodeEndpoint>,
}

/// Topic in share fetch response.
#[derive(Debug, Clone)]
pub struct ShareFetchTopicResponse {
    /// Topic ID.
    pub topic_id: [u8; 16],
    /// Partition responses.
    pub partitions: Vec<ShareFetchPartitionResponse>,
}

/// Partition in share fetch response.
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareFetchPartitionResponse {
    /// Partition index.
    pub partition_index: i32,
    /// Error code.
    pub error_code: ErrorCode,
    /// Error message, or None.
    pub error_message: Option<String>,
    /// Acknowledge error code.
    pub acknowledge_error_code: ErrorCode,
    /// Acknowledge error message, or None.
    pub acknowledge_error_message: Option<String>,
    /// Current leader info.
    pub current_leader: ShareLeaderIdAndEpoch,
    /// Record data, or None.
    pub records: Option<Bytes>,
    /// Acquired records batches.
    pub acquired_records: Vec<ShareAcquiredRecords>,
}

impl ShareFetchResponse {
    /// Decode from version 1.
    pub fn decode_v1(buf: &mut impl Buf) -> Result<Self> {
        let throttle_time_ms = i32::decode(buf)?;
        let error_code = ErrorCode::from_i16(i16::decode(buf)?);
        let error_message = KafkaString::decode_compact(buf)?.0;
        let acquisition_lock_timeout_ms = i32::decode(buf)?;
        // Responses compact array
        let topic_count =
            check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
        let mut responses = Vec::with_capacity(topic_count);
        for _ in 0..topic_count {
            let mut topic_id = [0u8; 16];
            if buf.remaining() < 16 {
                return Err(KrafkaError::protocol_kind(
                    ProtocolErrorKind::TruncatedFrame,
                    "not enough bytes for topic_id UUID",
                ));
            }
            buf.copy_to_slice(&mut topic_id);
            let part_count =
                check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
            let mut partitions = Vec::with_capacity(part_count);
            for _ in 0..part_count {
                let partition_index = i32::decode(buf)?;
                let part_error = ErrorCode::from_i16(i16::decode(buf)?);
                let part_error_message = KafkaString::decode_compact(buf)?.0;
                let ack_error = ErrorCode::from_i16(i16::decode(buf)?);
                let ack_error_message = KafkaString::decode_compact(buf)?.0;
                let leader_id = i32::decode(buf)?;
                let leader_epoch = i32::decode(buf)?;
                let _ = TaggedFields::decode(buf)?; // CurrentLeader tagged
                let records = KafkaBytes::decode_compact(buf)?.0;
                // AcquiredRecords compact array
                let ar_count =
                    check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
                let mut acquired_records = Vec::with_capacity(ar_count);
                for _ in 0..ar_count {
                    acquired_records.push(ShareAcquiredRecords {
                        first_offset: i64::decode(buf)?,
                        last_offset: i64::decode(buf)?,
                        delivery_count: i16::decode(buf)?,
                    });
                    let _ = TaggedFields::decode(buf)?;
                }
                let _ = TaggedFields::decode(buf)?; // partition tagged
                partitions.push(ShareFetchPartitionResponse {
                    partition_index,
                    error_code: part_error,
                    error_message: part_error_message,
                    acknowledge_error_code: ack_error,
                    acknowledge_error_message: ack_error_message,
                    current_leader: ShareLeaderIdAndEpoch {
                        leader_id,
                        leader_epoch,
                    },
                    records,
                    acquired_records,
                });
            }
            let _ = TaggedFields::decode(buf)?; // topic tagged
            responses.push(ShareFetchTopicResponse {
                topic_id,
                partitions,
            });
        }
        // NodeEndpoints compact array
        let ne_count = check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
        let mut node_endpoints = Vec::with_capacity(ne_count);
        for _ in 0..ne_count {
            let node_id = i32::decode(buf)?;
            let host = super::non_nullable_string("host", KafkaString::decode_compact(buf)?.0)?;
            let port = i32::decode(buf)?;
            let rack = KafkaString::decode_compact(buf)?.0;
            let _ = TaggedFields::decode(buf)?;
            node_endpoints.push(ShareNodeEndpoint {
                node_id,
                host,
                port,
                rack,
            });
        }
        let _ = TaggedFields::decode(buf)?; // top-level tagged
        Ok(Self {
            throttle_time_ms,
            error_code,
            error_message,
            acquisition_lock_timeout_ms,
            responses,
            node_endpoints,
        })
    }

    /// Decode from version 2 (KIP-1206 + KIP-1222). Same response wire format as v1.
    pub fn decode_v2(buf: &mut impl Buf) -> Result<Self> {
        Self::decode_v1(buf)
    }
}

// ============================================================================
// ShareAcknowledge (API Key 79) — KIP-932
// ============================================================================

/// Request to acknowledge share group records (KIP-932).
#[derive(Debug, Clone)]
pub struct ShareAcknowledgeRequest {
    /// Group identifier.
    pub group_id: Option<String>,
    /// Member ID.
    pub member_id: Option<String>,
    /// Share session epoch: 0 to open, -1 to close.
    pub share_session_epoch: i32,
    /// Topics with partitions to acknowledge.
    pub topics: Vec<ShareAcknowledgeTopic>,
}

/// Topic within a share acknowledge request.
#[derive(Debug, Clone)]
pub struct ShareAcknowledgeTopic {
    /// Topic ID.
    pub topic_id: [u8; 16],
    /// Partitions to acknowledge.
    pub partitions: Vec<ShareAcknowledgePartition>,
}

/// Partition within a share acknowledge topic.
#[derive(Debug, Clone)]
pub struct ShareAcknowledgePartition {
    /// Partition index.
    pub partition_index: i32,
    /// Acknowledgement batches.
    pub acknowledgement_batches: Vec<ShareAcknowledgementBatch>,
}

impl ShareAcknowledgeRequest {
    /// Encode topics array (shared by v1 and v2).
    fn encode_topics(&self, buf: &mut impl BufMut) -> Result<()> {
        encode_compact_array_len(self.topics.len(), buf)?;
        for topic in &self.topics {
            buf.put_slice(&topic.topic_id);
            encode_compact_array_len(topic.partitions.len(), buf)?;
            for partition in &topic.partitions {
                partition.partition_index.encode(buf);
                encode_compact_array_len(partition.acknowledgement_batches.len(), buf)?;
                for batch in &partition.acknowledgement_batches {
                    batch.first_offset.encode(buf);
                    batch.last_offset.encode(buf);
                    encode_compact_array_len(batch.acknowledge_types.len(), buf)?;
                    for &t in &batch.acknowledge_types {
                        t.encode(buf);
                    }
                    TaggedFields::default().try_encode(buf)?;
                }
                TaggedFields::default().try_encode(buf)?;
            }
            TaggedFields::default().try_encode(buf)?;
        }
        TaggedFields::default().try_encode(buf)?;
        Ok(())
    }

    /// Encode for version 1 (stable, KIP-932).
    pub fn encode_v1(&self, buf: &mut impl BufMut) -> Result<()> {
        KafkaString(self.group_id.clone()).try_encode_compact(buf)?;
        KafkaString(self.member_id.clone()).try_encode_compact(buf)?;
        self.share_session_epoch.encode(buf);
        self.encode_topics(buf)
    }

    /// Encode for version 2 (KIP-1222: adds IsRenewAck field).
    pub fn encode_v2(&self, buf: &mut impl BufMut, is_renew_ack: bool) -> Result<()> {
        KafkaString(self.group_id.clone()).try_encode_compact(buf)?;
        KafkaString(self.member_id.clone()).try_encode_compact(buf)?;
        self.share_session_epoch.encode(buf);
        buf.put_u8(u8::from(is_renew_ack));
        self.encode_topics(buf)
    }
}

/// Response from share acknowledge (KIP-932).
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareAcknowledgeResponse {
    /// Throttle time in milliseconds.
    pub throttle_time_ms: i32,
    /// Error code.
    pub error_code: ErrorCode,
    /// Error message, or None.
    pub error_message: Option<String>,
    /// Acquisition lock timeout in milliseconds (v2+ only, KIP-1222). -1 if absent.
    pub acquisition_lock_timeout_ms: i32,
    /// Topic responses.
    pub responses: Vec<ShareAcknowledgeTopicResponse>,
    /// Node endpoints.
    pub node_endpoints: Vec<ShareNodeEndpoint>,
}

/// Topic in share acknowledge response.
#[derive(Debug, Clone)]
pub struct ShareAcknowledgeTopicResponse {
    /// Topic ID.
    pub topic_id: [u8; 16],
    /// Partition responses.
    pub partitions: Vec<ShareAcknowledgePartitionResponse>,
}

/// Partition in share acknowledge response.
#[derive(Debug, Clone)]
#[non_exhaustive]
pub struct ShareAcknowledgePartitionResponse {
    /// Partition index.
    pub partition_index: i32,
    /// Error code.
    pub error_code: ErrorCode,
    /// Error message, or None.
    pub error_message: Option<String>,
    /// Current leader info.
    pub current_leader: ShareLeaderIdAndEpoch,
}

impl ShareAcknowledgeResponse {
    /// Decode responses and node endpoints (shared by v1 and v2).
    fn decode_responses_and_endpoints(
        buf: &mut impl Buf,
    ) -> Result<(Vec<ShareAcknowledgeTopicResponse>, Vec<ShareNodeEndpoint>)> {
        // Responses compact array
        let topic_count =
            check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
        let mut responses = Vec::with_capacity(topic_count);
        for _ in 0..topic_count {
            let mut topic_id = [0u8; 16];
            if buf.remaining() < 16 {
                return Err(KrafkaError::protocol_kind(
                    ProtocolErrorKind::TruncatedFrame,
                    "not enough bytes for topic_id UUID",
                ));
            }
            buf.copy_to_slice(&mut topic_id);
            let part_count =
                check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
            let mut partitions = Vec::with_capacity(part_count);
            for _ in 0..part_count {
                let partition_index = i32::decode(buf)?;
                let part_error = ErrorCode::from_i16(i16::decode(buf)?);
                let part_error_message = KafkaString::decode_compact(buf)?.0;
                let leader_id = i32::decode(buf)?;
                let leader_epoch = i32::decode(buf)?;
                let _ = TaggedFields::decode(buf)?; // CurrentLeader tagged
                let _ = TaggedFields::decode(buf)?; // partition tagged
                partitions.push(ShareAcknowledgePartitionResponse {
                    partition_index,
                    error_code: part_error,
                    error_message: part_error_message,
                    current_leader: ShareLeaderIdAndEpoch {
                        leader_id,
                        leader_epoch,
                    },
                });
            }
            let _ = TaggedFields::decode(buf)?; // topic tagged
            responses.push(ShareAcknowledgeTopicResponse {
                topic_id,
                partitions,
            });
        }
        // NodeEndpoints compact array
        let ne_count = check_compact_array_len(crate::util::varint::decode_unsigned_varint(buf)?)?;
        let mut node_endpoints = Vec::with_capacity(ne_count);
        for _ in 0..ne_count {
            let node_id = i32::decode(buf)?;
            let host = super::non_nullable_string("host", KafkaString::decode_compact(buf)?.0)?;
            let port = i32::decode(buf)?;
            let rack = KafkaString::decode_compact(buf)?.0;
            let _ = TaggedFields::decode(buf)?;
            node_endpoints.push(ShareNodeEndpoint {
                node_id,
                host,
                port,
                rack,
            });
        }
        let _ = TaggedFields::decode(buf)?; // top-level tagged
        Ok((responses, node_endpoints))
    }

    /// Decode from version 1.
    pub fn decode_v1(buf: &mut impl Buf) -> Result<Self> {
        let throttle_time_ms = i32::decode(buf)?;
        let error_code = ErrorCode::from_i16(i16::decode(buf)?);
        let error_message = KafkaString::decode_compact(buf)?.0;
        let (responses, node_endpoints) = Self::decode_responses_and_endpoints(buf)?;
        Ok(Self {
            throttle_time_ms,
            error_code,
            error_message,
            acquisition_lock_timeout_ms: -1,
            responses,
            node_endpoints,
        })
    }

    /// Decode from version 2 (KIP-1222). Adds `AcquisitionLockTimeoutMs` field.
    pub fn decode_v2(buf: &mut impl Buf) -> Result<Self> {
        let throttle_time_ms = i32::decode(buf)?;
        let error_code = ErrorCode::from_i16(i16::decode(buf)?);
        let error_message = KafkaString::decode_compact(buf)?.0;
        let acquisition_lock_timeout_ms = i32::decode(buf)?;
        let (responses, node_endpoints) = Self::decode_responses_and_endpoints(buf)?;
        Ok(Self {
            throttle_time_ms,
            error_code,
            error_message,
            acquisition_lock_timeout_ms,
            responses,
            node_endpoints,
        })
    }
}

// ── Versioned dispatch for feature-gated APIs ────────────────────────────

impl VersionedEncode for ShareGroupHeartbeatRequest {
    fn encode_versioned(&self, version: i16, buf: &mut impl BufMut) -> Result<()> {
        match version {
            1 => self.encode_v1(buf)?,
            _ => return unsupported_encode!("ShareGroupHeartbeatRequest", version),
        }
        Ok(())
    }
}

impl VersionedDecode for ShareGroupHeartbeatResponse {
    fn decode_versioned(version: i16, buf: &mut impl Buf) -> Result<Self> {
        match version {
            1 => Self::decode_v1(buf),
            _ => unsupported_decode!("ShareGroupHeartbeatResponse", version),
        }
    }
}

impl VersionedEncode for ShareGroupDescribeRequest {
    fn encode_versioned(&self, version: i16, buf: &mut impl BufMut) -> Result<()> {
        match version {
            1 => self.encode_v1(buf)?,
            _ => return unsupported_encode!("ShareGroupDescribeRequest", version),
        }
        Ok(())
    }
}

impl VersionedDecode for ShareGroupDescribeResponse {
    fn decode_versioned(version: i16, buf: &mut impl Buf) -> Result<Self> {
        match version {
            1 => Self::decode_v1(buf),
            _ => unsupported_decode!("ShareGroupDescribeResponse", version),
        }
    }
}

impl VersionedDecode for ShareFetchResponse {
    fn decode_versioned(version: i16, buf: &mut impl Buf) -> Result<Self> {
        match version {
            1 => Self::decode_v1(buf),
            2 => Self::decode_v2(buf),
            _ => unsupported_decode!("ShareFetchResponse", version),
        }
    }
}

impl VersionedDecode for ShareAcknowledgeResponse {
    fn decode_versioned(version: i16, buf: &mut impl Buf) -> Result<Self> {
        match version {
            1 => Self::decode_v1(buf),
            2 => Self::decode_v2(buf),
            _ => unsupported_decode!("ShareAcknowledgeResponse", version),
        }
    }
}

#[cfg(test)]
#[allow(clippy::unwrap_used, clippy::expect_used, clippy::panic)]
mod tests {
    use super::*;
    use crate::util::varint;
    use bytes::BytesMut;

    // ===================================================================
    // Story 18.6: Share Group Wire-Format Tests (KIP-932)
    // ===================================================================

    #[test]
    fn test_share_group_heartbeat_encode_v1() {
        let req = ShareGroupHeartbeatRequest {
            group_id: "sg-1".to_string(),
            member_id: "m-1".to_string(),
            member_epoch: 0,
            rack_id: Some("us-east-1a".to_string()),
            subscribed_topic_names: Some(vec!["topic-a".to_string()]),
        };
        let mut buf = BytesMut::new();
        req.encode_versioned(1, &mut buf).unwrap();
        assert!(!buf.is_empty());
    }

    #[test]
    fn test_share_group_heartbeat_response_decode_v1() {
        let mut buf = BytesMut::new();
        buf.put_i32(10); // throttle_time_ms
        buf.put_i16(0); // error_code
        // error_message: null compact string
        varint::encode_unsigned_varint(0, &mut buf);
        // member_id: compact string "m-1"
        varint::encode_unsigned_varint(4, &mut buf);
        buf.put_slice(b"m-1");
        buf.put_i32(1); // member_epoch
        buf.put_i32(5000); // heartbeat_interval_ms
        buf.put_i8(-1); // assignment: null
        varint::encode_unsigned_varint(0, &mut buf); // top-level tagged

        let resp = ShareGroupHeartbeatResponse::decode_versioned(1, &mut buf.freeze()).unwrap();
        assert_eq!(resp.throttle_time_ms, 10);
        assert_eq!(resp.member_id.as_deref(), Some("m-1"));
        assert_eq!(resp.member_epoch, 1);
        assert_eq!(resp.heartbeat_interval_ms, 5000);
        assert!(resp.assignment.is_none());
    }

    #[test]
    fn test_share_group_describe_encode_v1() {
        let req = ShareGroupDescribeRequest {
            group_ids: vec!["sg-1".to_string(), "sg-2".to_string()],
            include_authorized_operations: true,
        };
        let mut buf = BytesMut::new();
        req.encode_versioned(1, &mut buf).unwrap();
        assert!(!buf.is_empty());
    }

    #[test]
    fn test_share_fetch_encode_v1_and_v2() {
        let req = ShareFetchRequest {
            group_id: Some("sg-1".to_string()),
            member_id: Some("m-1".to_string()),
            share_session_epoch: 0,
            max_wait_ms: 500,
            min_bytes: 1,
            max_bytes: 1_048_576,
            max_records: 100,
            batch_size: 50,
            topics: vec![ShareFetchTopic {
                topic_id: [4u8; 16],
                partitions: vec![ShareFetchPartition {
                    partition_index: 0,
                    acknowledgement_batches: vec![],
                }],
            }],
            forgotten_topics: vec![],
        };
        let mut buf_v1 = BytesMut::new();
        req.encode_v1(&mut buf_v1).unwrap();
        assert!(!buf_v1.is_empty());

        let mut buf_v2 = BytesMut::new();
        req.encode_v2(&mut buf_v2, 0, false).unwrap();
        // v2 has two extra fields (share_acquire_mode + is_renew_ack)
        assert!(
            buf_v2.len() > buf_v1.len(),
            "v2 should be longer than v1 due to share_acquire_mode + is_renew_ack"
        );
    }

    #[test]
    fn test_share_acknowledge_encode_v1_and_v2() {
        let req = ShareAcknowledgeRequest {
            group_id: Some("sg-1".to_string()),
            member_id: Some("m-1".to_string()),
            share_session_epoch: 0,
            topics: vec![ShareAcknowledgeTopic {
                topic_id: [5u8; 16],
                partitions: vec![ShareAcknowledgePartition {
                    partition_index: 0,
                    acknowledgement_batches: vec![ShareAcknowledgementBatch {
                        first_offset: 0,
                        last_offset: 99,
                        acknowledge_types: vec![1; 100], // Accept all
                    }],
                }],
            }],
        };
        let mut buf_v1 = BytesMut::new();
        req.encode_v1(&mut buf_v1).unwrap();
        assert!(!buf_v1.is_empty());

        let mut buf_v2 = BytesMut::new();
        req.encode_v2(&mut buf_v2, true).unwrap();
        // v2 adds is_renew_ack field (1 byte).
        assert!(
            buf_v2.len() > buf_v1.len(),
            "v2 should be longer than v1 due to is_renew_ack"
        );
    }

    #[test]
    fn test_share_acknowledge_response_decode_v1() {
        let mut buf = BytesMut::new();
        buf.put_i32(0); // throttle_time_ms
        buf.put_i16(0); // error_code
        // error_message: null
        varint::encode_unsigned_varint(0, &mut buf);
        // responses: 1 topic
        varint::encode_unsigned_varint(2, &mut buf);
        buf.put_slice(&[6u8; 16]); // topic_id
        // 1 partition
        varint::encode_unsigned_varint(2, &mut buf);
        buf.put_i32(0); // partition_index
        buf.put_i16(0); // error_code
        // error_message: null
        varint::encode_unsigned_varint(0, &mut buf);
        // current_leader
        buf.put_i32(1); // leader_id
        buf.put_i32(5); // leader_epoch
        varint::encode_unsigned_varint(0, &mut buf); // leader tagged
        varint::encode_unsigned_varint(0, &mut buf); // partition tagged
        varint::encode_unsigned_varint(0, &mut buf); // topic tagged
        // node_endpoints: empty
        varint::encode_unsigned_varint(1, &mut buf);
        varint::encode_unsigned_varint(0, &mut buf); // top-level tagged

        let resp = ShareAcknowledgeResponse::decode_versioned(1, &mut buf.freeze()).unwrap();
        assert_eq!(resp.acquisition_lock_timeout_ms, -1);
        assert_eq!(resp.responses.len(), 1);
        assert_eq!(resp.responses[0].partitions[0].current_leader.leader_id, 1);
        assert_eq!(
            resp.responses[0].partitions[0].current_leader.leader_epoch,
            5
        );
    }

    #[test]
    fn test_share_acknowledge_response_decode_v2() {
        let mut buf = BytesMut::new();
        buf.put_i32(0); // throttle_time_ms
        buf.put_i16(0); // error_code
        // error_message: null
        varint::encode_unsigned_varint(0, &mut buf);
        buf.put_i32(30_000); // acquisition_lock_timeout_ms
        // responses: 1 topic
        varint::encode_unsigned_varint(2, &mut buf);
        buf.put_slice(&[6u8; 16]); // topic_id
        // 1 partition
        varint::encode_unsigned_varint(2, &mut buf);
        buf.put_i32(0); // partition_index
        buf.put_i16(0); // error_code
        // error_message: null
        varint::encode_unsigned_varint(0, &mut buf);
        // current_leader
        buf.put_i32(1); // leader_id
        buf.put_i32(5); // leader_epoch
        varint::encode_unsigned_varint(0, &mut buf); // leader tagged
        varint::encode_unsigned_varint(0, &mut buf); // partition tagged
        varint::encode_unsigned_varint(0, &mut buf); // topic tagged
        // node_endpoints: empty
        varint::encode_unsigned_varint(1, &mut buf);
        varint::encode_unsigned_varint(0, &mut buf); // top-level tagged

        let resp = ShareAcknowledgeResponse::decode_versioned(2, &mut buf.freeze()).unwrap();
        assert_eq!(resp.acquisition_lock_timeout_ms, 30_000);
        assert_eq!(resp.responses.len(), 1);
        assert_eq!(resp.responses[0].partitions[0].current_leader.leader_id, 1);
    }
}