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

Matthias Pohl commented on FLINK-32311:
---------------------------------------

Ok, I did go through the code comparing the {{DefaultLeaderElectionService}} 
with the {{DefaultMultipleComponentLeaderElectionService}} in that regards: The 
actual deadlock in the test happens between the ZK LeaderLatch lock in the 
curator's event thread and the test code's thread calling the 
{{DefaultLeaderElectionService#close()}} which is guarded by the 
{{DefaultLeaderElectionService}}'s lock. With FLINK-31733, we split up the 
driver lifecycle and the contender lifecycle and moved the driver's closing 
into the lock's monitor region (because we wanted the driver to be shutdown 
along the {{leaderOperationExecutor}} which processes the leader events to 
avoid {{RejectedExecutionException}}s from happening). This caused the 
concurrent nested lock scenario for the legacy driver to happen (as we see it 
being documented in the Jira issue).

Now the question is whether or why it didn't appear in the production code 
where we use the FLINK-24038 classes, i.e. 
{{MultipleComponentLeaderElectionDriverAdapter}} and 
{{DefaultMultipleComponentLeaderElectionService}}. The latter one does also 
close its driver implementation within the locks (analogously to how it's done 
in the current {{DefaultLeaderElectionService}} implementation). The difference 
is that the {{DefaultMultipleComponentLeaderElectionService}} owns its own 
lock. This prevents both threads from trying to own the same two locks.

Therefore, my conclusion is that it's a test code issue right now. But we have 
to address if we want to continue with FLINK-26522.

> ZooKeeperLeaderElectionTest.testZooKeeperReelectionWithReplacement and 
> DefaultLeaderElectionService.onGrantLeadership fell into dead lock
> -----------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-32311
>                 URL: https://issues.apache.org/jira/browse/FLINK-32311
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.18.0
>            Reporter: Sergey Nuyanzin
>            Assignee: Matthias Pohl
>            Priority: Critical
>              Labels: test-stability
>
> [https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=49750&view=logs&j=0da23115-68bb-5dcd-192c-bd4c8adebde1&t=24c3384f-1bcb-57b3-224f-51bf973bbee8]
>  
> there are 2 threads one locked {{0x00000000e3a8a1e8}} and waiting for 
> {{0x00000000e3a89c18}}
> {noformat}
> 2023-06-08T01:18:54.5609123Z Jun 08 01:18:54 
> "ForkJoinPool-50-worker-25-EventThread" #956 daemon prio=5 os_prio=0 
> tid=0x00007f9374253800 nid=0x6a4e waiting for monitor entry 
> [0x00007f94b63e1000]
> 2023-06-08T01:18:54.5609820Z Jun 08 01:18:54    java.lang.Thread.State: 
> BLOCKED (on object monitor)
> 2023-06-08T01:18:54.5610557Z Jun 08 01:18:54  at 
> org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService.runInLeaderEventThread(DefaultLeaderElectionService.java:425)
> 2023-06-08T01:18:54.5611459Z Jun 08 01:18:54  - waiting to lock 
> <0x00000000e3a89c18> (a java.lang.Object)
> 2023-06-08T01:18:54.5612198Z Jun 08 01:18:54  at 
> org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService.onGrantLeadership(DefaultLeaderElectionService.java:300)
> 2023-06-08T01:18:54.5613110Z Jun 08 01:18:54  at 
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver.isLeader(ZooKeeperLeaderElectionDriver.java:153)
> 2023-06-08T01:18:54.5614070Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch$$Lambda$1649/586959400.accept(Unknown
>  Source)
> 2023-06-08T01:18:54.5615014Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.listen.MappingListenerManager.lambda$forEach$0(MappingListenerManager.java:92)
> 2023-06-08T01:18:54.5616259Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.listen.MappingListenerManager$$Lambda$1640/1393625763.run(Unknown
>  Source)
> 2023-06-08T01:18:54.5617137Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.listen.MappingListenerManager$$Lambda$1633/2012730699.execute(Unknown
>  Source)
> 2023-06-08T01:18:54.5618047Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.listen.MappingListenerManager.forEach(MappingListenerManager.java:89)
> 2023-06-08T01:18:54.5618994Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.listen.StandardListenerManager.forEach(StandardListenerManager.java:89)
> 2023-06-08T01:18:54.5620071Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch.setLeadership(LeaderLatch.java:711)
> 2023-06-08T01:18:54.5621198Z Jun 08 01:18:54  - locked <0x00000000e3a8a1e8> 
> (a 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch)
> 2023-06-08T01:18:54.5622072Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch.checkLeadership(LeaderLatch.java:597)
> 2023-06-08T01:18:54.5622991Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch.access$600(LeaderLatch.java:64)
> 2023-06-08T01:18:54.5623988Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch$7.processResult(LeaderLatch.java:648)
> 2023-06-08T01:18:54.5624965Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.CuratorFrameworkImpl.sendToBackgroundCallback(CuratorFrameworkImpl.java:926)
> 2023-06-08T01:18:54.5626218Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.CuratorFrameworkImpl.processBackgroundOperation(CuratorFrameworkImpl.java:683)
> 2023-06-08T01:18:54.5627369Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.WatcherRemovalFacade.processBackgroundOperation(WatcherRemovalFacade.java:152)
> 2023-06-08T01:18:54.5628353Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.imps.GetChildrenBuilderImpl$2.processResult(GetChildrenBuilderImpl.java:187)
> 2023-06-08T01:18:54.5629281Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:666)
> 2023-06-08T01:18:54.5630124Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:553)
> {noformat}
> and another locked {{0x00000000e3a89c18}} and waits for {{0x00000000e3a8a1e8}}
> {noformat}
> 2023-06-08T01:18:54.5738286Z Jun 08 01:18:54 "ForkJoinPool-50-worker-25" #620 
> daemon prio=5 os_prio=0 tid=0x00007f953874f000 nid=0x682e waiting for monitor 
> entry [0x00007f95461d4000]
> 2023-06-08T01:18:54.5738959Z Jun 08 01:18:54    java.lang.Thread.State: 
> BLOCKED (on object monitor)
> 2023-06-08T01:18:54.5739645Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch.close(LeaderLatch.java:203)
> 2023-06-08T01:18:54.5740731Z Jun 08 01:18:54  - waiting to lock 
> <0x00000000e3a8a1e8> (a 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch)
> 2023-06-08T01:18:54.5741591Z Jun 08 01:18:54  at 
> org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch.close(LeaderLatch.java:190)
> 2023-06-08T01:18:54.5742609Z Jun 08 01:18:54  at 
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriver.close(ZooKeeperLeaderElectionDriver.java:135)
> 2023-06-08T01:18:54.5743491Z Jun 08 01:18:54  at 
> org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService.close(DefaultLeaderElectionService.java:217)
> 2023-06-08T01:18:54.5744427Z Jun 08 01:18:54  - locked <0x00000000e3a89c18> 
> (a java.lang.Object)
> 2023-06-08T01:18:54.5745200Z Jun 08 01:18:54  at 
> org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionTest.testZooKeeperReelectionWithReplacement(ZooKeeperLeaderElectionTest.java:346)
> 2023-06-08T01:18:54.5746206Z Jun 08 01:18:54  at 
> sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 2023-06-08T01:18:54.5746829Z Jun 08 01:18:54  at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 2023-06-08T01:18:54.5747552Z Jun 08 01:18:54  at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 2023-06-08T01:18:54.5748207Z Jun 08 01:18:54  at 
> java.lang.reflect.Method.invoke(Method.java:498)
> ...
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to