github.com/deanMdreon/kafka-go@v0.4.32/batch.go (about)

     1  package kafka
     2  
     3  import (
     4  	"bufio"
     5  	"io"
     6  	"sync"
     7  	"time"
     8  )
     9  
    10  // A Batch is an iterator over a sequence of messages fetched from a kafka
    11  // server.
    12  //
    13  // Batches are created by calling (*Conn).ReadBatch. They hold a internal lock
    14  // on the connection, which is released when the batch is closed. Failing to
    15  // call a batch's Close method will likely result in a dead-lock when trying to
    16  // use the connection.
    17  //
    18  // Batches are safe to use concurrently from multiple goroutines.
    19  type Batch struct {
    20  	mutex         sync.Mutex
    21  	conn          *Conn
    22  	lock          *sync.Mutex
    23  	msgs          *messageSetReader
    24  	deadline      time.Time
    25  	throttle      time.Duration
    26  	topic         string
    27  	partition     int
    28  	offset        int64
    29  	highWaterMark int64
    30  	err           error
    31  	// The last offset in the batch.
    32  	//
    33  	// We use lastOffset to skip offsets that have been compacted away.
    34  	//
    35  	// We store lastOffset because we get lastOffset when we read a new message
    36  	// but only try to handle compaction when we receive an EOF. However, when
    37  	// we get an EOF we do not get the lastOffset. So there is a mismatch
    38  	// between when we receive it and need to use it.
    39  	lastOffset int64
    40  }
    41  
    42  // Throttle gives the throttling duration applied by the kafka server on the
    43  // connection.
    44  func (batch *Batch) Throttle() time.Duration {
    45  	return batch.throttle
    46  }
    47  
    48  // Watermark returns the current highest watermark in a partition.
    49  func (batch *Batch) HighWaterMark() int64 {
    50  	return batch.highWaterMark
    51  }
    52  
    53  // Partition returns the batch partition.
    54  func (batch *Batch) Partition() int {
    55  	return batch.partition
    56  }
    57  
    58  // Offset returns the offset of the next message in the batch.
    59  func (batch *Batch) Offset() int64 {
    60  	batch.mutex.Lock()
    61  	offset := batch.offset
    62  	batch.mutex.Unlock()
    63  	return offset
    64  }
    65  
    66  // Close closes the batch, releasing the connection lock and returning an error
    67  // if reading the batch failed for any reason.
    68  func (batch *Batch) Close() error {
    69  	batch.mutex.Lock()
    70  	err := batch.close()
    71  	batch.mutex.Unlock()
    72  	return err
    73  }
    74  
    75  func (batch *Batch) close() (err error) {
    76  	conn := batch.conn
    77  	lock := batch.lock
    78  
    79  	batch.conn = nil
    80  	batch.lock = nil
    81  	if batch.msgs != nil {
    82  		batch.msgs.discard()
    83  	}
    84  
    85  	if err = batch.err; err == io.EOF {
    86  		err = nil
    87  	}
    88  
    89  	if conn != nil {
    90  		conn.rdeadline.unsetConnReadDeadline()
    91  		conn.mutex.Lock()
    92  		conn.offset = batch.offset
    93  		conn.mutex.Unlock()
    94  
    95  		if err != nil {
    96  			if _, ok := err.(Error); !ok && err != io.ErrShortBuffer {
    97  				conn.Close()
    98  			}
    99  		}
   100  	}
   101  
   102  	if lock != nil {
   103  		lock.Unlock()
   104  	}
   105  
   106  	return
   107  }
   108  
   109  // Err returns a non-nil error if the batch is broken. This is the same error
   110  // that would be returned by Read, ReadMessage or Close (except in the case of
   111  // io.EOF which is never returned by Close).
   112  //
   113  // This method is useful when building retry mechanisms for (*Conn).ReadBatch,
   114  // the program can check whether the batch carried a error before attempting to
   115  // read the first message.
   116  //
   117  // Note that checking errors on a batch is optional, calling Read or ReadMessage
   118  // is always valid and can be used to either read a message or an error in cases
   119  // where that's convenient.
   120  func (batch *Batch) Err() error { return batch.err }
   121  
   122  // Read reads the value of the next message from the batch into b, returning the
   123  // number of bytes read, or an error if the next message couldn't be read.
   124  //
   125  // If an error is returned the batch cannot be used anymore and calling Read
   126  // again will keep returning that error. All errors except io.EOF (indicating
   127  // that the program consumed all messages from the batch) are also returned by
   128  // Close.
   129  //
   130  // The method fails with io.ErrShortBuffer if the buffer passed as argument is
   131  // too small to hold the message value.
   132  func (batch *Batch) Read(b []byte) (int, error) {
   133  	n := 0
   134  
   135  	batch.mutex.Lock()
   136  	offset := batch.offset
   137  
   138  	_, _, _, err := batch.readMessage(
   139  		func(r *bufio.Reader, size int, nbytes int) (int, error) {
   140  			if nbytes < 0 {
   141  				return size, nil
   142  			}
   143  			return discardN(r, size, nbytes)
   144  		},
   145  		func(r *bufio.Reader, size int, nbytes int) (int, error) {
   146  			if nbytes < 0 {
   147  				return size, nil
   148  			}
   149  			// make sure there are enough bytes for the message value.  return
   150  			// errShortRead if the message is truncated.
   151  			if nbytes > size {
   152  				return size, errShortRead
   153  			}
   154  			n = nbytes // return value
   155  			if nbytes > cap(b) {
   156  				nbytes = cap(b)
   157  			}
   158  			if nbytes > len(b) {
   159  				b = b[:nbytes]
   160  			}
   161  			nbytes, err := io.ReadFull(r, b[:nbytes])
   162  			if err != nil {
   163  				return size - nbytes, err
   164  			}
   165  			return discardN(r, size-nbytes, n-nbytes)
   166  		},
   167  	)
   168  
   169  	if err == nil && n > len(b) {
   170  		n, err = len(b), io.ErrShortBuffer
   171  		batch.err = io.ErrShortBuffer
   172  		batch.offset = offset // rollback
   173  	}
   174  
   175  	batch.mutex.Unlock()
   176  	return n, err
   177  }
   178  
   179  // ReadMessage reads and return the next message from the batch.
   180  //
   181  // Because this method allocate memory buffers for the message key and value
   182  // it is less memory-efficient than Read, but has the advantage of never
   183  // failing with io.ErrShortBuffer.
   184  func (batch *Batch) ReadMessage() (Message, error) {
   185  	msg := Message{}
   186  	batch.mutex.Lock()
   187  
   188  	var offset, timestamp int64
   189  	var headers []Header
   190  	var err error
   191  
   192  	offset, timestamp, headers, err = batch.readMessage(
   193  		func(r *bufio.Reader, size int, nbytes int) (remain int, err error) {
   194  			msg.Key, remain, err = readNewBytes(r, size, nbytes)
   195  			return
   196  		},
   197  		func(r *bufio.Reader, size int, nbytes int) (remain int, err error) {
   198  			msg.Value, remain, err = readNewBytes(r, size, nbytes)
   199  			return
   200  		},
   201  	)
   202  	// A batch may start before the requested offset so skip messages
   203  	// until the requested offset is reached.
   204  	for batch.conn != nil && offset < batch.conn.offset {
   205  		if err != nil {
   206  			break
   207  		}
   208  		offset, timestamp, headers, err = batch.readMessage(
   209  			func(r *bufio.Reader, size int, nbytes int) (remain int, err error) {
   210  				msg.Key, remain, err = readNewBytes(r, size, nbytes)
   211  				return
   212  			},
   213  			func(r *bufio.Reader, size int, nbytes int) (remain int, err error) {
   214  				msg.Value, remain, err = readNewBytes(r, size, nbytes)
   215  				return
   216  			},
   217  		)
   218  	}
   219  
   220  	batch.mutex.Unlock()
   221  	msg.Topic = batch.topic
   222  	msg.Partition = batch.partition
   223  	msg.Offset = offset
   224  	msg.HighWaterMark = batch.highWaterMark
   225  	msg.Time = makeTime(timestamp)
   226  	msg.Headers = headers
   227  
   228  	return msg, err
   229  }
   230  
   231  func (batch *Batch) readMessage(
   232  	key func(*bufio.Reader, int, int) (int, error),
   233  	val func(*bufio.Reader, int, int) (int, error),
   234  ) (offset int64, timestamp int64, headers []Header, err error) {
   235  	if err = batch.err; err != nil {
   236  		return
   237  	}
   238  
   239  	var lastOffset int64
   240  	offset, lastOffset, timestamp, headers, err = batch.msgs.readMessage(batch.offset, key, val)
   241  	switch err {
   242  	case nil:
   243  		batch.offset = offset + 1
   244  		batch.lastOffset = lastOffset
   245  	case errShortRead:
   246  		// As an "optimization" kafka truncates the returned response after
   247  		// producing MaxBytes, which could then cause the code to return
   248  		// errShortRead.
   249  		err = batch.msgs.discard()
   250  		switch {
   251  		case err != nil:
   252  			// Since io.EOF is used by the batch to indicate that there is are
   253  			// no more messages to consume, it is crucial that any io.EOF errors
   254  			// on the underlying connection are repackaged.  Otherwise, the
   255  			// caller can't tell the difference between a batch that was fully
   256  			// consumed or a batch whose connection is in an error state.
   257  			batch.err = dontExpectEOF(err)
   258  		case batch.msgs.remaining() == 0:
   259  			// Because we use the adjusted deadline we could end up returning
   260  			// before the actual deadline occurred. This is necessary otherwise
   261  			// timing out the connection for real could end up leaving it in an
   262  			// unpredictable state, which would require closing it.
   263  			// This design decision was made to maximize the chances of keeping
   264  			// the connection open, the trade off being to lose precision on the
   265  			// read deadline management.
   266  			err = checkTimeoutErr(batch.deadline)
   267  			batch.err = err
   268  
   269  			// Checks the following:
   270  			// - `batch.err` for a "success" from the previous timeout check
   271  			// - `batch.msgs.lengthRemain` to ensure that this EOF is not due
   272  			//   to MaxBytes truncation
   273  			// - `batch.lastOffset` to ensure that the message format contains
   274  			//   `lastOffset`
   275  			if batch.err == io.EOF && batch.msgs.lengthRemain <= 0 && batch.lastOffset != -1 {
   276  				// Log compaction can create batches that end with compacted
   277  				// records so the normal strategy that increments the "next"
   278  				// offset as records are read doesn't work as the compacted
   279  				// records are "missing" and never get "read".
   280  				//
   281  				// In order to reliably reach the next non-compacted offset we
   282  				// jump past the saved lastOffset.
   283  				batch.offset = batch.lastOffset + 1
   284  			}
   285  		}
   286  	default:
   287  		// Since io.EOF is used by the batch to indicate that there is are
   288  		// no more messages to consume, it is crucial that any io.EOF errors
   289  		// on the underlying connection are repackaged.  Otherwise, the
   290  		// caller can't tell the difference between a batch that was fully
   291  		// consumed or a batch whose connection is in an error state.
   292  		batch.err = dontExpectEOF(err)
   293  	}
   294  
   295  	return
   296  }
   297  
   298  func checkTimeoutErr(deadline time.Time) (err error) {
   299  	if !deadline.IsZero() && time.Now().After(deadline) {
   300  		err = RequestTimedOut
   301  	} else {
   302  		err = io.EOF
   303  	}
   304  	return
   305  }