Anthony Maire created KAFKA-18046:
-------------------------------------

             Summary: High CPU usage when using Log4j2
                 Key: KAFKA-18046
                 URL: https://issues.apache.org/jira/browse/KAFKA-18046
             Project: Kafka
          Issue Type: Improvement
          Components: consumer
    Affects Versions: 3.8.1, 3.6.1
            Reporter: Anthony Maire
         Attachments: kafka_consumer_getLogger.png

This is pretty similar to KAFKA-15141 that impacted server side : when using 
log4j2 as SLF4J implementation, LoggerFactory.getLogger() is a pretty costly 
operation that use StackWalker under the hood and can have a performance impact 
in hot code paths

 

CPU profiling shows that getLogger() is responsible for more than 20% of CPU 
usage in KafkaConsumer.poll(), because 
org.apache.kafka.clients.consumer.internals.CompletedFetch constructor uses 
org.apache.kafka.common.utils.LogContext.logger(Class<?>) which calls 
LoggerFactory.getLogger().

!kafka_consumer_getLogger.png!

I can provide a PR for this, but there are 2 possible implementations :
- don't change LogContext API, but keep a static cache of loggers inside it 
(using ConcurrentHashMap.computeIfAbsent() ) : it will cache logger for all 
classes using LogContext, including cold paths. The memory retention induced by 
it was very low in my tests as loggers are already referenced in Log4J internal 
maps. A heap dump showed an additionnal 848 bytes retained by the cache for 24 
entries in my tests

 

- If we want to only do this on the hot code paths and make sure we avoid any 
memory retention, then we should add a new method LogContext.logger(Logger) 
that would wrap a static logger instantiated in the relevant classes.

My preference would be to do a global fix by using the first approach, but I 
wanted to ask there first before proviing a PR if I missed a scenario when it 
could create a undesirable significant memory retention



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to