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