github.com/deanMdreon/kafka-go@v0.4.32/message_reader.go (about) 1 package kafka 2 3 import ( 4 "bufio" 5 "bytes" 6 "encoding/hex" 7 "fmt" 8 "io" 9 "log" 10 ) 11 12 type readBytesFunc func(*bufio.Reader, int, int) (int, error) 13 14 // messageSetReader processes the messages encoded into a fetch response. 15 // The response may contain a mix of Record Batches (newer format) and Messages 16 // (older format). 17 type messageSetReader struct { 18 *readerStack // used for decompressing compressed messages and record batches 19 empty bool // if true, short circuits messageSetReader methods 20 debug bool // enable debug log messages 21 // How many bytes are expected to remain in the response. 22 // 23 // This is used to detect truncation of the response. 24 lengthRemain int 25 } 26 27 type readerStack struct { 28 reader *bufio.Reader 29 remain int 30 base int64 31 parent *readerStack 32 count int // how many messages left in the current message set 33 header messagesHeader // the current header for a subset of messages within the set. 34 } 35 36 // messagesHeader describes a set of records. there may be many messagesHeader's in a message set. 37 type messagesHeader struct { 38 firstOffset int64 39 length int32 40 crc int32 41 magic int8 42 // v1 composes attributes specific to v0 and v1 message headers 43 v1 struct { 44 attributes int8 45 timestamp int64 46 } 47 // v2 composes attributes specific to v2 message headers 48 v2 struct { 49 leaderEpoch int32 50 attributes int16 51 lastOffsetDelta int32 52 firstTimestamp int64 53 lastTimestamp int64 54 producerID int64 55 producerEpoch int16 56 baseSequence int32 57 count int32 58 } 59 } 60 61 func (h messagesHeader) compression() (codec CompressionCodec, err error) { 62 const compressionCodecMask = 0x07 63 var code int8 64 switch h.magic { 65 case 0, 1: 66 code = h.v1.attributes & compressionCodecMask 67 case 2: 68 code = int8(h.v2.attributes & compressionCodecMask) 69 default: 70 err = h.badMagic() 71 return 72 } 73 if code != 0 { 74 codec, err = resolveCodec(code) 75 } 76 return 77 } 78 79 func (h messagesHeader) badMagic() error { 80 return fmt.Errorf("unsupported magic byte %d in header", h.magic) 81 } 82 83 func newMessageSetReader(reader *bufio.Reader, remain int) (*messageSetReader, error) { 84 res := &messageSetReader{ 85 readerStack: &readerStack{ 86 reader: reader, 87 remain: remain, 88 }, 89 } 90 err := res.readHeader() 91 return res, err 92 } 93 94 func (r *messageSetReader) remaining() (remain int) { 95 if r.empty { 96 return 0 97 } 98 for s := r.readerStack; s != nil; s = s.parent { 99 remain += s.remain 100 } 101 return 102 } 103 104 func (r *messageSetReader) discard() (err error) { 105 switch { 106 case r.empty: 107 case r.readerStack == nil: 108 default: 109 // rewind up to the top-most reader b/c it's the only one that's doing 110 // actual i/o. the rest are byte buffers that have been pushed on the stack 111 // while reading compressed message sets. 112 for r.parent != nil { 113 r.readerStack = r.parent 114 } 115 err = r.discardN(r.remain) 116 } 117 return 118 } 119 120 func (r *messageSetReader) readMessage(min int64, key readBytesFunc, val readBytesFunc) ( 121 offset int64, lastOffset int64, timestamp int64, headers []Header, err error) { 122 123 if r.empty { 124 err = RequestTimedOut 125 return 126 } 127 if err = r.readHeader(); err != nil { 128 return 129 } 130 switch r.header.magic { 131 case 0, 1: 132 offset, timestamp, headers, err = r.readMessageV1(min, key, val) 133 // Set an invalid value so that it can be ignored 134 lastOffset = -1 135 case 2: 136 offset, lastOffset, timestamp, headers, err = r.readMessageV2(min, key, val) 137 default: 138 err = r.header.badMagic() 139 } 140 return 141 } 142 143 func (r *messageSetReader) readMessageV1(min int64, key readBytesFunc, val readBytesFunc) ( 144 offset int64, timestamp int64, headers []Header, err error) { 145 146 for r.readerStack != nil { 147 if r.remain == 0 { 148 r.readerStack = r.parent 149 continue 150 } 151 if err = r.readHeader(); err != nil { 152 return 153 } 154 offset = r.header.firstOffset 155 timestamp = r.header.v1.timestamp 156 var codec CompressionCodec 157 if codec, err = r.header.compression(); err != nil { 158 return 159 } 160 r.log("Reading with codec=%T", codec) 161 if codec != nil { 162 // discard next four bytes...will be -1 to indicate null key 163 if err = r.discardN(4); err != nil { 164 return 165 } 166 // read and decompress the contained message set. 167 var decompressed bytes.Buffer 168 if err = r.readBytesWith(func(r *bufio.Reader, sz int, n int) (remain int, err error) { 169 // x4 as a guess that the average compression ratio is near 75% 170 decompressed.Grow(4 * n) 171 limitReader := io.LimitedReader{R: r, N: int64(n)} 172 codecReader := codec.NewReader(&limitReader) 173 _, err = decompressed.ReadFrom(codecReader) 174 remain = sz - (n - int(limitReader.N)) 175 codecReader.Close() 176 return 177 }); err != nil { 178 return 179 } 180 181 // the compressed message's offset will be equal to the offset of 182 // the last message in the set. within the compressed set, the 183 // offsets will be relative, so we have to scan through them to 184 // get the base offset. for example, if there are four compressed 185 // messages at offsets 10-13, then the container message will have 186 // offset 13 and the contained messages will be 0,1,2,3. the base 187 // offset for the container, then is 13-3=10. 188 if offset, err = extractOffset(offset, decompressed.Bytes()); err != nil { 189 return 190 } 191 192 // mark the outer message as being read 193 r.markRead() 194 195 // then push the decompressed bytes onto the stack. 196 r.readerStack = &readerStack{ 197 // Allocate a buffer of size 0, which gets capped at 16 bytes 198 // by the bufio package. We are already reading buffered data 199 // here, no need to reserve another 4KB buffer. 200 reader: bufio.NewReaderSize(&decompressed, 0), 201 remain: decompressed.Len(), 202 base: offset, 203 parent: r.readerStack, 204 } 205 continue 206 } 207 208 // adjust the offset in case we're reading compressed messages. the 209 // base will be zero otherwise. 210 offset += r.base 211 212 // When the messages are compressed kafka may return messages at an 213 // earlier offset than the one that was requested, it's the client's 214 // responsibility to ignore those. 215 // 216 // At this point, the message header has been read, so discarding 217 // the rest of the message means we have to discard the key, and then 218 // the value. Each of those are preceded by a 4-byte length. Discarding 219 // them is then reading that length variable and then discarding that 220 // amount. 221 if offset < min { 222 // discard the key 223 if err = r.discardBytes(); err != nil { 224 return 225 } 226 // discard the value 227 if err = r.discardBytes(); err != nil { 228 return 229 } 230 // since we have fully consumed the message, mark as read 231 r.markRead() 232 continue 233 } 234 if err = r.readBytesWith(key); err != nil { 235 return 236 } 237 if err = r.readBytesWith(val); err != nil { 238 return 239 } 240 r.markRead() 241 return 242 } 243 err = errShortRead 244 return 245 } 246 247 func (r *messageSetReader) readMessageV2(_ int64, key readBytesFunc, val readBytesFunc) ( 248 offset int64, lastOffset int64, timestamp int64, headers []Header, err error) { 249 if err = r.readHeader(); err != nil { 250 return 251 } 252 if r.count == int(r.header.v2.count) { // first time reading this set, so check for compression headers. 253 var codec CompressionCodec 254 if codec, err = r.header.compression(); err != nil { 255 return 256 } 257 if codec != nil { 258 batchRemain := int(r.header.length - 49) // TODO: document this magic number 259 if batchRemain > r.remain { 260 err = errShortRead 261 return 262 } 263 if batchRemain < 0 { 264 err = fmt.Errorf("batch remain < 0 (%d)", batchRemain) 265 return 266 } 267 var decompressed bytes.Buffer 268 decompressed.Grow(4 * batchRemain) 269 limitReader := io.LimitedReader{R: r.reader, N: int64(batchRemain)} 270 codecReader := codec.NewReader(&limitReader) 271 _, err = decompressed.ReadFrom(codecReader) 272 codecReader.Close() 273 if err != nil { 274 return 275 } 276 r.remain -= batchRemain - int(limitReader.N) 277 r.readerStack = &readerStack{ 278 reader: bufio.NewReaderSize(&decompressed, 0), // the new stack reads from the decompressed buffer 279 remain: decompressed.Len(), 280 base: -1, // base is unused here 281 parent: r.readerStack, 282 header: r.header, 283 count: r.count, 284 } 285 // all of the messages in this set are in the decompressed set just pushed onto the reader 286 // stack. here we set the parent count to 0 so that when the child set is exhausted, the 287 // reader will then try to read the header of the next message set 288 r.readerStack.parent.count = 0 289 } 290 } 291 remainBefore := r.remain 292 var length int64 293 if err = r.readVarInt(&length); err != nil { 294 return 295 } 296 lengthOfLength := remainBefore - r.remain 297 var attrs int8 298 if err = r.readInt8(&attrs); err != nil { 299 return 300 } 301 var timestampDelta int64 302 if err = r.readVarInt(×tampDelta); err != nil { 303 return 304 } 305 timestamp = r.header.v2.firstTimestamp + timestampDelta 306 var offsetDelta int64 307 if err = r.readVarInt(&offsetDelta); err != nil { 308 return 309 } 310 offset = r.header.firstOffset + offsetDelta 311 if err = r.runFunc(key); err != nil { 312 return 313 } 314 if err = r.runFunc(val); err != nil { 315 return 316 } 317 var headerCount int64 318 if err = r.readVarInt(&headerCount); err != nil { 319 return 320 } 321 headers = make([]Header, headerCount) 322 for i := 0; i < int(headerCount); i++ { 323 if err = r.readMessageHeader(&headers[i]); err != nil { 324 return 325 } 326 } 327 lastOffset = r.header.firstOffset + int64(r.header.v2.lastOffsetDelta) 328 r.lengthRemain -= int(length) + lengthOfLength 329 r.markRead() 330 return 331 } 332 333 func (r *messageSetReader) discardBytes() (err error) { 334 r.remain, err = discardBytes(r.reader, r.remain) 335 return 336 } 337 338 func (r *messageSetReader) discardN(sz int) (err error) { 339 r.remain, err = discardN(r.reader, r.remain, sz) 340 return 341 } 342 343 func (r *messageSetReader) markRead() { 344 if r.count == 0 { 345 panic("markRead: negative count") 346 } 347 r.count-- 348 r.unwindStack() 349 r.log("Mark read remain=%d", r.remain) 350 } 351 352 func (r *messageSetReader) unwindStack() { 353 for r.count == 0 { 354 if r.remain == 0 { 355 if r.parent != nil { 356 r.log("Popped reader stack") 357 r.readerStack = r.parent 358 continue 359 } 360 } 361 break 362 } 363 } 364 365 func (r *messageSetReader) readMessageHeader(header *Header) (err error) { 366 var keyLen int64 367 if err = r.readVarInt(&keyLen); err != nil { 368 return 369 } 370 if header.Key, err = r.readNewString(int(keyLen)); err != nil { 371 return 372 } 373 var valLen int64 374 if err = r.readVarInt(&valLen); err != nil { 375 return 376 } 377 if header.Value, err = r.readNewBytes(int(valLen)); err != nil { 378 return 379 } 380 return nil 381 } 382 383 func (r *messageSetReader) runFunc(rbFunc readBytesFunc) (err error) { 384 var length int64 385 if err = r.readVarInt(&length); err != nil { 386 return 387 } 388 if r.remain, err = rbFunc(r.reader, r.remain, int(length)); err != nil { 389 return 390 } 391 return 392 } 393 394 func (r *messageSetReader) readHeader() (err error) { 395 if r.count > 0 { 396 // currently reading a set of messages, no need to read a header until they are exhausted. 397 return 398 } 399 r.header = messagesHeader{} 400 if err = r.readInt64(&r.header.firstOffset); err != nil { 401 return 402 } 403 if err = r.readInt32(&r.header.length); err != nil { 404 return 405 } 406 var crcOrLeaderEpoch int32 407 if err = r.readInt32(&crcOrLeaderEpoch); err != nil { 408 return 409 } 410 if err = r.readInt8(&r.header.magic); err != nil { 411 return 412 } 413 switch r.header.magic { 414 case 0: 415 r.header.crc = crcOrLeaderEpoch 416 if err = r.readInt8(&r.header.v1.attributes); err != nil { 417 return 418 } 419 r.count = 1 420 // Set arbitrary non-zero length so that we always assume the 421 // message is truncated since bytes remain. 422 r.lengthRemain = 1 423 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) 424 case 1: 425 r.header.crc = crcOrLeaderEpoch 426 if err = r.readInt8(&r.header.v1.attributes); err != nil { 427 return 428 } 429 if err = r.readInt64(&r.header.v1.timestamp); err != nil { 430 return 431 } 432 r.count = 1 433 // Set arbitrary non-zero length so that we always assume the 434 // message is truncated since bytes remain. 435 r.lengthRemain = 1 436 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) 437 case 2: 438 r.header.v2.leaderEpoch = crcOrLeaderEpoch 439 if err = r.readInt32(&r.header.crc); err != nil { 440 return 441 } 442 if err = r.readInt16(&r.header.v2.attributes); err != nil { 443 return 444 } 445 if err = r.readInt32(&r.header.v2.lastOffsetDelta); err != nil { 446 return 447 } 448 if err = r.readInt64(&r.header.v2.firstTimestamp); err != nil { 449 return 450 } 451 if err = r.readInt64(&r.header.v2.lastTimestamp); err != nil { 452 return 453 } 454 if err = r.readInt64(&r.header.v2.producerID); err != nil { 455 return 456 } 457 if err = r.readInt16(&r.header.v2.producerEpoch); err != nil { 458 return 459 } 460 if err = r.readInt32(&r.header.v2.baseSequence); err != nil { 461 return 462 } 463 if err = r.readInt32(&r.header.v2.count); err != nil { 464 return 465 } 466 r.count = int(r.header.v2.count) 467 // Subtracts the header bytes from the length 468 r.lengthRemain = int(r.header.length) - 49 469 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) 470 default: 471 err = r.header.badMagic() 472 return 473 } 474 return 475 } 476 477 func (r *messageSetReader) readNewBytes(len int) (res []byte, err error) { 478 res, r.remain, err = readNewBytes(r.reader, r.remain, len) 479 return 480 } 481 482 func (r *messageSetReader) readNewString(len int) (res string, err error) { 483 res, r.remain, err = readNewString(r.reader, r.remain, len) 484 return 485 } 486 487 func (r *messageSetReader) readInt8(val *int8) (err error) { 488 r.remain, err = readInt8(r.reader, r.remain, val) 489 return 490 } 491 492 func (r *messageSetReader) readInt16(val *int16) (err error) { 493 r.remain, err = readInt16(r.reader, r.remain, val) 494 return 495 } 496 497 func (r *messageSetReader) readInt32(val *int32) (err error) { 498 r.remain, err = readInt32(r.reader, r.remain, val) 499 return 500 } 501 502 func (r *messageSetReader) readInt64(val *int64) (err error) { 503 r.remain, err = readInt64(r.reader, r.remain, val) 504 return 505 } 506 507 func (r *messageSetReader) readVarInt(val *int64) (err error) { 508 r.remain, err = readVarInt(r.reader, r.remain, val) 509 return 510 } 511 512 func (r *messageSetReader) readBytesWith(fn readBytesFunc) (err error) { 513 r.remain, err = readBytesWith(r.reader, r.remain, fn) 514 return 515 } 516 517 func (r *messageSetReader) log(msg string, args ...interface{}) { 518 if r.debug { 519 log.Printf("[DEBUG] "+msg, args...) 520 } 521 } 522 523 func (r *messageSetReader) dumpHex(msg string) { 524 if r.debug { 525 buf := bytes.Buffer{} 526 io.Copy(&buf, r.reader) 527 bs := buf.Bytes() 528 r.log(fmt.Sprintf("Hex dump: %s (%d bytes)\n%s", msg, len(bs), hex.Dump(bs))) 529 r.reader = bufio.NewReader(bytes.NewReader(bs)) 530 } 531 } 532 533 func extractOffset(base int64, msgSet []byte) (offset int64, err error) { 534 r, remain := bufio.NewReader(bytes.NewReader(msgSet)), len(msgSet) 535 for remain > 0 { 536 if remain, err = readInt64(r, remain, &offset); err != nil { 537 return 538 } 539 var sz int32 540 if remain, err = readInt32(r, remain, &sz); err != nil { 541 return 542 } 543 if remain, err = discardN(r, remain, int(sz)); err != nil { 544 return 545 } 546 } 547 offset = base - offset 548 return 549 }