For who interested in this thread, there's a ticket created for it and we
believe it is a lurking bug and are trying to fix it before the 2.3
release: https://issues.apache.org/jira/browse/KAFKA-8335


Guozhang


On Fri, May 10, 2019 at 10:39 AM Michael Jaschob
<mjasc...@twilio.com.invalid> wrote:

> Weichu,
>
> while I don't have a solution we are seeing the same thing in our world. I
> put out a query to the mailing list a week or two ago (no responses
> unfortunately):
>
> https://lists.apache.org/thread.html/04273f5cfe4f6c6ed9ab370399f208a5cd780576880650aae839de25@%3Cusers.kafka.apache.org%3E
> .
>
> We're still investigating, and the path we think we might take is enabling
> deletion in addition to compaction on __consumer_offsets. As you saw, the
> old log segments have nothing but empty transaction batches and commit
> markers for long since expired producers, so we feel like we're probably
> safe doing that.
>
> Michael
>
> On Tue, May 7, 2019 at 3:52 AM Weichu Liu <wei...@indeed.com.invalid>
> wrote:
>
> > Hi,
> >
> > 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/
> >
> https://urldefense.proofpoint.com/v2/url?u=http-3A__kafka-2Drun-2Dclass.sh&d=DwIBaQ&c=x_Y1Lz9GyeGp2OvBCa_eow&r=ZHKtfNLXdH8j2N1pHGofCpIdHXPkUWyDl-Rljkb5iwQ&m=nuTQII_hOqT2plUWQS3zovBN0u6RGp24x9znIqONbKc&s=AA5gHWK2RAblghLeaVAJOOw_CFEVMtustckcD3g8FaM&e=
> > 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?
> >
> > Regards,
> >
> > Liu
> >
>


-- 
-- Guozhang

Reply via email to