github.com/segmentio/kafka-go@v0.4.48-0.20240318174348-3f6244eb34fd/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 83 if batch.msgs != nil { 84 batch.msgs.discard() 85 } 86 87 if batch.msgs != nil && batch.msgs.decompressed != nil { 88 releaseBuffer(batch.msgs.decompressed) 89 batch.msgs.decompressed = nil 90 } 91 92 if err = batch.err; errors.Is(batch.err, io.EOF) { 93 err = nil 94 } 95 96 if conn != nil { 97 conn.rdeadline.unsetConnReadDeadline() 98 conn.mutex.Lock() 99 conn.offset = batch.offset 100 conn.mutex.Unlock() 101 102 if err != nil { 103 var kafkaError Error 104 if !errors.As(err, &kafkaError) && !errors.Is(err, io.ErrShortBuffer) { 105 conn.Close() 106 } 107 } 108 } 109 110 if lock != nil { 111 lock.Unlock() 112 } 113 114 return 115 } 116 117 // Err returns a non-nil error if the batch is broken. This is the same error 118 // that would be returned by Read, ReadMessage or Close (except in the case of 119 // io.EOF which is never returned by Close). 120 // 121 // This method is useful when building retry mechanisms for (*Conn).ReadBatch, 122 // the program can check whether the batch carried a error before attempting to 123 // read the first message. 124 // 125 // Note that checking errors on a batch is optional, calling Read or ReadMessage 126 // is always valid and can be used to either read a message or an error in cases 127 // where that's convenient. 128 func (batch *Batch) Err() error { return batch.err } 129 130 // Read reads the value of the next message from the batch into b, returning the 131 // number of bytes read, or an error if the next message couldn't be read. 132 // 133 // If an error is returned the batch cannot be used anymore and calling Read 134 // again will keep returning that error. All errors except io.EOF (indicating 135 // that the program consumed all messages from the batch) are also returned by 136 // Close. 137 // 138 // The method fails with io.ErrShortBuffer if the buffer passed as argument is 139 // too small to hold the message value. 140 func (batch *Batch) Read(b []byte) (int, error) { 141 n := 0 142 143 batch.mutex.Lock() 144 offset := batch.offset 145 146 _, _, _, err := batch.readMessage( 147 func(r *bufio.Reader, size int, nbytes int) (int, error) { 148 if nbytes < 0 { 149 return size, nil 150 } 151 return discardN(r, size, nbytes) 152 }, 153 func(r *bufio.Reader, size int, nbytes int) (int, error) { 154 if nbytes < 0 { 155 return size, nil 156 } 157 // make sure there are enough bytes for the message value. return 158 // errShortRead if the message is truncated. 159 if nbytes > size { 160 return size, errShortRead 161 } 162 n = nbytes // return value 163 if nbytes > cap(b) { 164 nbytes = cap(b) 165 } 166 if nbytes > len(b) { 167 b = b[:nbytes] 168 } 169 nbytes, err := io.ReadFull(r, b[:nbytes]) 170 if err != nil { 171 return size - nbytes, err 172 } 173 return discardN(r, size-nbytes, n-nbytes) 174 }, 175 ) 176 177 if err == nil && n > len(b) { 178 n, err = len(b), io.ErrShortBuffer 179 batch.err = io.ErrShortBuffer 180 batch.offset = offset // rollback 181 } 182 183 batch.mutex.Unlock() 184 return n, err 185 } 186 187 // ReadMessage reads and return the next message from the batch. 188 // 189 // Because this method allocate memory buffers for the message key and value 190 // it is less memory-efficient than Read, but has the advantage of never 191 // failing with io.ErrShortBuffer. 192 func (batch *Batch) ReadMessage() (Message, error) { 193 msg := Message{} 194 batch.mutex.Lock() 195 196 var offset, timestamp int64 197 var headers []Header 198 var err error 199 200 offset, timestamp, headers, err = batch.readMessage( 201 func(r *bufio.Reader, size int, nbytes int) (remain int, err error) { 202 msg.Key, remain, err = readNewBytes(r, size, nbytes) 203 return 204 }, 205 func(r *bufio.Reader, size int, nbytes int) (remain int, err error) { 206 msg.Value, remain, err = readNewBytes(r, size, nbytes) 207 return 208 }, 209 ) 210 // A batch may start before the requested offset so skip messages 211 // until the requested offset is reached. 212 for batch.conn != nil && offset < batch.conn.offset { 213 if err != nil { 214 break 215 } 216 offset, timestamp, headers, err = batch.readMessage( 217 func(r *bufio.Reader, size int, nbytes int) (remain int, err error) { 218 msg.Key, remain, err = readNewBytes(r, size, nbytes) 219 return 220 }, 221 func(r *bufio.Reader, size int, nbytes int) (remain int, err error) { 222 msg.Value, remain, err = readNewBytes(r, size, nbytes) 223 return 224 }, 225 ) 226 } 227 228 batch.mutex.Unlock() 229 msg.Topic = batch.topic 230 msg.Partition = batch.partition 231 msg.Offset = offset 232 msg.HighWaterMark = batch.highWaterMark 233 msg.Time = makeTime(timestamp) 234 msg.Headers = headers 235 236 return msg, err 237 } 238 239 func (batch *Batch) readMessage( 240 key func(*bufio.Reader, int, int) (int, error), 241 val func(*bufio.Reader, int, int) (int, error), 242 ) (offset int64, timestamp int64, headers []Header, err error) { 243 if err = batch.err; err != nil { 244 return 245 } 246 247 var lastOffset int64 248 offset, lastOffset, timestamp, headers, err = batch.msgs.readMessage(batch.offset, key, val) 249 switch { 250 case err == nil: 251 batch.offset = offset + 1 252 batch.lastOffset = lastOffset 253 case errors.Is(err, errShortRead): 254 // As an "optimization" kafka truncates the returned response after 255 // producing MaxBytes, which could then cause the code to return 256 // errShortRead. 257 err = batch.msgs.discard() 258 switch { 259 case err != nil: 260 // Since io.EOF is used by the batch to indicate that there is are 261 // no more messages to consume, it is crucial that any io.EOF errors 262 // on the underlying connection are repackaged. Otherwise, the 263 // caller can't tell the difference between a batch that was fully 264 // consumed or a batch whose connection is in an error state. 265 batch.err = dontExpectEOF(err) 266 case batch.msgs.remaining() == 0: 267 // Because we use the adjusted deadline we could end up returning 268 // before the actual deadline occurred. This is necessary otherwise 269 // timing out the connection for real could end up leaving it in an 270 // unpredictable state, which would require closing it. 271 // This design decision was made to maximize the chances of keeping 272 // the connection open, the trade off being to lose precision on the 273 // read deadline management. 274 err = checkTimeoutErr(batch.deadline) 275 batch.err = err 276 277 // Checks the following: 278 // - `batch.err` for a "success" from the previous timeout check 279 // - `batch.msgs.lengthRemain` to ensure that this EOF is not due 280 // to MaxBytes truncation 281 // - `batch.lastOffset` to ensure that the message format contains 282 // `lastOffset` 283 if errors.Is(batch.err, io.EOF) && batch.msgs.lengthRemain == 0 && batch.lastOffset != -1 { 284 // Log compaction can create batches that end with compacted 285 // records so the normal strategy that increments the "next" 286 // offset as records are read doesn't work as the compacted 287 // records are "missing" and never get "read". 288 // 289 // In order to reliably reach the next non-compacted offset we 290 // jump past the saved lastOffset. 291 batch.offset = batch.lastOffset + 1 292 } 293 } 294 default: 295 // Since io.EOF is used by the batch to indicate that there is are 296 // no more messages to consume, it is crucial that any io.EOF errors 297 // on the underlying connection are repackaged. Otherwise, the 298 // caller can't tell the difference between a batch that was fully 299 // consumed or a batch whose connection is in an error state. 300 batch.err = dontExpectEOF(err) 301 } 302 303 return 304 } 305 306 func checkTimeoutErr(deadline time.Time) (err error) { 307 if !deadline.IsZero() && time.Now().After(deadline) { 308 err = RequestTimedOut 309 } else { 310 err = io.EOF 311 } 312 return 313 }