[ 
https://issues.apache.org/jira/browse/KAFKA-7421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17208344#comment-17208344
 ] 

Kyle Leiby commented on KAFKA-7421:
-----------------------------------

Hi all, we've been encountering a similar deadlock (I think the same as the one 
[~xakassi] is seeing). We are running a single Debezium JAR inside a 
{{confluentinc/cp-kafka-connect-base:5.5.1-1-deb8}} container. We tried several 
5.x Debian 8 images, and encounter the deadlocks in all of them.

Here's the relevant portion from an example thread dump:
{code:java}
Found one Java-level deadlock:
=============================
"StartAndStopExecutor-connect-1-2":
  waiting to lock monitor 0x00007f2b68001d58 (object 0x00000000c118c3c8, a 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader),
  which is held by "StartAndStopExecutor-connect-1-1"
"StartAndStopExecutor-connect-1-1":
  waiting to lock monitor 0x00007f2b68001eb8 (object 0x00000000c5100000, a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader),
  which is held by "StartAndStopExecutor-connect-1-2"

Java stack information for the threads listed above:
===================================================
"StartAndStopExecutor-connect-1-2":
        at java.lang.ClassLoader.loadClass(ClassLoader.java:404)
        - waiting to lock <0x00000000c118c3c8> (a 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader)
        at 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:397)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:411)
        - locked <0x00000000c6a9e908> (a java.lang.Object)
        at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:104)
        - locked <0x00000000c6a9e908> (a java.lang.Object)
        - locked <0x00000000c5100000> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
        at java.lang.Class.forName0(Native Method)
        at java.lang.Class.forName(Class.java:348)
        at 
org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:719)
        at 
org.apache.kafka.connect.runtime.ConnectorConfig.enrich(ConnectorConfig.java:311)
        at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:215)
        at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:209)
        at org.apache.kafka.connect.runtime.Worker.startTask(Worker.java:432)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.startTask(DistributedHerder.java:1186)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.access$1600(DistributedHerder.java:127)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder$12.call(DistributedHerder.java:1201)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder$12.call(DistributedHerder.java:1197)
        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)
"StartAndStopExecutor-connect-1-1":
        at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:91)
        - waiting to lock <0x00000000c5100000> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader)
        at 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:394)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
        at java.lang.Class.forName0(Native Method)
        at java.lang.Class.forName(Class.java:348)
        at 
org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:719)
        at 
org.apache.kafka.connect.runtime.ConnectorConfig.enrich(ConnectorConfig.java:311)
        at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:215)
        at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:209)
        at 
org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:251)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.startConnector(DistributedHerder.java:1229)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.access$1300(DistributedHerder.java:127)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder$14.call(DistributedHerder.java:1245)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder$14.call(DistributedHerder.java:1241)
        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)

Found 1 deadlock.
{code}

> Deadlock in Kafka Connect
> -------------------------
>
>                 Key: KAFKA-7421
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7421
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>    Affects Versions: 2.0.0
>            Reporter: Maciej Bryński
>            Assignee: Konstantine Karantasis
>            Priority: Major
>
> 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:
> {code}
> "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)
> {code}
> Thread 2:
> {code}
> "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)
> {code}
> I'm using official Confluent Docker images.



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

Reply via email to