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