Skip to main content

Segments Not Compacted Due to Empty Transactional Record Batches

We're seeing an issue with some __consumer_offsets partitions that host
consumer groups being used in EOS-style read-process-write cycles. Although
the brokers' log cleaners are running, we are finding more and more segment
files for these partitions, for example:

```
$ ls -lh data/__consumer_offsets-40/
total 68G
-rw-r--r-- 1 root root 2.9K Mar 20 14:49 00000000000000000000.index
-rw-r--r-- 1 root root 90M Mar 20 14:49 00000000000000000000.log
-rw-r--r-- 1 root root 4.3K Mar 20 14:49 00000000000000000000.timeindex
-rw-r--r-- 1 root root 2.8K Mar 20 15:49 00000000001494007647.index
-rw-r--r-- 1 root root 87M Mar 20 15:49 00000000001494007647.log
-rw-r--r-- 1 root root 4.1K Mar 20 15:49 00000000001494007647.timeindex
-rw-r--r-- 1 root root 2.7K Mar 20 17:06 00000000001495428980.index
-rw-r--r-- 1 root root 86M Mar 20 17:06 00000000001495428980.log
-rw-r--r-- 1 root root 4.0K Mar 20 17:06 00000000001495428980.timeindex
-rw-r--r-- 1 root root 2.7K Mar 20 18:18 00000000001496852294.index
-rw-r--r-- 1 root root 86M Mar 20 18:18 00000000001496852294.log
-rw-r--r-- 1 root root 4.1K Mar 20 18:18 00000000001496852294.timeindex
-rw-r--r-- 1 root root 2.7K Mar 20 19:47 00000000001498275579.index
-rw-r--r-- 1 root root 84M Mar 20 19:47 00000000001498275579.log
-rw-r--r-- 1 root root 4.0K Mar 20 19:47 00000000001498275579.timeindex
... <snip> ...
-rw-r--r-- 1 root root 192K Apr 25 21:56 00000000002563044728.index
-rw-r--r-- 1 root root 100M Apr 25 21:56 00000000002563044728.log
-rw-r--r-- 1 root root 8.8K Apr 25 21:29 00000000002563044728.snapshot
-rw-r--r-- 1 root root 287K Apr 25 21:56 00000000002563044728.timeindex
-rw-r--r-- 1 root root 10M Apr 25 22:16 00000000002563761528.index
-rw-r--r-- 1 root root 71M Apr 25 22:16 00000000002563761528.log
-rw-r--r-- 1 root root 8.8K Apr 25 21:56 00000000002563761528.snapshot
-rw-r--r-- 1 root root 10M Apr 25 22:16 00000000002563761528.timeindex
-rw-r--r-- 1 root root 290 Mar 8 00:17 leader-epoch-checkpoint
```

Running the DumpLogSegments tool, we found these older segments contained
empty transactional record batches, followed by control batches with
commits for the producer ids from the transactional record batches. There
were no non-control records in these segments.

```
Dumping ./00000000001498275579.log
Starting offset: 1498275579
baseOffset: 1498275580 lastOffset: 1498275580 count: 0 baseSequence: 0
lastSequence: 0 producerId: 366015 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: false position: 0 CreateTime:
1553131137199 size: 61 magic: 2 compresscodec: NONE crc: 1166784824
isvalid: true
baseOffset: 1498275585 lastOffset: 1498275585 count: 0 baseSequence: 0
lastSequence: 0 producerId: 375012 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: false position: 61 CreateTime:
1553131137212 size: 61 magic: 2 compresscodec: NONE crc: 1328557806
isvalid: true
baseOffset: 1498275586 lastOffset: 1498275586 count: 1 baseSequence: -1
lastSequence: -1 producerId: 366015 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: true position: 122 CreateTime:
1553131137217 size: 78 magic: 2 compresscodec: NONE crc: 2218262672
isvalid: true
| offset: 1498275586 CreateTime: 1553131137217 keysize: 4 valuesize: 6
sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 36
baseOffset: 1498275587 lastOffset: 1498275587 count: 0 baseSequence: 0
lastSequence: 0 producerId: 366004 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: false position: 200 CreateTime:
1553131137218 size: 61 magic: 2 compresscodec: NONE crc: 2017345467
isvalid: true
baseOffset: 1498275588 lastOffset: 1498275588 count: 0 baseSequence: 0
lastSequence: 0 producerId: 457007 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: false position: 261 CreateTime:
1553131137222 size: 61 magic: 2 compresscodec: NONE crc: 1906396684
isvalid: true
baseOffset: 1498275589 lastOffset: 1498275589 count: 0 baseSequence: 0
lastSequence: 0 producerId: 500008 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: false position: 322 CreateTime:
1553131137223 size: 61 magic: 2 compresscodec: NONE crc: 3787786740
isvalid: true
baseOffset: 1498275590 lastOffset: 1498275590 count: 0 baseSequence: 0
lastSequence: 0 producerId: 364010 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: false position: 383 CreateTime:
1553131137227 size: 61 magic: 2 compresscodec: NONE crc: 1926841942
isvalid: true
baseOffset: 1498275591 lastOffset: 1498275591 count: 1 baseSequence: -1
lastSequence: -1 producerId: 375012 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: true position: 444 CreateTime:
1553131137229 size: 78 magic: 2 compresscodec: NONE crc: 3785252293
isvalid: true
| offset: 1498275591 CreateTime: 1553131137229 keysize: 4 valuesize: 6
sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 33
baseOffset: 1498275592 lastOffset: 1498275592 count: 1 baseSequence: -1
lastSequence: -1 producerId: 366004 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: true position: 522 CreateTime:
1553131137234 size: 78 magic: 2 compresscodec: NONE crc: 3124357672
isvalid: true
| offset: 1498275592 CreateTime: 1553131137234 keysize: 4 valuesize: 6
sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 36
baseOffset: 1498275593 lastOffset: 1498275593 count: 1 baseSequence: -1
lastSequence: -1 producerId: 457007 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: true position: 600 CreateTime:
1553131137238 size: 78 magic: 2 compresscodec: NONE crc: 3743563856
isvalid: true
| offset: 1498275593 CreateTime: 1553131137238 keysize: 4 valuesize: 6
sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 37
baseOffset: 1498275594 lastOffset: 1498275594 count: 1 baseSequence: -1
lastSequence: -1 producerId: 500008 producerEpoch: 10 partitionLeaderEpoch:
208 isTransactional: true isControl: true position: 678 CreateTime:
1553131137239 size: 78 magic: 2 compresscodec: NONE crc: 2697480640
isvalid: true
| offset: 1498275594 CreateTime: 1553131137239 keysize: 4 valuesize: 6
sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 45
... <snip> ...
```

Retention of empty batches was introduced with
https://issues.apache.org/jira/browse/KAFKA-5490, so the empty batches are
intentional and serve a purpose. But reading the log cleaner code, it looks
like our segments will remain uncleanable in perpetuity: unaborted
transactional batches are retained, regardless of whether there are records
in the batch, and commit records for the producer ids of those batches are
subsequently also retained. This leads us to our current situation where we
have unbounded growth on __consumer_offsets, with disastrous effects when a
consumer group coordinator moves to a new broker.

Our cluster is running Apache Kafka 1.1.1, but we have observed the same
behavior in a test 2.1.1 replica. Given the fallout of this issue, we're
surprised at being unable to find other reports, so we're wondering if
there is something unusual about our configurations or our clients' usage
patterns. Any pointers or things to look for would be appreciated!

Michael

Comments