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

2020-10-16 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 edited comment on KAFKA-7421 at 10/16/20, 6:25 AM:


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 connectors (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 
single JAR 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?


was (Author: xakassi):
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 connectors (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 

[jira] [Comment Edited] (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 edited comment on KAFKA-7421 at 10/15/20, 1:30 PM:


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 connectors (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?


was (Author: xakassi):
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 

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

2020-08-28 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 edited comment on KAFKA-7421 at 8/28/20, 8:16 AM:
---

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. And as I can see, this method do not use 
any fields of *{{PluginClassLoader}}* class.

[~kkonstantine] what do you think about it? Can we propose a PR to remove 
synchronized from *{{PluginClassLoader}}'s loadClass* method?


was (Author: xakassi):
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)

[jira] [Comment Edited] (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 edited comment on KAFKA-7421 at 8/27/20, 2:43 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 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"{code}
Thread-1:
{code:java}
"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){code}
Thread-2:
{code:java}
"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){code}
Thread-3:
{code:java}
"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] [Comment Edited] (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 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 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"{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 <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){code}

Thread-2:
{code:java}
"pool-9-thread-5":"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){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 <0x84400d80> (a 
org.apache.kafka.connect.runtime.isolation.PluginClassLoader) at 

[jira] [Comment Edited] (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 edited comment on KAFKA-7421 at 8/27/20, 2:38 PM:
---

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?


was (Author: xakassi):
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 
> 

[jira] [Comment Edited] (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 edited comment on KAFKA-7421 at 9/22/18 8:18 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 to 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.(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 
> 

[jira] [Comment Edited] (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 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.(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 
> 

[jira] [Comment Edited] (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 edited comment on KAFKA-7421 at 9/21/18 8:35 PM:


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


was (Author: maver1ck):
[~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 
>