[
https://issues.apache.org/jira/browse/KAFKA-8335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16872013#comment-16872013
]
Francisco Juan commented on KAFKA-8335:
---------------------------------------
I'm seeing this problem only in some partitions, check below partitions sorted
by disk size:
{code:java}
| Topic | P | Disk (MB) |
| __consumer_offsets | 6 | 23599 |
| __consumer_offsets | 8 | 12285 |
| __consumer_offsets | 43 | 4725 |
| __consumer_offsets | 23 | 875.86 |
| __consumer_offsets | 31 | 443.35 |
| __consumer_offsets | 16 | 433.02 |
| __consumer_offsets | 18 | 418.52 |
| __consumer_offsets | 11 | 385.40 |
| __consumer_offsets | 14 | 308.03 |
| __consumer_offsets | 47 | 268.80 |
| __consumer_offsets | 13 | 254.77 |
| __consumer_offsets | 29 | 250.02 |
| __consumer_offsets | 30 | 233.38 |
| __consumer_offsets | 12 | 148.13 |
| __consumer_offsets | 46 | 116.77 |
| __consumer_offsets | 26 | 114.36 |
| __consumer_offsets | 3 | 101.09 |
...
| __consumer_offsets | 24 | 7.09 |{code}
Our topic is using 50 partitions.
> Log cleaner skips Transactional mark and batch record, causing unlimited
> growth of __consumer_offsets
> -----------------------------------------------------------------------------------------------------
>
> Key: KAFKA-8335
> URL: https://issues.apache.org/jira/browse/KAFKA-8335
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 2.2.0
> Reporter: Boquan Tang
> Assignee: Jason Gustafson
> Priority: Major
> Fix For: 2.0.2, 2.1.2, 2.2.1
>
> Attachments: seg_april_25.zip, segment.zip
>
>
> 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?
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)