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

Francisco Juan edited comment on KAFKA-8335 at 6/24/19 7:55 AM:
----------------------------------------------------------------

Hello [~boquan]

I have applied your suggestion but seems to not be able to solve the issue for 
2 out of 50 partitions.

The biggest offender in my case would be partition 6 of __consumer_offsets 
which is now at 22GB.

This is what the new state after I have cleaned up the 
cleaner-offset-checkpoint:
{code:java}
[2019-06-24 06:18:05,968] DEBUG Finding range of cleanable offsets for 
log=__consumer_offsets-6 topicPartition=__consumer_offsets-6. Last clean 
offset=Some(0) now=1561357085930 => firstDirtyOffset=0 
firstUncleanableOffset=5149534913 activeSegment.baseOffset=5149534913 
(kafka.log.LogCleanerManager$){code}
and this is the outcome of the cleaner:
{code:java}
[2019-06-24 06:25:48,969] INFO [kafka-log-cleaner-thread-0]: Log cleaner thread 
0 cleaned log __consumer_offsets-6 (dirty section = [0, 0]) 25,096.6 MB of log 
processed in 463.0 seconds (54.2 MB/sec). Indexed 25,096.6 MB in 177.6 seconds 
(141.3 Mb/sec, 38.4% of total time) Buffer utilization: 0.0% Cleaned 25,096.6 
MB in 285.4 seconds (87.9 Mb/sec, 61.6% of total time) Start size: 25,096.6 MB 
(313,976,723 messages) End size: 21,674.0 MB (291,368,805 messages) 13.6% size 
reduction (7.2% fewer messages) (kafka.log.LogCleaner){code}
The partition currently has 226 log files and takes 22G of disk space.

When sampling one of the earliest file in the partition, these are my results:
{code:java}
$> kafka-dump-log.sh --files 00000000004114035438.log | head -n 5
Dumping 00000000004114035438.log
Starting offset: 4114035438
baseOffset: 4114035438 lastOffset: 4114035438 count: 1 baseSequence: -1 
lastSequence: -1 producerId: 547022 producerEpoch: 0 partitionLeaderEpoch: 490 
isTransactional: true isControl: true position: 0 CreateTime: 1556176906520 
size: 78 magic: 2 compresscodec: NONE crc: 3444550803 isvalid: true
baseOffset: 4114035439 lastOffset: 4114035439 count: 1 baseSequence: -1 
lastSequence: -1 producerId: 551003 producerEpoch: 0 partitionLeaderEpoch: 490 
isTransactional: true isControl: true position: 78 CreateTime: 1556176906524 
size: 78 magic: 2 compresscodec: NONE crc: 517398751 isvalid: true
baseOffset: 4114035442 lastOffset: 4114035442 count: 1 baseSequence: -1 
lastSequence: -1 producerId: 556020 producerEpoch: 0 partitionLeaderEpoch: 490 
isTransactional: true isControl: true position: 156 CreateTime: 1556176906560 
size: 78 magic: 2 compresscodec: NONE crc: 174729981 isvalid: true
$> kafka-dump-log.sh --files 00000000004114035438.log | grep "isTransactional: 
true" | wc -l
1340521
$> kafka-dump-log.sh --files 00000000004114035438.log | grep -v 
"isTransactional: true" | wc -l
2 
{code}
The timestamp shown here is from {{Thu Apr 25 07:21:46 UTC 2019}}

What I think are relevant settings for my cluster are:
 * kafka cluster version: 2.2.1
 * inter.broker.protocol.version: 1.1
 * offsets.retention.minutes=20160
 * __consumer_offsets settings: segment.bytes=104857600,cleanup.policy=compact
 * I've applied temporarily the suggestion from above:  retention.ms=1209600000 
(two weeks) and min.cleanable.dirty.ratio=0.2

What additional information could be useful to try and dig up the issue?

Thanks


was (Author: francisco.juan):
Hello [~boquan]

I have applied your suggestion but seems to not be able to solve the issue for 
2 out of 50 partitions.

The biggest offender in my case would be partition 6 of __consumer_offsets 
which is now at 22GB.

This is what the new state after I have cleaned up the 
cleaner-offset-checkpoint:
{code:java}
[2019-06-24 06:18:05,968] DEBUG Finding range of cleanable offsets for 
log=__consumer_offsets-6 topicPartition=__consumer_offsets-6. Last clean 
offset=Some(0) now=1561357085930 => firstDirtyOffset=0 
firstUncleanableOffset=5149534913 activeSegment.baseOffset=5149534913 
(kafka.log.LogCleanerManager$){code}
and this is the outcome of the cleaner:
{code:java}
[2019-06-24 06:25:48,969] INFO [kafka-log-cleaner-thread-0]: Log cleaner thread 
0 cleaned log __consumer_offsets-6 (dirty section = [0, 0]) 25,096.6 MB of log 
processed in 463.0 seconds (54.2 MB/sec). Indexed 25,096.6 MB in 177.6 seconds 
(141.3 Mb/sec, 38.4% of total time) Buffer utilization: 0.0% Cleaned 25,096.6 
MB in 285.4 seconds (87.9 Mb/sec, 61.6% of total time) Start size: 25,096.6 MB 
(313,976,723 messages) End size: 21,674.0 MB (291,368,805 messages) 13.6% size 
reduction (7.2% fewer messages) (kafka.log.LogCleaner){code}
The partition currently has 226 log files and takes 22G of disk space.

When sampling one of the earliest file in the partition, these are my results:
{code:java}
$> kafka-dump-log.sh --files 00000000004114035438.log | head -n 5
Dumping 00000000004114035438.log
Starting offset: 4114035438
baseOffset: 4114035438 lastOffset: 4114035438 count: 1 baseSequence: -1 
lastSequence: -1 producerId: 547022 producerEpoch: 0 partitionLeaderEpoch: 490 
isTransactional: true isControl: true position: 0 CreateTime: 1556176906520 
size: 78 magic: 2 compresscodec: NONE crc: 3444550803 isvalid: true
baseOffset: 4114035439 lastOffset: 4114035439 count: 1 baseSequence: -1 
lastSequence: -1 producerId: 551003 producerEpoch: 0 partitionLeaderEpoch: 490 
isTransactional: true isControl: true position: 78 CreateTime: 1556176906524 
size: 78 magic: 2 compresscodec: NONE crc: 517398751 isvalid: true
baseOffset: 4114035442 lastOffset: 4114035442 count: 1 baseSequence: -1 
lastSequence: -1 producerId: 556020 producerEpoch: 0 partitionLeaderEpoch: 490 
isTransactional: true isControl: true position: 156 CreateTime: 1556176906560 
size: 78 magic: 2 compresscodec: NONE crc: 174729981 isvalid: true
$> kafka-dump-log.sh --files 00000000004114035438.log | grep "isTransactional: 
true" | wc -l
1340521
$> kafka-dump-log.sh --files 00000000004114035438.log | grep -v 
"isTransactional: true" | wc -l
2 
{code}
The timestamp shown here is from {{Thu Apr 25 07:21:46 UTC 2019}}

What I think are relevant settings for my cluster are:
 * offsets.retention.minutes=20160
 * __consumer_offsets settings: segment.bytes=104857600,cleanup.policy=compact
 * I've applied temporarily the suggestion from above:  retention.ms=1209600000 
(two weeks) and min.cleanable.dirty.ratio=0.2

What additional information could be useful to try and dig up the issue?

Thanks

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

Reply via email to