forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
message.go
593 lines (512 loc) · 13.6 KB
/
message.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
package kafka
import (
"bufio"
"bytes"
"fmt"
"io"
"time"
)
// Message is a data structure representing kafka messages.
type Message struct {
// Topic is reads only and MUST NOT be set when writing messages
Topic string
// Partition is reads only and MUST NOT be set when writing messages
Partition int
Offset int64
Key []byte
Value []byte
Headers []Header
// If not set at the creation, Time will be automatically set when
// writing the message.
Time time.Time
}
func (msg Message) message(cw *crc32Writer) message {
m := message{
MagicByte: 1,
Key: msg.Key,
Value: msg.Value,
Timestamp: timestamp(msg.Time),
}
if cw != nil {
m.CRC = m.crc32(cw)
}
return m
}
const timestampSize = 8
func (msg Message) size() int32 {
return 4 + 1 + 1 + sizeofBytes(msg.Key) + sizeofBytes(msg.Value) + timestampSize
}
type message struct {
CRC int32
MagicByte int8
Attributes int8
Timestamp int64
Key []byte
Value []byte
}
func (m message) crc32(cw *crc32Writer) int32 {
cw.crc32 = 0
cw.writeInt8(m.MagicByte)
cw.writeInt8(m.Attributes)
if m.MagicByte != 0 {
cw.writeInt64(m.Timestamp)
}
cw.writeBytes(m.Key)
cw.writeBytes(m.Value)
return int32(cw.crc32)
}
func (m message) size() int32 {
size := 4 + 1 + 1 + sizeofBytes(m.Key) + sizeofBytes(m.Value)
if m.MagicByte != 0 {
size += timestampSize
}
return size
}
func (m message) writeTo(wb *writeBuffer) {
wb.writeInt32(m.CRC)
wb.writeInt8(m.MagicByte)
wb.writeInt8(m.Attributes)
if m.MagicByte != 0 {
wb.writeInt64(m.Timestamp)
}
wb.writeBytes(m.Key)
wb.writeBytes(m.Value)
}
type messageSetItem struct {
Offset int64
MessageSize int32
Message message
}
func (m messageSetItem) size() int32 {
return 8 + 4 + m.Message.size()
}
func (m messageSetItem) writeTo(wb *writeBuffer) {
wb.writeInt64(m.Offset)
wb.writeInt32(m.MessageSize)
m.Message.writeTo(wb)
}
type messageSet []messageSetItem
func (s messageSet) size() (size int32) {
for _, m := range s {
size += m.size()
}
return
}
func (s messageSet) writeTo(wb *writeBuffer) {
for _, m := range s {
m.writeTo(wb)
}
}
type messageSetReader struct {
empty bool
version int
v1 messageSetReaderV1
v2 messageSetReaderV2
}
func (r *messageSetReader) readMessage(min int64,
key func(*bufio.Reader, int, int) (int, error),
val func(*bufio.Reader, int, int) (int, error),
) (offset int64, timestamp int64, headers []Header, err error) {
if r.empty {
return 0, 0, nil, RequestTimedOut
}
switch r.version {
case 1:
return r.v1.readMessage(min, key, val)
case 2:
return r.v2.readMessage(min, key, val)
default:
panic("Invalid messageSetReader - unknown message reader version")
}
}
func (r *messageSetReader) remaining() (remain int) {
if r.empty {
return 0
}
switch r.version {
case 1:
return r.v1.remaining()
case 2:
return r.v2.remaining()
default:
panic("Invalid messageSetReader - unknown message reader version")
}
}
func (r *messageSetReader) discard() (err error) {
if r.empty {
return nil
}
switch r.version {
case 1:
return r.v1.discard()
case 2:
return r.v2.discard()
default:
panic("Invalid messageSetReader - unknown message reader version")
}
}
type messageSetReaderV1 struct {
*readerStack
}
type readerStack struct {
reader *bufio.Reader
remain int
base int64
parent *readerStack
}
func newMessageSetReader(reader *bufio.Reader, remain int) (*messageSetReader, error) {
headerLength := 8 + 4 + 4 + 1 // offset + messageSize + crc + magicByte
if headerLength > remain {
return nil, errShortRead
}
b, err := reader.Peek(headerLength)
if err != nil {
return nil, err
}
var version int8 = int8(b[headerLength-1])
switch version {
case 0, 1:
return &messageSetReader{
version: 1,
v1: messageSetReaderV1{&readerStack{
reader: reader,
remain: remain,
}}}, nil
case 2:
mr := &messageSetReader{
version: 2,
v2: messageSetReaderV2{
readerStack: &readerStack{
reader: reader,
remain: remain,
},
messageCount: 0,
}}
return mr, nil
default:
return nil, fmt.Errorf("unsupported message version %d found in fetch response", version)
}
}
func (r *messageSetReaderV1) readMessage(min int64,
key func(*bufio.Reader, int, int) (int, error),
val func(*bufio.Reader, int, int) (int, error),
) (offset int64, timestamp int64, headers []Header, err error) {
for r.readerStack != nil {
if r.remain == 0 {
r.readerStack = r.parent
continue
}
var attributes int8
if offset, attributes, timestamp, r.remain, err = readMessageHeader(r.reader, r.remain); err != nil {
return
}
// if the message is compressed, decompress it and push a new reader
// onto the stack.
code := attributes & compressionCodecMask
if code != 0 {
var codec CompressionCodec
if codec, err = resolveCodec(code); err != nil {
return
}
// discard next four bytes...will be -1 to indicate null key
if r.remain, err = discardN(r.reader, r.remain, 4); err != nil {
return
}
// read and decompress the contained message set.
var decompressed bytes.Buffer
if r.remain, err = readBytesWith(r.reader, r.remain, func(r *bufio.Reader, sz, n int) (remain int, err error) {
// x4 as a guess that the average compression ratio is near 75%
decompressed.Grow(4 * n)
l := io.LimitedReader{R: r, N: int64(n)}
d := codec.NewReader(&l)
_, err = decompressed.ReadFrom(d)
remain = sz - (n - int(l.N))
d.Close()
return
}); err != nil {
return
}
// the compressed message's offset will be equal to the offset of
// the last message in the set. within the compressed set, the
// offsets will be relative, so we have to scan through them to
// get the base offset. for example, if there are four compressed
// messages at offsets 10-13, then the container message will have
// offset 13 and the contained messages will be 0,1,2,3. the base
// offset for the container, then is 13-3=10.
if offset, err = extractOffset(offset, decompressed.Bytes()); err != nil {
return
}
r.readerStack = &readerStack{
// Allocate a buffer of size 0, which gets capped at 16 bytes
// by the bufio package. We are already reading buffered data
// here, no need to reserve another 4KB buffer.
reader: bufio.NewReaderSize(&decompressed, 0),
remain: decompressed.Len(),
base: offset,
parent: r.readerStack,
}
continue
}
// adjust the offset in case we're reading compressed messages. the
// base will be zero otherwise.
offset += r.base
// When the messages are compressed kafka may return messages at an
// earlier offset than the one that was requested, it's the client's
// responsibility to ignore those.
if offset < min {
if r.remain, err = discardBytes(r.reader, r.remain); err != nil {
return
}
if r.remain, err = discardBytes(r.reader, r.remain); err != nil {
return
}
continue
}
if r.remain, err = readBytesWith(r.reader, r.remain, key); err != nil {
return
}
r.remain, err = readBytesWith(r.reader, r.remain, val)
return
}
err = errShortRead
return
}
func (r *messageSetReaderV1) remaining() (remain int) {
for s := r.readerStack; s != nil; s = s.parent {
remain += s.remain
}
return
}
func (r *messageSetReaderV1) discard() (err error) {
if r.readerStack == nil {
return
}
// rewind up to the top-most reader b/c it's the only one that's doing
// actual i/o. the rest are byte buffers that have been pushed on the stack
// while reading compressed message sets.
for r.parent != nil {
r.readerStack = r.parent
}
r.remain, err = discardN(r.reader, r.remain, r.remain)
return
}
func extractOffset(base int64, msgSet []byte) (offset int64, err error) {
r, remain := bufio.NewReader(bytes.NewReader(msgSet)), len(msgSet)
for remain > 0 {
if remain, err = readInt64(r, remain, &offset); err != nil {
return
}
var sz int32
if remain, err = readInt32(r, remain, &sz); err != nil {
return
}
if remain, err = discardN(r, remain, int(sz)); err != nil {
return
}
}
offset = base - offset
return
}
type Header struct {
Key string
Value []byte
}
type messageSetHeaderV2 struct {
firstOffset int64
length int32
partitionLeaderEpoch int32
magic int8
crc int32
batchAttributes int16
lastOffsetDelta int32
firstTimestamp int64
maxTimestamp int64
producerId int64
producerEpoch int16
firstSequence int32
}
type timestampType int8
const (
createTime timestampType = 0
logAppendTime timestampType = 1
)
type transactionType int8
const (
nonTransactional transactionType = 0
transactional transactionType = 1
)
type controlType int8
const (
nonControlMessage controlType = 0
controlMessage controlType = 1
)
func (h *messageSetHeaderV2) compression() int8 {
return int8(h.batchAttributes & 7)
}
func (h *messageSetHeaderV2) timestampType() timestampType {
return timestampType((h.batchAttributes & (1 << 3)) >> 3)
}
func (h *messageSetHeaderV2) transactionType() transactionType {
return transactionType((h.batchAttributes & (1 << 4)) >> 4)
}
func (h *messageSetHeaderV2) controlType() controlType {
return controlType((h.batchAttributes & (1 << 5)) >> 5)
}
type messageSetReaderV2 struct {
*readerStack
messageCount int
header messageSetHeaderV2
}
func (r *messageSetReaderV2) readHeader() (err error) {
h := &r.header
if r.remain, err = readInt64(r.reader, r.remain, &h.firstOffset); err != nil {
return
}
if r.remain, err = readInt32(r.reader, r.remain, &h.length); err != nil {
return
}
if r.remain, err = readInt32(r.reader, r.remain, &h.partitionLeaderEpoch); err != nil {
return
}
if r.remain, err = readInt8(r.reader, r.remain, &h.magic); err != nil {
return
}
if r.remain, err = readInt32(r.reader, r.remain, &h.crc); err != nil {
return
}
if r.remain, err = readInt16(r.reader, r.remain, &h.batchAttributes); err != nil {
return
}
if r.remain, err = readInt32(r.reader, r.remain, &h.lastOffsetDelta); err != nil {
return
}
if r.remain, err = readInt64(r.reader, r.remain, &h.firstTimestamp); err != nil {
return
}
if r.remain, err = readInt64(r.reader, r.remain, &h.maxTimestamp); err != nil {
return
}
if r.remain, err = readInt64(r.reader, r.remain, &h.producerId); err != nil {
return
}
if r.remain, err = readInt16(r.reader, r.remain, &h.producerEpoch); err != nil {
return
}
if r.remain, err = readInt32(r.reader, r.remain, &h.firstSequence); err != nil {
return
}
var messageCount int32
if r.remain, err = readInt32(r.reader, r.remain, &messageCount); err != nil {
return
}
r.messageCount = int(messageCount)
return nil
}
func (r *messageSetReaderV2) readMessage(min int64,
key func(*bufio.Reader, int, int) (int, error),
val func(*bufio.Reader, int, int) (int, error),
) (offset int64, timestamp int64, headers []Header, err error) {
if r.messageCount == 0 {
if r.remain == 0 {
if r.parent != nil {
r.readerStack = r.parent
}
}
if err = r.readHeader(); err != nil {
return
}
if code := r.header.compression(); code != 0 {
var codec CompressionCodec
if codec, err = resolveCodec(code); err != nil {
return
}
var batchRemain = int(r.header.length - 49)
if batchRemain > r.remain {
err = errShortRead
return
}
var decompressed bytes.Buffer
decompressed.Grow(4 * batchRemain)
l := io.LimitedReader{R: r.reader, N: int64(batchRemain)}
d := codec.NewReader(&l)
_, err = decompressed.ReadFrom(d)
r.remain = r.remain - (batchRemain - int(l.N))
d.Close()
if err != nil {
return
}
r.readerStack = &readerStack{
reader: bufio.NewReaderSize(&decompressed, 0),
remain: decompressed.Len(),
base: -1, // base is unused here
parent: r.readerStack,
}
}
}
var length int64
if r.remain, err = readVarInt(r.reader, r.remain, &length); err != nil {
return
}
var attrs int8
if r.remain, err = readInt8(r.reader, r.remain, &attrs); err != nil {
return
}
var timestampDelta int64
if r.remain, err = readVarInt(r.reader, r.remain, ×tampDelta); err != nil {
return
}
var offsetDelta int64
if r.remain, err = readVarInt(r.reader, r.remain, &offsetDelta); err != nil {
return
}
var keyLen int64
if r.remain, err = readVarInt(r.reader, r.remain, &keyLen); err != nil {
return
}
if r.remain, err = key(r.reader, r.remain, int(keyLen)); err != nil {
return
}
var valueLen int64
if r.remain, err = readVarInt(r.reader, r.remain, &valueLen); err != nil {
return
}
if r.remain, err = val(r.reader, r.remain, int(valueLen)); err != nil {
return
}
var headerCount int64
if r.remain, err = readVarInt(r.reader, r.remain, &headerCount); err != nil {
return
}
headers = make([]Header, headerCount)
for i := 0; i < int(headerCount); i++ {
if err = r.readMessageHeader(&headers[i]); err != nil {
return
}
}
r.messageCount--
return r.header.firstOffset + offsetDelta, r.header.firstTimestamp + timestampDelta, headers, nil
}
func (r *messageSetReaderV2) readMessageHeader(header *Header) (err error) {
var keyLen int64
if r.remain, err = readVarInt(r.reader, r.remain, &keyLen); err != nil {
return
}
if header.Key, r.remain, err = readNewString(r.reader, r.remain, int(keyLen)); err != nil {
return
}
var valLen int64
if r.remain, err = readVarInt(r.reader, r.remain, &valLen); err != nil {
return
}
if header.Value, r.remain, err = readNewBytes(r.reader, r.remain, int(valLen)); err != nil {
return
}
return nil
}
func (r *messageSetReaderV2) remaining() (remain int) {
return r.remain
}
func (r *messageSetReaderV2) discard() (err error) {
r.remain, err = discardN(r.reader, r.remain, r.remain)
return
}