[ https://issues.apache.org/jira/browse/KAFKA-8335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16866665#comment-16866665 ]
Francisco Juan edited comment on KAFKA-8335 at 6/18/19 2:51 PM: ---------------------------------------------------------------- Hello, we have recently updated a Kafka cluster with this same problem from version 1.1 to version 2.2.1, without updating theĀ inter.broker.protocol.version yet, still set as 1.1. We were expecting this update to reduce the size on some partitions of __consumer_offsets that keep growing. The observed behaviour is that there's still many segments with full of only "isTransactional: true" kind of messages. This is a sample of the kafka-dump-log.sh: {code:java} /usr/kafka/bin/kafka-dump-log.sh --files 00000000004107011120.log --value-decoder-class "kafka.serializer.StringDecoder" | head -n 20 Dumping 00000000004107011120.log Starting offset: 4107011120 baseOffset: 4107011154 lastOffset: 4107011154 count: 1 baseSequence: -1 lastSequence: -1 producerId: 558010 producerEpoch: 0 partitionLeaderEpoch: 490 isTransactional: true isControl: true position: 0 CreateTime: 1556123964832 size: 78 magic: 2 compresscodec: NONE crc: 1007341472 isvalid: true | offset: 4107011154 CreateTime: 1556123964832 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 84 baseOffset: 4107011178 lastOffset: 4107011178 count: 1 baseSequence: -1 lastSequence: -1 producerId: 559002 producerEpoch: 0 partitionLeaderEpoch: 490 isTransactional: true isControl: true position: 78 CreateTime: 1556123964895 size: 78 magic: 2 compresscodec: NONE crc: 470005994 isvalid: true | offset: 4107011178 CreateTime: 1556123964895 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 84 baseOffset: 4107011180 lastOffset: 4107011180 count: 1 baseSequence: -1 lastSequence: -1 producerId: 559002 producerEpoch: 0 partitionLeaderEpoch: 490 isTransactional: true isControl: true position: 156 CreateTime: 1556123964916 size: 78 magic: 2 compresscodec: NONE crc: 681157535 isvalid: true | offset: 4107011180 CreateTime: 1556123964916 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 84{code} This command is executed on Jun 18th. The `offsets.retention.minutes` is set to 40 days. The timestamps shown on this dump are way beyond the retention period. The LogCleaner DEBUG log is next: {code:java} DEBUG Finding range of cleanable offsets for log=__consumer_offsets-6 topicPartition=__consumer_offsets-6. Last clean offset=Some(5006278217) now=1560855479531 => firstDirtyOffset=5006278217 firstUncleanableOffset=5069232666 activeSegment.baseOffset=5069232666 (kafka.log.LogCleanerManager$) {code} Offsets shown on the dump are not on the active segment and are way below the firstUncleanbleOffset was (Author: francisco.juan): Hello, we have recently updated a Kafka cluster with this same problem from version 1.1 to version 2.2.1, without updating theĀ inter.broker.protocol.version yet, still set as 1.1. We were expecting this update to reduce the size on some partitions of __consumer_offsets that keep growing. The observed behaviour is that there's still many segments with full of only "isTransactional: true" kind of. This is a sample of the kafka-dump-log.sh: {code:java} /usr/kafka/bin/kafka-dump-log.sh --files 00000000004107011120.log --value-decoder-class "kafka.serializer.StringDecoder" | head -n 20 Dumping 00000000004107011120.log Starting offset: 4107011120 baseOffset: 4107011154 lastOffset: 4107011154 count: 1 baseSequence: -1 lastSequence: -1 producerId: 558010 producerEpoch: 0 partitionLeaderEpoch: 490 isTransactional: true isControl: true position: 0 CreateTime: 1556123964832 size: 78 magic: 2 compresscodec: NONE crc: 1007341472 isvalid: true | offset: 4107011154 CreateTime: 1556123964832 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 84 baseOffset: 4107011178 lastOffset: 4107011178 count: 1 baseSequence: -1 lastSequence: -1 producerId: 559002 producerEpoch: 0 partitionLeaderEpoch: 490 isTransactional: true isControl: true position: 78 CreateTime: 1556123964895 size: 78 magic: 2 compresscodec: NONE crc: 470005994 isvalid: true | offset: 4107011178 CreateTime: 1556123964895 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 84 baseOffset: 4107011180 lastOffset: 4107011180 count: 1 baseSequence: -1 lastSequence: -1 producerId: 559002 producerEpoch: 0 partitionLeaderEpoch: 490 isTransactional: true isControl: true position: 156 CreateTime: 1556123964916 size: 78 magic: 2 compresscodec: NONE crc: 681157535 isvalid: true | offset: 4107011180 CreateTime: 1556123964916 keysize: 4 valuesize: 6 sequence: -1 headerKeys: [] endTxnMarker: COMMIT coordinatorEpoch: 84{code} This command is executed on Jun 18th. The `offsets.retention.minutes` is set to 40 days. The timestamps shown on this dump are way beyond the retention period. The LogCleaner DEBUG log is next: {code:java} DEBUG Finding range of cleanable offsets for log=__consumer_offsets-6 topicPartition=__consumer_offsets-6. Last clean offset=Some(5006278217) now=1560855479531 => firstDirtyOffset=5006278217 firstUncleanableOffset=5069232666 activeSegment.baseOffset=5069232666 (kafka.log.LogCleanerManager$) {code} Offsets shown on the dump are not on the active segment and are way below the firstUncleanbleOffset > 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)