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