[ https://issues.apache.org/jira/browse/KAFKA-7421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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 <gr...@confluent.io> *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.<init>(ConnectorConfig.java:200) > at > org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:194) > at > org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:233) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder.startConnector(DistributedHerder.java:916) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder.access$1300(DistributedHerder.java:111) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder$15.call(DistributedHerder.java:932) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder$15.call(DistributedHerder.java:928) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > {code} > Thread 2: > {code} > "pool-22-thread-1@4747" prio=5 tid=0x4c nid=NA waiting for monitor entry > java.lang.Thread.State: BLOCKED > blocks pool-22-thread-2@4748 > waiting for pool-22-thread-2@4748 to release lock on <0x1421> (a > org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader) > at java.lang.ClassLoader.loadClass(ClassLoader.java:406) > at > org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader.loadClass(DelegatingClassLoader.java:358) > at java.lang.ClassLoader.loadClass(ClassLoader.java:411) > - locked <0x1424> (a java.lang.Object) > at > org.apache.kafka.connect.runtime.isolation.PluginClassLoader.loadClass(PluginClassLoader.java:104) > - locked <0x1423> (a > org.apache.kafka.connect.runtime.isolation.PluginClassLoader) > at java.lang.ClassLoader.loadClass(ClassLoader.java:357) > at > io.debezium.transforms.ByLogicalTableRouter.<clinit>(ByLogicalTableRouter.java:57) > at java.lang.Class.forName0(Class.java:-1) > at java.lang.Class.forName(Class.java:348) > at > org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:715) > at > org.apache.kafka.connect.runtime.ConnectorConfig.enrich(ConnectorConfig.java:295) > at > org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:200) > at > org.apache.kafka.connect.runtime.ConnectorConfig.<init>(ConnectorConfig.java:194) > at > org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:233) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder.startConnector(DistributedHerder.java:916) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder.access$1300(DistributedHerder.java:111) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder$15.call(DistributedHerder.java:932) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder$15.call(DistributedHerder.java:928) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > {code} > I'm using official Confluent Docker images. -- This message was sent by Atlassian Jira (v8.3.4#803005)