forked from kubernetes-retired/heapster
-
Notifications
You must be signed in to change notification settings - Fork 0
/
messages.go
1473 lines (1264 loc) · 33.8 KB
/
messages.go
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
package proto
import (
"bytes"
"compress/gzip"
"encoding/binary"
"errors"
"fmt"
"hash/crc32"
"io"
"io/ioutil"
"time"
"github.com/golang/snappy"
)
/*
Kafka wire protocol implemented as described in
https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-Messagesets
*/
const (
ProduceReqKind = 0
FetchReqKind = 1
OffsetReqKind = 2
MetadataReqKind = 3
OffsetCommitReqKind = 8
OffsetFetchReqKind = 9
ConsumerMetadataReqKind = 10
// receive the latest offset (i.e. the offset of the next coming message)
OffsetReqTimeLatest = -1
// receive the earliest available offset. Note that because offsets are
// pulled in descending order, asking for the earliest offset will always
// return you a single element.
OffsetReqTimeEarliest = -2
// Server will not send any response.
RequiredAcksNone = 0
// Server will block until the message is committed by all in sync replicas
// before sending a response.
RequiredAcksAll = -1
// Server will wait the data is written to the local log before sending a
// response.
RequiredAcksLocal = 1
)
type Compression int8
const (
CompressionNone Compression = 0
CompressionGzip Compression = 1
CompressionSnappy Compression = 2
)
// ReadReq returns request kind ID and byte representation of the whole message
// in wire protocol format.
func ReadReq(r io.Reader) (requestKind int16, b []byte, err error) {
dec := NewDecoder(r)
msgSize := dec.DecodeInt32()
requestKind = dec.DecodeInt16()
if err := dec.Err(); err != nil {
return 0, nil, err
}
// size of the message + size of the message itself
b = make([]byte, msgSize+4)
binary.BigEndian.PutUint32(b, uint32(msgSize))
binary.BigEndian.PutUint16(b[4:], uint16(requestKind))
if _, err := io.ReadFull(r, b[6:]); err != nil {
return 0, nil, err
}
return requestKind, b, err
}
// ReadResp returns message correlation ID and byte representation of the whole
// message in wire protocol that is returned when reading from given stream,
// including 4 bytes of message size itself.
// Byte representation returned by ReadResp can be parsed by all response
// reeaders to transform it into specialized response structure.
func ReadResp(r io.Reader) (correlationID int32, b []byte, err error) {
dec := NewDecoder(r)
msgSize := dec.DecodeInt32()
correlationID = dec.DecodeInt32()
if err := dec.Err(); err != nil {
return 0, nil, err
}
// size of the message + size of the message itself
b = make([]byte, msgSize+4)
binary.BigEndian.PutUint32(b, uint32(msgSize))
binary.BigEndian.PutUint32(b[4:], uint32(correlationID))
_, err = io.ReadFull(r, b[8:])
return correlationID, b, err
}
// Message represents single entity of message set.
type Message struct {
Key []byte
Value []byte
Offset int64 // set when fetching and after successful producing
Crc uint32 // set when fetching, ignored when producing
Topic string // set when fetching, ignored when producing
Partition int32 // set when fetching, ignored when producing
TipOffset int64 // set when fetching, ignored when processing
}
// ComputeCrc returns crc32 hash for given message content.
func ComputeCrc(m *Message, compression Compression) uint32 {
var buf bytes.Buffer
enc := NewEncoder(&buf)
enc.EncodeInt8(0) // magic byte is always 0
enc.EncodeInt8(int8(compression))
enc.EncodeBytes(m.Key)
enc.EncodeBytes(m.Value)
return crc32.ChecksumIEEE(buf.Bytes())
}
// writeMessageSet writes a Message Set into w.
// It returns the number of bytes written and any error.
func writeMessageSet(w io.Writer, messages []*Message, compression Compression) (int, error) {
if len(messages) == 0 {
return 0, nil
}
// NOTE(caleb): it doesn't appear to be documented, but I observed that the
// Java client sets the offset of the synthesized message set for a group of
// compressed messages to be the offset of the last message in the set.
compressOffset := messages[len(messages)-1].Offset
switch compression {
case CompressionGzip:
var buf bytes.Buffer
gz := gzip.NewWriter(&buf)
if _, err := writeMessageSet(gz, messages, CompressionNone); err != nil {
return 0, err
}
if err := gz.Close(); err != nil {
return 0, err
}
messages = []*Message{
{
Value: buf.Bytes(),
Offset: compressOffset,
},
}
case CompressionSnappy:
var buf bytes.Buffer
if _, err := writeMessageSet(&buf, messages, CompressionNone); err != nil {
return 0, err
}
messages = []*Message{
{
Value: snappy.Encode(nil, buf.Bytes()),
Offset: compressOffset,
},
}
}
totalSize := 0
b := newSliceWriter(0)
for _, message := range messages {
bsize := 26 + len(message.Key) + len(message.Value)
b.Reset(bsize)
enc := NewEncoder(b)
enc.EncodeInt64(message.Offset)
msize := int32(14 + len(message.Key) + len(message.Value))
enc.EncodeInt32(msize)
enc.EncodeUint32(0) // crc32 placeholder
enc.EncodeInt8(0) // magic byte
enc.EncodeInt8(int8(compression))
enc.EncodeBytes(message.Key)
enc.EncodeBytes(message.Value)
if err := enc.Err(); err != nil {
return totalSize, err
}
const hsize = 8 + 4 + 4 // offset + message size + crc32
const crcoff = 8 + 4 // offset + message size
binary.BigEndian.PutUint32(b.buf[crcoff:crcoff+4], crc32.ChecksumIEEE(b.buf[hsize:bsize]))
if n, err := w.Write(b.Slice()); err != nil {
return totalSize, err
} else {
totalSize += n
}
}
return totalSize, nil
}
type slicewriter struct {
buf []byte
pos int
size int
}
func newSliceWriter(bufsize int) *slicewriter {
return &slicewriter{
buf: make([]byte, bufsize),
pos: 0,
}
}
func (w *slicewriter) Write(p []byte) (int, error) {
if len(w.buf) < w.pos+len(p) {
return 0, errors.New("buffer too small")
}
copy(w.buf[w.pos:], p)
w.pos += len(p)
return len(p), nil
}
func (w *slicewriter) Reset(size int) {
if size > len(w.buf) {
w.buf = make([]byte, size+1000) // allocate a bit more than required
}
w.size = size
w.pos = 0
}
func (w *slicewriter) Slice() []byte {
return w.buf[:w.pos]
}
// readMessageSet reads and return messages from the stream.
// The size is known before a message set is decoded.
// Because kafka is sending message set directly from the drive, it might cut
// off part of the last message. This also means that the last message can be
// shorter than the header is saying. In such case just ignore the last
// malformed message from the set and returned earlier data.
func readMessageSet(r io.Reader, size int32) ([]*Message, error) {
rd := io.LimitReader(r, int64(size))
dec := NewDecoder(rd)
set := make([]*Message, 0, 256)
var buf []byte
for {
offset := dec.DecodeInt64()
if err := dec.Err(); err != nil {
if err == io.EOF || err == io.ErrUnexpectedEOF {
return set, nil
}
return nil, err
}
// single message size
size := dec.DecodeInt32()
if err := dec.Err(); err != nil {
if err == io.EOF || err == io.ErrUnexpectedEOF {
return set, nil
}
return nil, err
}
// read message to buffer to compute its content crc
if int(size) > len(buf) {
// allocate a bit more than needed
buf = make([]byte, size+10240)
}
msgbuf := buf[:size]
if _, err := io.ReadFull(rd, msgbuf); err != nil {
if err == io.EOF || err == io.ErrUnexpectedEOF {
return set, nil
}
return nil, err
}
msgdec := NewDecoder(bytes.NewBuffer(msgbuf))
msg := &Message{
Offset: offset,
Crc: msgdec.DecodeUint32(),
}
if msg.Crc != crc32.ChecksumIEEE(msgbuf[4:]) {
// ignore this message and because we want to have constant
// history, do not process anything more
return set, nil
}
// magic byte
_ = msgdec.DecodeInt8()
attributes := msgdec.DecodeInt8()
switch compression := Compression(attributes & 3); compression {
case CompressionNone:
msg.Key = msgdec.DecodeBytes()
msg.Value = msgdec.DecodeBytes()
if err := msgdec.Err(); err != nil {
return nil, fmt.Errorf("cannot decode message: %s", err)
}
set = append(set, msg)
case CompressionGzip, CompressionSnappy:
_ = msgdec.DecodeBytes() // ignore key
val := msgdec.DecodeBytes()
if err := msgdec.Err(); err != nil {
return nil, fmt.Errorf("cannot decode message: %s", err)
}
var decoded []byte
switch compression {
case CompressionGzip:
cr, err := gzip.NewReader(bytes.NewReader(val))
if err != nil {
return nil, fmt.Errorf("error decoding gzip message: %s", err)
}
decoded, err = ioutil.ReadAll(cr)
if err != nil {
return nil, fmt.Errorf("error decoding gzip message: %s", err)
}
_ = cr.Close()
case CompressionSnappy:
var err error
decoded, err = snappyDecode(val)
if err != nil {
return nil, fmt.Errorf("error decoding snappy message: %s", err)
}
}
msgs, err := readMessageSet(bytes.NewReader(decoded), int32(len(decoded)))
if err != nil {
return nil, err
}
set = append(set, msgs...)
default:
return nil, fmt.Errorf("cannot handle compression method: %d", compression)
}
}
}
type MetadataReq struct {
CorrelationID int32
ClientID string
Topics []string
}
func ReadMetadataReq(r io.Reader) (*MetadataReq, error) {
var req MetadataReq
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
// api key + api version
_ = dec.DecodeInt32()
req.CorrelationID = dec.DecodeInt32()
req.ClientID = dec.DecodeString()
req.Topics = make([]string, dec.DecodeArrayLen())
for i := range req.Topics {
req.Topics[i] = dec.DecodeString()
}
if dec.Err() != nil {
return nil, dec.Err()
}
return &req, nil
}
func (r *MetadataReq) Bytes() ([]byte, error) {
var buf bytes.Buffer
enc := NewEncoder(&buf)
// message size - for now just placeholder
enc.Encode(int32(0))
enc.Encode(int16(MetadataReqKind))
enc.Encode(int16(0))
enc.Encode(r.CorrelationID)
enc.Encode(r.ClientID)
enc.EncodeArrayLen(len(r.Topics))
for _, name := range r.Topics {
enc.Encode(name)
}
if enc.Err() != nil {
return nil, enc.Err()
}
// update the message size information
b := buf.Bytes()
binary.BigEndian.PutUint32(b, uint32(len(b)-4))
return b, nil
}
func (r *MetadataReq) WriteTo(w io.Writer) (int64, error) {
b, err := r.Bytes()
if err != nil {
return 0, err
}
n, err := w.Write(b)
return int64(n), err
}
type MetadataResp struct {
CorrelationID int32
Brokers []MetadataRespBroker
Topics []MetadataRespTopic
}
type MetadataRespBroker struct {
NodeID int32
Host string
Port int32
}
type MetadataRespTopic struct {
Name string
Err error
Partitions []MetadataRespPartition
}
type MetadataRespPartition struct {
ID int32
Err error
Leader int32
Replicas []int32
Isrs []int32
}
func (r *MetadataResp) Bytes() ([]byte, error) {
var buf bytes.Buffer
enc := NewEncoder(&buf)
// message size - for now just placeholder
enc.Encode(int32(0))
enc.Encode(r.CorrelationID)
enc.EncodeArrayLen(len(r.Brokers))
for _, broker := range r.Brokers {
enc.Encode(broker.NodeID)
enc.Encode(broker.Host)
enc.Encode(broker.Port)
}
enc.EncodeArrayLen(len(r.Topics))
for _, topic := range r.Topics {
enc.EncodeError(topic.Err)
enc.Encode(topic.Name)
enc.EncodeArrayLen(len(topic.Partitions))
for _, part := range topic.Partitions {
enc.EncodeError(part.Err)
enc.Encode(part.ID)
enc.Encode(part.Leader)
enc.Encode(part.Replicas)
enc.Encode(part.Isrs)
}
}
if enc.Err() != nil {
return nil, enc.Err()
}
// update the message size information
b := buf.Bytes()
binary.BigEndian.PutUint32(b, uint32(len(b)-4))
return b, nil
}
func ReadMetadataResp(r io.Reader) (*MetadataResp, error) {
var resp MetadataResp
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
resp.CorrelationID = dec.DecodeInt32()
resp.Brokers = make([]MetadataRespBroker, dec.DecodeArrayLen())
for i := range resp.Brokers {
var b = &resp.Brokers[i]
b.NodeID = dec.DecodeInt32()
b.Host = dec.DecodeString()
b.Port = dec.DecodeInt32()
}
resp.Topics = make([]MetadataRespTopic, dec.DecodeArrayLen())
for ti := range resp.Topics {
var t = &resp.Topics[ti]
t.Err = errFromNo(dec.DecodeInt16())
t.Name = dec.DecodeString()
t.Partitions = make([]MetadataRespPartition, dec.DecodeArrayLen())
for pi := range t.Partitions {
var p = &t.Partitions[pi]
p.Err = errFromNo(dec.DecodeInt16())
p.ID = dec.DecodeInt32()
p.Leader = dec.DecodeInt32()
p.Replicas = make([]int32, dec.DecodeArrayLen())
for ri := range p.Replicas {
p.Replicas[ri] = dec.DecodeInt32()
}
p.Isrs = make([]int32, dec.DecodeArrayLen())
for ii := range p.Isrs {
p.Isrs[ii] = dec.DecodeInt32()
}
}
}
if dec.Err() != nil {
return nil, dec.Err()
}
return &resp, nil
}
type FetchReq struct {
CorrelationID int32
ClientID string
MaxWaitTime time.Duration
MinBytes int32
Topics []FetchReqTopic
}
type FetchReqTopic struct {
Name string
Partitions []FetchReqPartition
}
type FetchReqPartition struct {
ID int32
FetchOffset int64
MaxBytes int32
}
func ReadFetchReq(r io.Reader) (*FetchReq, error) {
var req FetchReq
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
// api key + api version
_ = dec.DecodeInt32()
req.CorrelationID = dec.DecodeInt32()
req.ClientID = dec.DecodeString()
// replica id
_ = dec.DecodeInt32()
req.MaxWaitTime = time.Duration(dec.DecodeInt32()) * time.Millisecond
req.MinBytes = dec.DecodeInt32()
req.Topics = make([]FetchReqTopic, dec.DecodeArrayLen())
for ti := range req.Topics {
var topic = &req.Topics[ti]
topic.Name = dec.DecodeString()
topic.Partitions = make([]FetchReqPartition, dec.DecodeArrayLen())
for pi := range topic.Partitions {
var part = &topic.Partitions[pi]
part.ID = dec.DecodeInt32()
part.FetchOffset = dec.DecodeInt64()
part.MaxBytes = dec.DecodeInt32()
}
}
if dec.Err() != nil {
return nil, dec.Err()
}
return &req, nil
}
func (r *FetchReq) Bytes() ([]byte, error) {
var buf bytes.Buffer
enc := NewEncoder(&buf)
// message size - for now just placeholder
enc.Encode(int32(0))
enc.Encode(int16(FetchReqKind))
enc.Encode(int16(0))
enc.Encode(r.CorrelationID)
enc.Encode(r.ClientID)
// replica id
enc.Encode(int32(-1))
enc.Encode(int32(r.MaxWaitTime / time.Millisecond))
enc.Encode(r.MinBytes)
enc.EncodeArrayLen(len(r.Topics))
for _, topic := range r.Topics {
enc.Encode(topic.Name)
enc.EncodeArrayLen(len(topic.Partitions))
for _, part := range topic.Partitions {
enc.Encode(part.ID)
enc.Encode(part.FetchOffset)
enc.Encode(part.MaxBytes)
}
}
if enc.Err() != nil {
return nil, enc.Err()
}
// update the message size information
b := buf.Bytes()
binary.BigEndian.PutUint32(b, uint32(len(b)-4))
return b, nil
}
func (r *FetchReq) WriteTo(w io.Writer) (int64, error) {
b, err := r.Bytes()
if err != nil {
return 0, err
}
n, err := w.Write(b)
return int64(n), err
}
type FetchResp struct {
CorrelationID int32
Topics []FetchRespTopic
}
type FetchRespTopic struct {
Name string
Partitions []FetchRespPartition
}
type FetchRespPartition struct {
ID int32
Err error
TipOffset int64
Messages []*Message
}
func (r *FetchResp) Bytes() ([]byte, error) {
var buf buffer
enc := NewEncoder(&buf)
enc.Encode(int32(0)) // placeholder
enc.Encode(r.CorrelationID)
enc.EncodeArrayLen(len(r.Topics))
for _, topic := range r.Topics {
enc.Encode(topic.Name)
enc.EncodeArrayLen(len(topic.Partitions))
for _, part := range topic.Partitions {
enc.Encode(part.ID)
enc.EncodeError(part.Err)
enc.Encode(part.TipOffset)
i := len(buf)
enc.Encode(int32(0)) // placeholder
// NOTE(caleb): writing compressed fetch response isn't implemented
// for now, since that's not needed for clients.
n, err := writeMessageSet(&buf, part.Messages, CompressionNone)
if err != nil {
return nil, err
}
binary.BigEndian.PutUint32(buf[i:i+4], uint32(n))
}
}
if enc.Err() != nil {
return nil, enc.Err()
}
binary.BigEndian.PutUint32(buf[:4], uint32(len(buf)-4))
return []byte(buf), nil
}
func ReadFetchResp(r io.Reader) (*FetchResp, error) {
var err error
var resp FetchResp
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
resp.CorrelationID = dec.DecodeInt32()
resp.Topics = make([]FetchRespTopic, dec.DecodeArrayLen())
for ti := range resp.Topics {
var topic = &resp.Topics[ti]
topic.Name = dec.DecodeString()
topic.Partitions = make([]FetchRespPartition, dec.DecodeArrayLen())
for pi := range topic.Partitions {
var part = &topic.Partitions[pi]
part.ID = dec.DecodeInt32()
part.Err = errFromNo(dec.DecodeInt16())
part.TipOffset = dec.DecodeInt64()
if dec.Err() != nil {
return nil, dec.Err()
}
msgSetSize := dec.DecodeInt32()
if dec.Err() != nil {
return nil, dec.Err()
}
if part.Messages, err = readMessageSet(r, msgSetSize); err != nil {
return nil, err
}
for _, msg := range part.Messages {
msg.Topic = topic.Name
msg.Partition = part.ID
msg.TipOffset = part.TipOffset
}
}
}
if dec.Err() != nil {
return nil, dec.Err()
}
return &resp, nil
}
type ConsumerMetadataReq struct {
CorrelationID int32
ClientID string
ConsumerGroup string
}
func ReadConsumerMetadataReq(r io.Reader) (*ConsumerMetadataReq, error) {
var req ConsumerMetadataReq
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
// api key + api version
_ = dec.DecodeInt32()
req.CorrelationID = dec.DecodeInt32()
req.ClientID = dec.DecodeString()
req.ConsumerGroup = dec.DecodeString()
if dec.Err() != nil {
return nil, dec.Err()
}
return &req, nil
}
func (r *ConsumerMetadataReq) Bytes() ([]byte, error) {
var buf bytes.Buffer
enc := NewEncoder(&buf)
// message size - for now just placeholder
enc.Encode(int32(0))
enc.Encode(int16(ConsumerMetadataReqKind))
enc.Encode(int16(0))
enc.Encode(r.CorrelationID)
enc.Encode(r.ClientID)
enc.Encode(r.ConsumerGroup)
if enc.Err() != nil {
return nil, enc.Err()
}
// update the message size information
b := buf.Bytes()
binary.BigEndian.PutUint32(b, uint32(len(b)-4))
return b, nil
}
func (r *ConsumerMetadataReq) WriteTo(w io.Writer) (int64, error) {
b, err := r.Bytes()
if err != nil {
return 0, err
}
n, err := w.Write(b)
return int64(n), err
}
type ConsumerMetadataResp struct {
CorrelationID int32
Err error
CoordinatorID int32
CoordinatorHost string
CoordinatorPort int32
}
func ReadConsumerMetadataResp(r io.Reader) (*ConsumerMetadataResp, error) {
var resp ConsumerMetadataResp
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
resp.CorrelationID = dec.DecodeInt32()
resp.Err = errFromNo(dec.DecodeInt16())
resp.CoordinatorID = dec.DecodeInt32()
resp.CoordinatorHost = dec.DecodeString()
resp.CoordinatorPort = dec.DecodeInt32()
if err := dec.Err(); err != nil {
return nil, err
}
return &resp, nil
}
func (r *ConsumerMetadataResp) Bytes() ([]byte, error) {
var buf bytes.Buffer
enc := NewEncoder(&buf)
// message size - for now just placeholder
enc.Encode(int32(0))
enc.Encode(r.CorrelationID)
enc.EncodeError(r.Err)
enc.Encode(r.CoordinatorID)
enc.Encode(r.CoordinatorHost)
enc.Encode(r.CoordinatorPort)
if enc.Err() != nil {
return nil, enc.Err()
}
// update the message size information
b := buf.Bytes()
binary.BigEndian.PutUint32(b, uint32(len(b)-4))
return b, nil
}
type OffsetCommitReq struct {
CorrelationID int32
ClientID string
ConsumerGroup string
Topics []OffsetCommitReqTopic
}
type OffsetCommitReqTopic struct {
Name string
Partitions []OffsetCommitReqPartition
}
type OffsetCommitReqPartition struct {
ID int32
Offset int64
TimeStamp time.Time
Metadata string
}
func ReadOffsetCommitReq(r io.Reader) (*OffsetCommitReq, error) {
var req OffsetCommitReq
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
// api key + api version
_ = dec.DecodeInt32()
req.CorrelationID = dec.DecodeInt32()
req.ClientID = dec.DecodeString()
req.ConsumerGroup = dec.DecodeString()
req.Topics = make([]OffsetCommitReqTopic, dec.DecodeArrayLen())
for ti := range req.Topics {
var topic = &req.Topics[ti]
topic.Name = dec.DecodeString()
topic.Partitions = make([]OffsetCommitReqPartition, dec.DecodeArrayLen())
for pi := range topic.Partitions {
var part = &topic.Partitions[pi]
part.ID = dec.DecodeInt32()
part.Offset = dec.DecodeInt64()
part.TimeStamp = time.Unix(0, dec.DecodeInt64()*int64(time.Millisecond))
part.Metadata = dec.DecodeString()
}
}
if dec.Err() != nil {
return nil, dec.Err()
}
return &req, nil
}
func (r *OffsetCommitReq) Bytes() ([]byte, error) {
var buf bytes.Buffer
enc := NewEncoder(&buf)
// message size - for now just placeholder
enc.Encode(int32(0))
enc.Encode(int16(OffsetCommitReqKind))
enc.Encode(int16(0))
enc.Encode(r.CorrelationID)
enc.Encode(r.ClientID)
enc.Encode(r.ConsumerGroup)
enc.EncodeArrayLen(len(r.Topics))
for _, topic := range r.Topics {
enc.Encode(topic.Name)
enc.EncodeArrayLen(len(topic.Partitions))
for _, part := range topic.Partitions {
enc.Encode(part.ID)
enc.Encode(part.Offset)
// TODO(husio) is this really in milliseconds?
enc.Encode(part.TimeStamp.UnixNano() / int64(time.Millisecond))
enc.Encode(part.Metadata)
}
}
if enc.Err() != nil {
return nil, enc.Err()
}
// update the message size information
b := buf.Bytes()
binary.BigEndian.PutUint32(b, uint32(len(b)-4))
return b, nil
}
func (r *OffsetCommitReq) WriteTo(w io.Writer) (int64, error) {
b, err := r.Bytes()
if err != nil {
return 0, err
}
n, err := w.Write(b)
return int64(n), err
}
type OffsetCommitResp struct {
CorrelationID int32
Topics []OffsetCommitRespTopic
}
type OffsetCommitRespTopic struct {
Name string
Partitions []OffsetCommitRespPartition
}
type OffsetCommitRespPartition struct {
ID int32
Err error
}
func ReadOffsetCommitResp(r io.Reader) (*OffsetCommitResp, error) {
var resp OffsetCommitResp
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
resp.CorrelationID = dec.DecodeInt32()
resp.Topics = make([]OffsetCommitRespTopic, dec.DecodeArrayLen())
for ti := range resp.Topics {
var t = &resp.Topics[ti]
t.Name = dec.DecodeString()
t.Partitions = make([]OffsetCommitRespPartition, dec.DecodeArrayLen())
for pi := range t.Partitions {
var p = &t.Partitions[pi]
p.ID = dec.DecodeInt32()
p.Err = errFromNo(dec.DecodeInt16())
}
}
if err := dec.Err(); err != nil {
return nil, err
}
return &resp, nil
}
func (r *OffsetCommitResp) Bytes() ([]byte, error) {
var buf bytes.Buffer
enc := NewEncoder(&buf)
// message size - for now just placeholder
enc.Encode(int32(0))
enc.Encode(r.CorrelationID)
enc.EncodeArrayLen(len(r.Topics))
for _, t := range r.Topics {
enc.Encode(t.Name)
enc.EncodeArrayLen(len(t.Partitions))
for _, p := range t.Partitions {
enc.Encode(p.ID)
enc.EncodeError(p.Err)
}
}
if enc.Err() != nil {
return nil, enc.Err()
}
// update the message size information
b := buf.Bytes()
binary.BigEndian.PutUint32(b, uint32(len(b)-4))
return b, nil
}
type OffsetFetchReq struct {
CorrelationID int32
ClientID string
ConsumerGroup string
Topics []OffsetFetchReqTopic
}
type OffsetFetchReqTopic struct {
Name string
Partitions []int32
}
func ReadOffsetFetchReq(r io.Reader) (*OffsetFetchReq, error) {
var req OffsetFetchReq
dec := NewDecoder(r)
// total message size
_ = dec.DecodeInt32()
// api key + api version
_ = dec.DecodeInt32()
req.CorrelationID = dec.DecodeInt32()
req.ClientID = dec.DecodeString()
req.ConsumerGroup = dec.DecodeString()
req.Topics = make([]OffsetFetchReqTopic, dec.DecodeArrayLen())
for ti := range req.Topics {
var topic = &req.Topics[ti]
topic.Name = dec.DecodeString()
topic.Partitions = make([]int32, dec.DecodeArrayLen())
for pi := range topic.Partitions {
topic.Partitions[pi] = dec.DecodeInt32()