Jeff Nadler created KAFKA-9899:
----------------------------------

             Summary: LogCleaner Tries To Clean Single Partition Over 
1000x/Minute
                 Key: KAFKA-9899
                 URL: https://issues.apache.org/jira/browse/KAFKA-9899
             Project: Kafka
          Issue Type: Bug
          Components: log cleaner
    Affects Versions: 2.4.1
         Environment: ubuntu bionic, openjdk11.0.6, kafka 2.4.1
            Reporter: Jeff Nadler
         Attachments: CPU-Usage.png

I had previously believed this to be the same issue as KAFKA-8764, but I took a 
closer look when it persisted after upgrading to 2.4.1 and now believe this is 
a different bug.

For a topic that is a very low traffic, compact topic the log cleaner will 
sometimes - for a period of usually 2 hours or longer - get stuck in a loop 
where it tries to clean the same partition for the same offset range nonstop, 
and the log cleaner thread consumes 100% of a single core during this time.
h4. 1396 attempts in a single minute:

 

{{root@stage-obs-kafka01:/var/log/kafka# cat log-cleaner.log | grep 22:22: | 
grep "offset range" | wc -l}}

{{1396}}

 
h4. All 1396 of these are looking at the same partition and same offset range:

{{[2020-04-21 22:22:59,862] INFO Cleaner 0: Building offset map for log 
elauneind-firebolt-messages-sfo-0 for 0 segments in offset range [22943108, 
22912825). (kafka.log.LogCleaner)}}

 

These attempts are separated by on average only 30ms.   This is a small 3 node 
cluster, note that the CPU graph attached is very clearly bimodal for each 
node:   low when the log cleaner is not "stuck", and much higher when it is.

Eventually the log cleaner appears to find a segment to clean (because enough 
traffic has arrived?) and the loop is broken... for a time.   Note that it 
finds "1 segments" and then finally moves on to check other topic-partitions.

{{...tens of thousands of this first one then}}

{{[2020-04-21 20:06:02,531] INFO Cleaner 0: Building offset map for log 
elauneind-firebolt-messages-sfo-0 for 0 segments in *offset range* [23591841, 
23575583). (kafka.log.LogCleaner)}}{{[2020-04-21 20:06:02,567] INFO Cleaner 0: 
Building offset map for log elauneind-firebolt-messages-sfo-0 for 1 segments in 
*offset range* [23591841, 23621641). (kafka.log.LogCleaner)}}{{[2020-04-21 
20:43:04,309] INFO Cleaner 0: Building offset map for log 
elauneind-firebolt-messages-s2r1-0 for 1 segments in *offset range* [2687968, 
2732498). (kafka.log.LogCleaner)}}

 
h4. The topic gets about 100 messsages/minute, and it's config is:

{{Topic: elauneind-firebolt-messages-sfo PartitionCount: 1 ReplicationFactor: 3 
Configs: 
min.insync.replicas=1,cleanup.policy=compact,delete,segment.bytes=102400000,retention.ms=900000,message.format.version=2.3-IV1,min.compaction.lag.ms=300000,min.cleanable.dirty.ratio=0.2,unclean.leader.election.enable=true,retention.bytes=1073741824}}{{
 Topic: elauneind-firebolt-messages-sfo Partition: 0 Leader: 0 Replicas: 0,2,1 
Isr: 0,1,2}}

 

 

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to