The old segment isn't rolled, a new one is. eg, take this partition directory:
drwxr-xr-x 2 kafka kafka 4096 Jan 1 10:31 . drwxr-xr-x 1483 kafka kafka 94208 Jan 3 02:11 .. -rw-r--r-- 1 kafka kafka 15504 Jan 1 10:31 00000000000004050355.index -rw-r--r-- 1 kafka kafka 8210576 Jan 1 10:31 00000000000004050355.log -rw-r--r-- 1 kafka kafka 23268 Jan 1 10:31 00000000000004050355.timeindex -rw-r--r-- 1 kafka kafka 10485760 Jan 3 02:07 00000000000004100746.index -rw-r--r-- 1 kafka kafka 1928286 Jan 3 02:11 00000000000004100746.log -rw-r--r-- 1 kafka kafka 10 Jan 1 10:31 00000000000004100746.snapshot -rw-r--r-- 1 kafka kafka 10485756 Jan 3 02:07 00000000000004100746.timeindex -rw-r--r-- 1 kafka kafka 125 Dec 30 18:19 leader-epoch-checkpoint kafka-run-class.sh kafka.tools.DumpLogSegments --print-data-log --files 00000000000004050355.log 2>/dev/null | (head -n3 && tail -n1) Dumping 00000000000004050355.log Starting offset: 4050355 baseOffset: 4050355 lastOffset: 4050355 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 165 isTransactional: false position: 0 CreateTime: 1514197881429 isvalid: true size: 163 magic: 2 compresscodec: NONE crc: 3038770522 baseOffset: 4100745 lastOffset: 4100745 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 174 isTransactional: false position: 8210414 CreateTime: 1514802680844 isvalid: true size: 162 magic: 2 compresscodec: NONE crc: 2436714595 That is, the first and last message in that segment. In the past there was maybe just the *00000000000004050355.log* segment. It started with offset *4050355*. After offset *4100745* a new segment was rolled, *00000000000004100746.log*, and new messages starting from offset *4100746* went in there. Consumers aren't effected. Maybe you're curious what happens when a segment is removed -- If there's size/time based retention of a topic, old segments may later be deleted. Say the time comes for *00000000000004050355.log* to be deleted, if a (probably stalled, or extremely lagging) consumer's current offset is between *4050355* and *4100745* then it wouldn't be able to fetch the next message anymore. Then the client will receive a corresponding error and its behavior will be determined by how *auto.offset.reset* config is defined. [earliest, latest, none], with the default being latest. documented here: https://kafka.apache.org/documentation.html Look over the various log.retention.* settings as well. On Tue, Jan 2, 2018 at 9:00 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > Can someone let me know how does consumer behaves when Kafka rolls the old > log file (e.g. i.e. INFO Rolled new log segment for 'topic-{partition}') > while the consumer is still consuming the log segment ? >