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

Maciej Bryński edited comment on KAFKA-7421 at 9/22/18 8:17 AM:
----------------------------------------------------------------

[~kkonstantine]
I think I found the reason and workaround.

Some introduction:
1) I'm building my docker image from official confluent docker image by adding 
there:
- debezium connector
- my own SMTs (let's call it abc and xyz)

All jars went to one directory (this part is important)

2) My configuration is following - I have two tasks:
- HDFS sink (official) 
it's using abc SMT
- Debezium source
it's using xyz SMT

I saw that we create one instance of classloader per directory.
So debezium and both SMTs are provided by same classloader.
So both tasks are using same classloader and debezium task is using it for both 
SMTs and task class.

Then I moved my SMTs do different directory and this solved the problem.
Still there is deadlock in special circumstances.



was (Author: maver1ck):
[~kkonstantine]
I think I found the reason and workaround.

Some introduction:
1) I'm building my docker image from official confluent docker image by adding 
there:
- debezium connector
- my own SMTs (let's call it abc and xyz)
All jars went to one directory (this part is important)

2) My configuration is following - I have two tasks:
- HDFS sink (official) 
it's using abc SMT
- Debezium source
it's using xyz SMT

I saw that we create one instance of classloader per directory.
So debezium and both SMTs are provided by same classloader.
So both tasks are using same classloader and debezium task is using it for both 
SMTs and task class.

Then I moved my SMTs do different directory and this solved the problem.
Still there is deadlock in special circumstances.


> 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: Critical
>
> 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
(v7.6.3#76005)

Reply via email to