Description
My Colleague Weichu already sent out a mail to kafka user mailing list regarding this issue, but we think it's worth having a ticket tracking it.
We are using Kafka Streams with exactly-once enabled on a Kafka cluster for
a while.
Recently we found that the size of __consumer_offsets partitions grew huge.
Some partition went over 30G. This caused Kafka to take quite long to load
"__consumer_offsets" topic on startup (it loads the topic in order to
become group coordinator).
We dumped the __consumer_offsets segments and found that while normal
offset commits are nicely compacted, transaction records (COMMIT, etc) are
all preserved. Looks like that since these messages don't have a key, the
LogCleaner is keeping them all:
----------
$ bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files
..../00000000003484332061.log --key-decoder-class
kafka.serializer.StringDecoder 2>/dev/null | cat -v | head
Dumping 00000000003484332061.log
Starting offset: 3484332061
offset: 3484332089 position: 549 CreateTime: 1556003706952 isvalid: true
keysize: 4 valuesize: 6 magic: 2 compresscodec: NONE producerId: 1006
producerEpoch: 2530 sequence: -1 isTransactional: true headerKeys: []
endTxnMarker: COMMIT coordinatorEpoch: 81
offset: 3484332090 position: 627 CreateTime: 1556003706952 isvalid: true
keysize: 4 valuesize: 6 magic: 2 compresscodec: NONE producerId: 4005
producerEpoch: 2520 sequence: -1 isTransactional: true headerKeys: []
endTxnMarker: COMMIT coordinatorEpoch: 84
...
----------
Streams is doing transaction commits per 100ms (commit.interval.ms=100 when
exactly-once) so the __consumer_offsets is growing really fast.
Is this (to keep all transactions) by design, or is that a bug for
LogCleaner? What would be the way to clean up the topic?
Attachments
Attachments
Issue Links
- links to