[
https://issues.apache.org/jira/browse/FLINK-20008?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17227248#comment-17227248
]
Till Rohrmann commented on FLINK-20008:
---------------------------------------
The problem is that {{curator.framework.leader.LeaderLatch}} calls {{isLeader}}
holding an internal lock {{a}}. {{isLeader}} will try to acquire the lock {{b}}
of {{DefaultLeaderElectionService}}. Therefore, if there is a concurrent close
operation first acquiring lock {{b}} and then trying to close
{{curator.framework.leader.LeaderLatch}} which requires lock {{a}}, we run into
the deadlock.
> Java Deadlock in
> ZooKeeperLeaderElectionTest.testZooKeeperReelectionWithReplacement()
> -------------------------------------------------------------------------------------
>
> Key: FLINK-20008
> URL: https://issues.apache.org/jira/browse/FLINK-20008
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Coordination
> Affects Versions: 1.12.0
> Reporter: Stephan Ewen
> Priority: Critical
> Labels: test-stability
> Fix For: 1.12.0
>
>
> The stack trace detects a deadlock between the testing thread and the curator
> event thread.
> Full log:
> https://dev.azure.com/sewen0794/Flink/_build/results?buildId=176&view=logs&j=6e58d712-c5cc-52fb-0895-6ff7bd56c46b&t=f30a8e80-b2cf-535c-9952-7f521a4ae374
> Relevant Stack Trace:
> {code}
> Found one Java-level deadlock:
> =============================
> "main-EventThread":
> waiting to lock monitor 0x00007f74c00045e8 (object 0x000000008ed14cb0, a
> java.lang.Object),
> which is held by "main"
> "main":
> waiting to lock monitor 0x00007f74e401a1f8 (object 0x000000008ed15008, a
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch),
> which is held by "main-EventThread"
> Java stack information for the threads listed above:
> ===================================================
> "main-EventThread":
> at
> org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService.onGrantLeadership(DefaultLeaderElectionService.java:186)
> - waiting to lock <0x000000008ed14cb0> (a java.lang.Object)
> at
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver.isLeader(ZooKeeperLeaderElectionDriver.java:158)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch$9.apply(LeaderLatch.java:693)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch$9.apply(LeaderLatch.java:689)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:100)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.shaded.com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:92)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch.setLeadership(LeaderLatch.java:688)
> - locked <0x000000008ed15008> (a
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch.checkLeadership(LeaderLatch.java:567)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch.access$700(LeaderLatch.java:65)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch$7.processResult(LeaderLatch.java:618)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.imps.CuratorFrameworkImpl.sendToBackgroundCallback(CuratorFrameworkImpl.java:883)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.imps.CuratorFrameworkImpl.processBackgroundOperation(CuratorFrameworkImpl.java:653)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.imps.WatcherRemovalFacade.processBackgroundOperation(WatcherRemovalFacade.java:152)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.imps.GetChildrenBuilderImpl$2.processResult(GetChildrenBuilderImpl.java:187)
> at
> org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:601)
> at
> org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
> "main":
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch.close(LeaderLatch.java:203)
> - waiting to lock <0x000000008ed15008> (a
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch.close(LeaderLatch.java:190)
> at
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver.close(ZooKeeperLeaderElectionDriver.java:140)
> at
> org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService.stop(DefaultLeaderElectionService.java:103)
> - locked <0x000000008ed14cb0> (a java.lang.Object)
> at
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionTest.testZooKeeperReelectionWithReplacement(ZooKeeperLeaderElectionTest.java:310)
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)