[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect during class loading

2021-07-14 Thread Konstantine Karantasis (Jira)


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

Konstantine Karantasis commented on KAFKA-7421:
---

The fix has now been merged. Let's keep track and report any new issues if they 
appear. Some context exists on 
https://issues.apache.org/jira/browse/KAFKA-12308 as well which reported a 
similar issue. 

> Deadlock in Kafka Connect during class loading
> --
>
> 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
> Fix For: 3.0.0
>
>
> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2021-03-09 Thread Eazhilan Nagarajan (Jira)


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

Eazhilan Nagarajan commented on KAFKA-7421:
---

Good to see that there is a fix on this issue. 

We're facing the deadlocks occurring more frequently during connect startup. 
We're using almost 6 connect plugins along with some custom SMTs and convertors 
and seems like the frequency of deadlocks has increased after we added couple 
of more SMTs and connector JARs.

While a fix is in progress, I'd like to know if there are any workarounds which 
would reduce the frequency of deadlocks. Any suggestion to overcome this issue 
would be highly appreciated.

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2021-02-25 Thread Konstantine Karantasis (Jira)


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

Konstantine Karantasis commented on KAFKA-7421:
---

The suggested fix suggested by [~ivanyu] seems plausible. The method locking 
seems to have been an oversight given the fine grain locking based on 
classloader instance and requested class name. 
I've extended the PR that added a test with the probable fix here: 
https://github.com/apache/kafka/pull/8259

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2020-11-13 Thread Kyle Leiby (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-7421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2020-10-15 Thread Goltseva Taisiia (Jira)


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

Goltseva Taisiia commented on KAFKA-7421:
-

Hi, [~kleiby] !

Let me describe my issue more detailed. Maybe we will be able to figure out 
some similar details with you which causing the problem.

We have a base docker image containing several connector (each connector in a 
separate folder under /plugins folder). And one custom converter in /libs. And 
this docker image works perfect, never such deadlocks.

Also we have a child docker image which adds additional connector in a separate 
folder under /plugins folder. And several additional custom converters in the 
same folder! We cannot put them in /libs folder currently, but I will try to do 
it. And this docker image get this deadlock very often.

Several custom converters in the same folder with a connector is the only 
difference with the approach of our base docker image. So, I suppose maybe this 
cause our deadlocks. But I'm not sure, I will try to separate converters and 
see.

 

Can you describe your configuration also& Maybe there are some similarities 
with us? Do you have SMTs or converters in one folder, in one jar?

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2020-10-05 Thread Kyle Leiby (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-7421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 0x7f2b68001d58 (object 0xc118c3c8, a 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader),
  which is held by "StartAndStopExecutor-connect-1-1"
"StartAndStopExecutor-connect-1-1":
  waiting to lock monitor 0x7f2b68001eb8 (object 0xc510, 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 <0xc118c3c8> (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 <0xc6a9e908> (a java.lang.Object)
at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:104)
- locked <0xc6a9e908> (a java.lang.Object)
- locked <0xc510> (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.(ConnectorConfig.java:215)
at 
org.apache.kafka.connect.runtime.ConnectorConfig.(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 <0xc510> (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.(ConnectorConfig.java:215)
at 
org.apache.kafka.connect.runtime.ConnectorConfig.(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 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2020-08-27 Thread Goltseva Taisiia (Jira)


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

Goltseva Taisiia commented on KAFKA-7421:
-

And I agree with [~ivanyu], I do not understand why {{PluginClassLoader}}'s 
{{loadClass }}method is marked as synchronized, because  it has fine-grain 
synchronization immediately inside.

[~kkonstantine] what do you think about it?

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2020-08-27 Thread Goltseva Taisiia (Jira)


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

Goltseva Taisiia commented on KAFKA-7421:
-

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 0x7f112e5565a8 (object 0x84400d80, a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader),  which is held 
by "pool-9-thread-5""pool-9-thread-5":  waiting to lock monitor 
0x7f112e528c48 (object 0x8441a4c8, a 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader),  which is 
held by "pool-9-thread-2""pool-9-thread-2":  waiting to lock monitor 
0x7f112e5565a8 (object 0x84400d80, 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 <0x84400d80> (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.(AbstractConfig.java:108) 
at 
org.apache.kafka.common.config.AbstractConfig.(AbstractConfig.java:129) 
at 
org.apache.kafka.connect.runtime.ConnectorConfig.(ConnectorConfig.java:210)
 at 
org.apache.kafka.connect.runtime.ConnectorConfig.(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 
<0x8441a4c8> (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 
<0x80a80780> (a java.lang.Object) at 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:104)
 - locked <0x80a80780> (a java.lang.Object) - locked 
<0x84400d80> (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.(AbstractConfig.java:108) 
at 
org.apache.kafka.common.config.AbstractConfig.(AbstractConfig.java:129) 
at 
org.apache.kafka.connect.runtime.ConnectorConfig.(ConnectorConfig.java:210)
 at 
org.apache.kafka.connect.runtime.ConnectorConfig.(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 <0x84400d80> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader) at 
org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:389)
 at 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2020-03-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on KAFKA-7421:
---

gharris1727 commented on pull request #8259: KAFKA-7421: Reproduce 
Plugin/Delegating ClassLoader deadlock
URL: https://github.com/apache/kafka/pull/8259
 
 
   * Adds SynchronizationTest with a single case
   * Simulates Worker::startConnector and Worker::startTask classloading 
difference
   
   Signed-off-by: Greg Harris 
   
   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(ByLogicalTableRouter.java:57)
> at 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2019-08-21 Thread Ivan Yurchenko (Jira)


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

Ivan Yurchenko commented on KAFKA-7421:
---

I wonder if {{PluginClassLoader}}'s {{loadClass}} method needs to be 
{{synchronized}} in 
[here|https://github.com/apache/kafka/blob/3cdc78e6bb1f83973a14ce1550fe3874f7348b05/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginClassLoader.java#L89]:
 it has fine-grain synchronization immediately inside (due to being registered 
as parallel capable). This method's outher synchronization on {{this}} is what 
seems to cause occasional deadlocks on concurrent class loading.

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2019-08-06 Thread Ivan Yurchenko (JIRA)


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

Ivan Yurchenko commented on KAFKA-7421:
---

I hit this in prod and am interested in getting it fixed. [~kkonstantine] 
please let me know if you're still considering fixing this yourself. If no, 
I'll pick this up.

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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)
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2018-09-22 Thread JIRA


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

Maciej Bryński commented on KAFKA-7421:
---

[~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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(ConnectorConfig.java:194)
> at 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2018-09-22 Thread JIRA


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

Maciej Bryński commented on KAFKA-7421:
---

I'm configuring two connectors. (Debezium and HDFS)
Then when I'm restarting connect-distributed process the deadlock appears.

I'm running everything on K8S using confluent helm chart and confluent docker 
images.
(so restart is removing pod)

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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)
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2018-09-21 Thread Konstantine Karantasis (JIRA)


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

Konstantine Karantasis commented on KAFKA-7421:
---

Thanks [~maver1ck] for the feedback. It makes sense. I have an idea for a fix 
and I'll put out a PR soon. 

Can you share some info about how you are able to reproduce the issue? Do you 
just start the two aforementioned connectors at the same time? I see it's in 
distributed mode and that they both get stuck during their startup. Anything 
special about their configs? 

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2018-09-21 Thread JIRA


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

Maciej Bryński commented on KAFKA-7421:
---

PS. I never have such a problem with older version of Kafka (confluent platform 
4.0.0)

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2018-09-21 Thread JIRA


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

Maciej Bryński commented on KAFKA-7421:
---

[~kkonstantine] Nope. I was able to reproduce it wit running connect process 
with: -XX:+AlwaysLockClassLoader

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 
> 

[jira] [Commented] (KAFKA-7421) Deadlock in Kafka Connect

2018-09-20 Thread Konstantine Karantasis (JIRA)


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

Konstantine Karantasis commented on KAFKA-7421:
---

[~maver1ck] does setting {{-XX:+AlwaysLockClassLoader}} in the JVM properties 
make the issue go away? It'd be good to know. 

> 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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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.(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.(ConnectorConfig.java:200)
> at 
> org.apache.kafka.connect.runtime.ConnectorConfig.(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 
>