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

Goltseva Taisiia edited comment on KAFKA-7421 at 8/27/20, 2:42 PM:
-------------------------------------------------------------------

Hi, guys!

We faced the similar deadlock (not equal but similar):
{code:java}
Found one Java-level deadlock:
=============================
"pool-9-thread-6":
  waiting to lock monitor 0x00007f112e5565a8 (object 0x0000000084400d80, a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader),
  which is held by "pool-9-thread-5"
"pool-9-thread-5":
  waiting to lock monitor 0x00007f112e528c48 (object 0x000000008441a4c8, a 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader),
  which is held by "pool-9-thread-2"
"pool-9-thread-2":
  waiting to lock monitor 0x00007f112e5565a8 (object 0x0000000084400d80, a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader),
  which is held by "pool-9-thread-5"{code}

Thread-1:
{code:java}
"pool-9-thread-6":"pool-9-thread-6": at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:91)
 - waiting to lock <0x0000000084400d80> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader) at 
java.lang.ClassLoader.loadClass(ClassLoader.java:352) at 
org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:716) at 
org.apache.kafka.common.config.ConfigDef.parseValue(ConfigDef.java:474) at 
org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:467) at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:108) 
at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:129) 
at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:210)
 at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:206)
 at org.apache.kafka.connect.runtime.Worker.startTask(Worker.java:430) at 
com.company.streaming.platform.kafka.DistributedHerder.startTask(DistributedHerder.java:1216)
 at 
com.company.streaming.platform.kafka.DistributedHerder.access$1700(DistributedHerder.java:97)
 at 
com.company.streaming.platform.kafka.DistributedHerder$13.call(DistributedHerder.java:1231)
 at 
com.company.streaming.platform.kafka.DistributedHerder$13.call(DistributedHerder.java:1227)
 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:java}
"pool-9-thread-5":"pool-9-thread-5": at 
java.lang.ClassLoader.loadClass(ClassLoader.java:399) - waiting to lock 
<0x000000008441a4c8> (a 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader) at 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:392)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:406) - locked 
<0x0000000080a80780> (a java.lang.Object) at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:104)
 - locked <0x0000000080a80780> (a java.lang.Object) - locked 
<0x0000000084400d80> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader) at 
java.lang.ClassLoader.loadClass(ClassLoader.java:352) at 
org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:716) at 
org.apache.kafka.common.config.ConfigDef.parseValue(ConfigDef.java:474) at 
org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:467) at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:108) 
at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:129) 
at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:210)
 at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:206)
 at org.apache.kafka.connect.runtime.Worker.startTask(Worker.java:430) at 
com.company.streaming.platform.kafka.DistributedHerder.startTask(DistributedHerder.java:1216)
 at 
com.company.streaming.platform.kafka.DistributedHerder.access$1700(DistributedHerder.java:97)
 at 
com.company.streaming.platform.kafka.DistributedHerder$13.call(DistributedHerder.java:1231)
 at 
com.company.streaming.platform.kafka.DistributedHerder$13.call(DistributedHerder.java:1227)
 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-3:
{code:java}
"pool-9-thread-2":"pool-9-thread-2": at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:91)
 - waiting to lock <0x0000000084400d80> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader) at 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:389)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:352) 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.common.config.ConfigDef.parseValue(ConfigDef.java:474) at 
org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:467) at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:108) 
at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:129) 
at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:210)
 at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:206)
 at org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:249) at 
com.company.streaming.platform.kafka.DistributedHerder.startConnector(DistributedHerder.java:1265)
 at 
com.company.streaming.platform.kafka.DistributedHerder.access$1400(DistributedHerder.java:97)
 at 
com.company.streaming.platform.kafka.DistributedHerder$15.call(DistributedHerder.java:1326)
 at 
com.company.streaming.platform.kafka.DistributedHerder$15.call(DistributedHerder.java:1322)
 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}


was (Author: xakassi):
Hi, guys!

We faced the similar deadlock (not equal but similar):


{code:java}
Found one Java-level deadlock:Found one Java-level 
deadlock:============================="pool-9-thread-6":  waiting to lock 
monitor 0x00007f112e5565a8 (object 0x0000000084400d80, a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader),  which is held 
by "pool-9-thread-5""pool-9-thread-5":  waiting to lock monitor 
0x00007f112e528c48 (object 0x000000008441a4c8, a 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader),  which is 
held by "pool-9-thread-2""pool-9-thread-2":  waiting to lock monitor 
0x00007f112e5565a8 (object 0x0000000084400d80, a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader),  which is held 
by "pool-9-thread-5"
Java stack information for the threads listed 
above:==================================================="pool-9-thread-6": at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:91)
 - waiting to lock <0x0000000084400d80> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader) at 
java.lang.ClassLoader.loadClass(ClassLoader.java:352) at 
org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:716) at 
org.apache.kafka.common.config.ConfigDef.parseValue(ConfigDef.java:474) at 
org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:467) at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:108) 
at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:129) 
at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:210)
 at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:206)
 at org.apache.kafka.connect.runtime.Worker.startTask(Worker.java:430) at 
com.company.streaming.platform.kafka.DistributedHerder.startTask(DistributedHerder.java:1216)
 at 
com.company.streaming.platform.kafka.DistributedHerder.access$1700(DistributedHerder.java:97)
 at 
com.company.streaming.platform.kafka.DistributedHerder$13.call(DistributedHerder.java:1231)
 at 
com.company.streaming.platform.kafka.DistributedHerder$13.call(DistributedHerder.java:1227)
 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)"pool-9-thread-5": at 
java.lang.ClassLoader.loadClass(ClassLoader.java:399) - waiting to lock 
<0x000000008441a4c8> (a 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader) at 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:392)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:406) - locked 
<0x0000000080a80780> (a java.lang.Object) at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:104)
 - locked <0x0000000080a80780> (a java.lang.Object) - locked 
<0x0000000084400d80> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader) at 
java.lang.ClassLoader.loadClass(ClassLoader.java:352) at 
org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:716) at 
org.apache.kafka.common.config.ConfigDef.parseValue(ConfigDef.java:474) at 
org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:467) at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:108) 
at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:129) 
at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:210)
 at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:206)
 at org.apache.kafka.connect.runtime.Worker.startTask(Worker.java:430) at 
com.company.streaming.platform.kafka.DistributedHerder.startTask(DistributedHerder.java:1216)
 at 
com.company.streaming.platform.kafka.DistributedHerder.access$1700(DistributedHerder.java:97)
 at 
com.company.streaming.platform.kafka.DistributedHerder$13.call(DistributedHerder.java:1231)
 at 
com.company.streaming.platform.kafka.DistributedHerder$13.call(DistributedHerder.java:1227)
 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)"pool-9-thread-2": at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:91)
 - waiting to lock <0x0000000084400d80> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader) at 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:389)
 at java.lang.ClassLoader.loadClass(ClassLoader.java:352) 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.common.config.ConfigDef.parseValue(ConfigDef.java:474) at 
org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:467) at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:108) 
at 
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:129) 
at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:210)
 at 
org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:206)
 at org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:249) at 
com.company.streaming.platform.kafka.DistributedHerder.startConnector(DistributedHerder.java:1265)
 at 
com.company.streaming.platform.kafka.DistributedHerder.access$1400(DistributedHerder.java:97)
 at 
com.company.streaming.platform.kafka.DistributedHerder$15.call(DistributedHerder.java:1326)
 at 
com.company.streaming.platform.kafka.DistributedHerder$15.call(DistributedHerder.java:1322)
 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}

> 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