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

Deadlock in Kafka Connect during class loading

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.0.0
    • 3.0.0
    • connect
    • None

    Description

      I'm getting this deadlock on half of Kafka Connect runs when having two different types connectors (in this configuration it's debezium and hdfs).
      Thread 1:

      "pool-22-thread-2@4748" prio=5 tid=0x4d nid=NA waiting for monitor entry
        java.lang.Thread.State: BLOCKED
      	 waiting for pool-22-thread-1@4747 to release lock on <0x1423> (a org.apache.kafka.connect.runtime.isolation.PluginClassLoader)
      	  at org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:91)
      	  at org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:367)
      	  at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
      	  at java.lang.Class.forName0(Class.java:-1)
      	  at java.lang.Class.forName(Class.java:348)
      	  at org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:715)
      	  at org.apache.kafka.connect.runtime.ConnectorConfig.enrich(ConnectorConfig.java:295)
      	  at org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:200)
      	  at org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:194)
      	  at org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:233)
      	  at org.apache.kafka.connect.runtime.distributed.DistributedHerder.startConnector(DistributedHerder.java:916)
      	  at org.apache.kafka.connect.runtime.distributed.DistributedHerder.access$1300(DistributedHerder.java:111)
      	  at org.apache.kafka.connect.runtime.distributed.DistributedHerder$15.call(DistributedHerder.java:932)
      	  at org.apache.kafka.connect.runtime.distributed.DistributedHerder$15.call(DistributedHerder.java:928)
      	  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	  at java.lang.Thread.run(Thread.java:748)
      

      Thread 2:

      "pool-22-thread-1@4747" prio=5 tid=0x4c nid=NA waiting for monitor entry
        java.lang.Thread.State: BLOCKED
      	 blocks pool-22-thread-2@4748
      	 waiting for pool-22-thread-2@4748 to release lock on <0x1421> (a org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader)
      	  at java.lang.ClassLoader.loadClass(ClassLoader.java:406)
      	  at org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:358)
      	  at java.lang.ClassLoader.loadClass(ClassLoader.java:411)
      	  - locked <0x1424> (a java.lang.Object)
      	  at org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:104)
      	  - locked <0x1423> (a org.apache.kafka.connect.runtime.isolation.PluginClassLoader)
      	  at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
      	  at io.debezium.transforms.ByLogicalTableRouter.<clinit>(ByLogicalTableRouter.java:57)
      	  at java.lang.Class.forName0(Class.java:-1)
      	  at java.lang.Class.forName(Class.java:348)
      	  at org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:715)
      	  at org.apache.kafka.connect.runtime.ConnectorConfig.enrich(ConnectorConfig.java:295)
      	  at org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:200)
      	  at org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:194)
      	  at org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:233)
      	  at org.apache.kafka.connect.runtime.distributed.DistributedHerder.startConnector(DistributedHerder.java:916)
      	  at org.apache.kafka.connect.runtime.distributed.DistributedHerder.access$1300(DistributedHerder.java:111)
      	  at org.apache.kafka.connect.runtime.distributed.DistributedHerder$15.call(DistributedHerder.java:932)
      	  at org.apache.kafka.connect.runtime.distributed.DistributedHerder$15.call(DistributedHerder.java:928)
      	  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	  at java.lang.Thread.run(Thread.java:748)
      

      I'm using official Confluent Docker images.

      Attachments

        Issue Links

          Activity

            People

              kkonstantine Konstantine Karantasis
              maver1ck Maciej BryƄski
              Votes:
              3 Vote for this issue
              Watchers:
              10 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: