You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Juan Olivares (JIRA)" <ji...@apache.org> on 2019/05/30 14:26:00 UTC
[jira] [Created] (KAFKA-8448) Too many kafka.log.Log instances
(Memory Leak)
Juan Olivares created KAFKA-8448:
------------------------------------
Summary: Too many kafka.log.Log instances (Memory Leak)
Key: KAFKA-8448
URL: https://issues.apache.org/jira/browse/KAFKA-8448
Project: Kafka
Issue Type: Bug
Affects Versions: 2.2.0
Environment: Red Hat 4.4.7-16, java version "1.8.0_152", kafka_2.12-2.2.0
Reporter: Juan Olivares
We have a custom Kafka health check which creates a topic add some ACLs (read/write topic and group), produce & consume a single message and then quickly remove it and all the related ACLs created.
We have observed that # of instances of {{kafka.log.Log}} keep growing, while there's no evidence of topics being leaked, neither running {{/opt/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --describe}} , nor looking at the disk directory where topics are stored.
After looking at the heapdump we've observed the following
- None of the {{kafka.log.Log}} references ({{currentLogs}}, {{logsToBeDeleted }} and {{logsToBeDeleted}}) in {{kafka.log.LogManager}} is holding the big amount of {{kafka.log.Log}} instances.
- The only reference preventing {{kafka.log.Log}} to be Garbage collected seems to be {{java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue}} which contains schedule tasks created with the name {{PeriodicProducerExpirationCheck}}.
I can see in the code that for every {{kafka.log.Log}} a task with this name is scheduled.
{code:java}
scheduler.schedule(name = "PeriodicProducerExpirationCheck", fun = () => {
lock synchronized {
producerStateManager.removeExpiredProducers(time.milliseconds)
}
}, period = producerIdExpirationCheckIntervalMs, delay = producerIdExpirationCheckIntervalMs, unit = TimeUnit.MILLISECONDS)
{code}
However it seems those tasks are never unscheduled/cancelled
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)