[ https://issues.apache.org/jira/browse/KAFKA-8335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16836075#comment-16836075 ]
Boquan Tang commented on KAFKA-8335: ------------------------------------ Hi [~hachikuji] thanks for replying. As Weichu commented we have log.cleaner.delete.retention.ms = 86400000 which is one day. To better illustrate the suspected issue, I uploaded the full segment from April 25 [^seg_april_25.zip], which is 2 weeks ago from the time it was retrieved. log dump shows not only endTxnMarker is not deleted, the record batch metadata is also retained: {code:java} Dumping /home/boquan/Downloads/Users/boquan/Documents/00000000003530931566.log Starting offset: 3530931566 baseOffset: 3530931566 lastOffset: 3530931567 count: 0 baseSequence: 0 lastSequence: 1 producerId: 4001 producerEpoch: 2539 partitionLeaderEpoch: 94 isTransactional: true isControl: false position: 0 CreateTime: 1556161832882 size: 61 magic: 2 compresscodec: NONE crc: 1683579819 isvalid: true baseOffset: 3530931575 lastOffset: 3530931575 count: 1 baseSequence: -1 lastSequence: -1 producerId: 4001 producerEpoch: 2539 partitionLeaderEpoch: 94 isTransactional: true isControl: true position: 61 CreateTime: 1556161832899 size: 78 magic: 2 compresscodec: NONE crc: 535474521 isvalid: true | offset: 3530931575 CreateTime: 1556161832899 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 84 baseOffset: 3530931576 lastOffset: 3530931577 count: 0 baseSequence: 0 lastSequence: 1 producerId: 3007 producerEpoch: 2516 partitionLeaderEpoch: 94 isTransactional: true isControl: false position: 139 CreateTime: 1556161832997 size: 61 magic: 2 compresscodec: NONE crc: 3760382141 isvalid: true baseOffset: 3530931578 lastOffset: 3530931579 count: 0 baseSequence: 0 lastSequence: 1 producerId: 1004 producerEpoch: 2576 partitionLeaderEpoch: 94 isTransactional: true isControl: false position: 200 CreateTime: 1556161832998 size: 61 magic: 2 compresscodec: NONE crc: 3285369041 isvalid: true baseOffset: 3530931580 lastOffset: 3530931581 count: 0 baseSequence: 0 lastSequence: 1 producerId: 1005 producerEpoch: 2545 partitionLeaderEpoch: 94 isTransactional: true isControl: false position: 261 CreateTime: 1556161832998 size: 61 magic: 2 compresscodec: NONE crc: 1698037918 isvalid: true baseOffset: 3530931582 lastOffset: 3530931583 count: 0 baseSequence: 0 lastSequence: 1 producerId: 3003 producerEpoch: 2529 partitionLeaderEpoch: 94 isTransactional: true isControl: false position: 322 CreateTime: 1556161832998 size: 61 magic: 2 compresscodec: NONE crc: 3446788505 isvalid: true baseOffset: 3530931584 lastOffset: 3530931585 count: 0 baseSequence: 0 lastSequence: 1 producerId: 3001 producerEpoch: 2486 partitionLeaderEpoch: 94 isTransactional: true isControl: false position: 383 CreateTime: 1556161832998 size: 61 magic: 2 compresscodec: NONE crc: 2245471394 isvalid: true baseOffset: 3530931586 lastOffset: 3530931587 count: 0 baseSequence: 0 lastSequence: 1 producerId: 3006 producerEpoch: 2503 partitionLeaderEpoch: 94 isTransactional: true isControl: false position: 444 CreateTime: 1556161832999 size: 61 magic: 2 compresscodec: NONE crc: 1819109301 isvalid: true baseOffset: 3530931588 lastOffset: 3530931588 count: 1 baseSequence: -1 lastSequence: -1 producerId: 3007 producerEpoch: 2516 partitionLeaderEpoch: 94 isTransactional: true isControl: true position: 505 CreateTime: 1556161833001 size: 78 magic: 2 compresscodec: NONE crc: 2403915653 isvalid: true | offset: 3530931588 CreateTime: 1556161833001 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 95 baseOffset: 3530931589 lastOffset: 3530931589 count: 1 baseSequence: -1 lastSequence: -1 producerId: 3003 producerEpoch: 2529 partitionLeaderEpoch: 94 isTransactional: true isControl: true position: 583 CreateTime: 1556161833004 size: 78 magic: 2 compresscodec: NONE crc: 4184380477 isvalid: true | offset: 3530931589 CreateTime: 1556161833004 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 95 {code} > 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 > Priority: Major > 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)