-
Notifications
You must be signed in to change notification settings - Fork 0
/
message_reader.go
531 lines (493 loc) · 13.9 KB
/
message_reader.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
package kafka
import (
"bufio"
"bytes"
"encoding/hex"
"fmt"
"io"
"log"
)
type readBytesFunc func(*bufio.Reader, int, int) (int, error)
// messageSetReader processes the messages encoded into a fetch response.
// The response may contain a mix of Record Batches (newer format) and Messages
// (older format).
type messageSetReader struct {
*readerStack // used for decompressing compressed messages and record batches
empty bool // if true, short circuits messageSetReader methods
debug bool // enable debug log messages
}
type readerStack struct {
reader *bufio.Reader
remain int
base int64
parent *readerStack
count int // how many messages left in the current message set
header messagesHeader // the current header for a subset of messages within the set.
}
// messagesHeader describes a set of records. there may be many messagesHeader's in a message set.
type messagesHeader struct {
firstOffset int64
length int32
crc int32
magic int8
// v1 composes attributes specific to v0 and v1 message headers
v1 struct {
attributes int8
timestamp int64
}
// v2 composes attributes specific to v2 message headers
v2 struct {
leaderEpoch int32
attributes int16
lastOffsetDelta int32
firstTimestamp int64
lastTimestamp int64
producerID int64
producerEpoch int16
baseSequence int32
count int32
}
}
func (h messagesHeader) compression() (codec CompressionCodec, err error) {
const compressionCodecMask = 0x07
var code int8
switch h.magic {
case 0, 1:
code = h.v1.attributes & compressionCodecMask
case 2:
code = int8(h.v2.attributes & compressionCodecMask)
default:
err = h.badMagic()
return
}
if code != 0 {
codec, err = resolveCodec(code)
}
return
}
func (h messagesHeader) badMagic() error {
return fmt.Errorf("unsupported magic byte %d in header", h.magic)
}
func newMessageSetReader(reader *bufio.Reader, remain int) (*messageSetReader, error) {
res := &messageSetReader{
readerStack: &readerStack{
reader: reader,
remain: remain,
},
}
err := res.readHeader()
return res, err
}
func (r *messageSetReader) remaining() (remain int) {
if r.empty {
return 0
}
for s := r.readerStack; s != nil; s = s.parent {
remain += s.remain
}
return
}
func (r *messageSetReader) discard() (err error) {
switch {
case r.empty:
case r.readerStack == nil:
default:
// 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
}
err = r.discardN(r.remain)
}
return
}
func (r *messageSetReader) readMessage(min int64, key readBytesFunc, val readBytesFunc) (
offset int64, timestamp int64, headers []Header, err error) {
if r.empty {
err = RequestTimedOut
return
}
if err = r.readHeader(); err != nil {
return
}
switch r.header.magic {
case 0, 1:
offset, timestamp, headers, err = r.readMessageV1(min, key, val)
case 2:
offset, timestamp, headers, err = r.readMessageV2(min, key, val)
default:
err = r.header.badMagic()
}
return
}
func (r *messageSetReader) readMessageV1(min int64, key readBytesFunc, val readBytesFunc) (
offset int64, timestamp int64, headers []Header, err error) {
for r.readerStack != nil {
if r.remain == 0 {
r.readerStack = r.parent
continue
}
if err = r.readHeader(); err != nil {
return
}
offset = r.header.firstOffset
timestamp = r.header.v1.timestamp
var codec CompressionCodec
if codec, err = r.header.compression(); err != nil {
return
}
r.log("Reading with codec=%T", codec)
if codec != nil {
// discard next four bytes...will be -1 to indicate null key
if err = r.discardN(4); err != nil {
return
}
// read and decompress the contained message set.
var decompressed bytes.Buffer
if err = r.readBytesWith(func(r *bufio.Reader, sz int, n int) (remain int, err error) {
// x4 as a guess that the average compression ratio is near 75%
decompressed.Grow(4 * n)
limitReader := io.LimitedReader{R: r, N: int64(n)}
codecReader := codec.NewReader(&limitReader)
_, err = decompressed.ReadFrom(codecReader)
remain = sz - (n - int(limitReader.N))
codecReader.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
}
// mark the outer message as being read
r.markRead()
// then push the decompressed bytes onto the stack.
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.
//
// At this point, the message header has been read, so discarding
// the rest of the message means we have to discard the key, and then
// the value. Each of those are preceded by a 4-byte length. Discarding
// them is then reading that length variable and then discarding that
// amount.
if offset < min {
// discard the key
if err = r.discardBytes(); err != nil {
return
}
// discard the value
if err = r.discardBytes(); err != nil {
return
}
// since we have fully consumed the message, mark as read
r.markRead()
continue
}
if err = r.readBytesWith(key); err != nil {
return
}
if err = r.readBytesWith(val); err != nil {
return
}
r.markRead()
return
}
err = errShortRead
return
}
func (r *messageSetReader) readMessageV2(_ int64, key readBytesFunc, val readBytesFunc) (
offset int64, timestamp int64, headers []Header, err error) {
if err = r.readHeader(); err != nil {
return
}
if r.count == int(r.header.v2.count) { // first time reading this set, so check for compression headers.
var codec CompressionCodec
if codec, err = r.header.compression(); err != nil {
return
}
if codec != nil {
batchRemain := int(r.header.length - 49) // TODO: document this magic number
if batchRemain > r.remain {
err = errShortRead
return
}
if batchRemain < 0 {
err = fmt.Errorf("batch remain < 0 (%d)", batchRemain)
return
}
var decompressed bytes.Buffer
decompressed.Grow(4 * batchRemain)
limitReader := io.LimitedReader{R: r.reader, N: int64(batchRemain)}
codecReader := codec.NewReader(&limitReader)
_, err = decompressed.ReadFrom(codecReader)
codecReader.Close()
if err != nil {
return
}
r.remain -= batchRemain - int(limitReader.N)
r.readerStack = &readerStack{
reader: bufio.NewReaderSize(&decompressed, 0), // the new stack reads from the decompressed buffer
remain: decompressed.Len(),
base: -1, // base is unused here
parent: r.readerStack,
header: r.header,
count: r.count,
}
// all of the messages in this set are in the decompressed set just pushed onto the reader
// stack. here we set the parent count to 0 so that when the child set is exhausted, the
// reader will then try to read the header of the next message set
r.readerStack.parent.count = 0
}
}
var length int64
if err = r.readVarInt(&length); err != nil {
return
}
var attrs int8
if err = r.readInt8(&attrs); err != nil {
return
}
var timestampDelta int64
if err = r.readVarInt(×tampDelta); err != nil {
return
}
timestamp = r.header.v2.firstTimestamp + timestampDelta
var offsetDelta int64
if err = r.readVarInt(&offsetDelta); err != nil {
return
}
offset = r.header.firstOffset + offsetDelta
if err = r.runFunc(key); err != nil {
return
}
if err = r.runFunc(val); err != nil {
return
}
var headerCount int64
if err = r.readVarInt(&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.markRead()
return
}
func (r *messageSetReader) discardBytes() (err error) {
r.remain, err = discardBytes(r.reader, r.remain)
return
}
func (r *messageSetReader) discardN(sz int) (err error) {
r.remain, err = discardN(r.reader, r.remain, sz)
return
}
func (r *messageSetReader) markRead() {
if r.count == 0 {
panic("markRead: negative count")
}
r.count--
r.unwindStack()
r.log("Mark read remain=%d", r.remain)
}
func (r *messageSetReader) unwindStack() {
for r.count == 0 {
if r.remain == 0 {
if r.parent != nil {
r.log("Popped reader stack")
r.readerStack = r.parent
continue
}
}
break
}
}
func (r *messageSetReader) readMessageHeader(header *Header) (err error) {
var keyLen int64
if err = r.readVarInt(&keyLen); err != nil {
return
}
if header.Key, err = r.readNewString(int(keyLen)); err != nil {
return
}
var valLen int64
if err = r.readVarInt(&valLen); err != nil {
return
}
if header.Value, err = r.readNewBytes(int(valLen)); err != nil {
return
}
return nil
}
func (r *messageSetReader) runFunc(rbFunc readBytesFunc) (err error) {
var length int64
if err = r.readVarInt(&length); err != nil {
return
}
if r.remain, err = rbFunc(r.reader, r.remain, int(length)); err != nil {
return
}
return
}
func (r *messageSetReader) readHeader() (err error) {
if r.count > 0 {
// currently reading a set of messages, no need to read a header until they are exhausted.
return
}
r.header = messagesHeader{}
if err = r.readInt64(&r.header.firstOffset); err != nil {
return
}
if err = r.readInt32(&r.header.length); err != nil {
return
}
var crcOrLeaderEpoch int32
if err = r.readInt32(&crcOrLeaderEpoch); err != nil {
return
}
if err = r.readInt8(&r.header.magic); err != nil {
return
}
switch r.header.magic {
case 0:
r.header.crc = crcOrLeaderEpoch
if err = r.readInt8(&r.header.v1.attributes); err != nil {
return
}
r.count = 1
r.log("Read v0 header with offset=%d len=%d magic=%d attributes=%d", r.header.firstOffset, r.header.length, r.header.magic, r.header.v1.attributes)
case 1:
r.header.crc = crcOrLeaderEpoch
if err = r.readInt8(&r.header.v1.attributes); err != nil {
return
}
if err = r.readInt64(&r.header.v1.timestamp); err != nil {
return
}
r.count = 1
r.log("Read v1 header with remain=%d offset=%d magic=%d and attributes=%d", r.remain, r.header.firstOffset, r.header.magic, r.header.v1.attributes)
case 2:
r.header.v2.leaderEpoch = crcOrLeaderEpoch
if err = r.readInt32(&r.header.crc); err != nil {
return
}
if err = r.readInt16(&r.header.v2.attributes); err != nil {
return
}
if err = r.readInt32(&r.header.v2.lastOffsetDelta); err != nil {
return
}
if err = r.readInt64(&r.header.v2.firstTimestamp); err != nil {
return
}
if err = r.readInt64(&r.header.v2.lastTimestamp); err != nil {
return
}
if err = r.readInt64(&r.header.v2.producerID); err != nil {
return
}
if err = r.readInt16(&r.header.v2.producerEpoch); err != nil {
return
}
if err = r.readInt32(&r.header.v2.baseSequence); err != nil {
return
}
if err = r.readInt32(&r.header.v2.count); err != nil {
return
}
r.count = int(r.header.v2.count)
r.log("Read v2 header with count=%d offset=%d len=%d magic=%d attributes=%d", r.count, r.header.firstOffset, r.header.length, r.header.magic, r.header.v2.attributes)
default:
err = r.header.badMagic()
return
}
return
}
func (r *messageSetReader) readNewBytes(len int) (res []byte, err error) {
res, r.remain, err = readNewBytes(r.reader, r.remain, len)
return
}
func (r *messageSetReader) readNewString(len int) (res string, err error) {
res, r.remain, err = readNewString(r.reader, r.remain, len)
return
}
func (r *messageSetReader) readInt8(val *int8) (err error) {
r.remain, err = readInt8(r.reader, r.remain, val)
return
}
func (r *messageSetReader) readInt16(val *int16) (err error) {
r.remain, err = readInt16(r.reader, r.remain, val)
return
}
func (r *messageSetReader) readInt32(val *int32) (err error) {
r.remain, err = readInt32(r.reader, r.remain, val)
return
}
func (r *messageSetReader) readInt64(val *int64) (err error) {
r.remain, err = readInt64(r.reader, r.remain, val)
return
}
func (r *messageSetReader) readVarInt(val *int64) (err error) {
r.remain, err = readVarInt(r.reader, r.remain, val)
return
}
func (r *messageSetReader) readBytesWith(fn readBytesFunc) (err error) {
r.remain, err = readBytesWith(r.reader, r.remain, fn)
return
}
func (r *messageSetReader) log(msg string, args ...interface{}) {
if r.debug {
log.Printf("[DEBUG] "+msg, args...)
}
}
func (r *messageSetReader) dumpHex(msg string) {
if r.debug {
buf := bytes.Buffer{}
io.Copy(&buf, r.reader)
bs := buf.Bytes()
r.log(fmt.Sprintf("Hex dump: %s (%d bytes)\n%s", msg, len(bs), hex.Dump(bs)))
r.reader = bufio.NewReader(bytes.NewReader(bs))
}
}
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
}