Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-8448

Too many kafka.log.Log instances (Memory Leak)

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.2.0
    • 2.4.0
    • None
    • None
    • Red Hat 4.4.7-16, java version "1.8.0_152", kafka_2.12-2.2.0

    Description

      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 close the consumer involved, but no the producer.

      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 (currentLogslogsToBeDeleted  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.

        scheduler.schedule(name = "PeriodicProducerExpirationCheck", fun = () => {
          lock synchronized {
            producerStateManager.removeExpiredProducers(time.milliseconds)
          }
        }, period = producerIdExpirationCheckIntervalMs, delay = producerIdExpirationCheckIntervalMs, unit = TimeUnit.MILLISECONDS)
      

      However it seems those tasks are never unscheduled/cancelled

      Attachments

        Activity

          People

            jolshan Justine Olshan
            juan.olivares Juan Olivares
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: