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

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

We are currently running a container whose base image is 
{{confluentinc/cp-kafka-connect-base:6.0.0}}, and it is designed to only ever 
run a single Debezium connector. We build a single JAR containing both our 
Debezium connector and one custom SMT. We copy this JAR into the directory 
specified by the {{plugin.path}} property on Kafka Connect, and that's about it.

We have been using this approach (with a single JAR) for several years, and 
last month we upgraded our Kafka Connect base image (previously 5.2.2) to work 
around some deprecated package archives in Debian 8. We upgraded to 5.5.1 and 
encountered these deadlocks, and more recently we have upgraded to 6.0.0 and 
switched to the UBI image. It seems like we're encountering fewer deadlocks 
than before with this newest version, but they haven't disappeared entirely.

> 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