[ 
https://issues.apache.org/jira/browse/KAFKA-8335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16837664#comment-16837664
 ] 

ASF GitHub Bot commented on KAFKA-8335:
---------------------------------------

hachikuji commented on pull request #6715: KAFKA-8335; Clean empty batches when 
sequence numbers are reused
URL: https://github.com/apache/kafka/pull/6715
 
 
   The log cleaner attempts to preserve the last entry for each producerId in 
order to ensure that sequence/epoch state is not lost. The current validation 
checks only the last sequence number for each producerId in order to decide 
whether a batch should be retained. There are two problems with this:
   
   1. Sequence numbers are not unique alone. It is the tuple of sequence number 
and epoch which is uniquely defined.
   2. The group coordinator always writes batches beginning with sequence 
number 0, which means there could be many batches which have the same sequence 
number.
   
   The complete fix for the second issue is probably to do the proper sequence 
number bookkeeping in the coordinator. For now, I have left the coordinator 
implementation unchanged and changed the cleaner logic to use the last offset 
written by a producer instead of the last sequence number. 
   
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> 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
>         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)

Reply via email to