You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by "Attila Doroszlai (Jira)" <ji...@apache.org> on 2022/08/26 12:41:00 UTC

[jira] [Created] (RATIS-1691) Potential deadlock in server shutdown

Attila Doroszlai created RATIS-1691:
---------------------------------------

             Summary: Potential deadlock in server shutdown
                 Key: RATIS-1691
                 URL: https://issues.apache.org/jira/browse/RATIS-1691
             Project: Ratis
          Issue Type: Bug
          Components: server
    Affects Versions: 2.4.0, 3.0.0
            Reporter: Attila Doroszlai
            Assignee: Attila Doroszlai


Ozone integration test with Ratis 2.4.0 (rc0) is timing out during shutdown.  There seems to be a sort of deadlock.

# main thread initiates shutdown of SCM Ratis server ({{RaftServerProxy#close}}, which triggers {{RaftServerImpl#close}} in {{RaftServerProxy#executor}} thread, then waits for termination of the executor
# {{RaftServerImpl#close}}, running in {{RaftServerProxy#executor}} thread, [waits for {{StateMachineUpdater}} thread to exit|https://github.com/apache/ratis/blob/ebb39e840e6d13245f76a044c1d60bafc7ca44cf/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java#L155]
# {{StateMachineUpdater}} thread during stop also triggers {{RaftServerProxy.close}} (HDDS-6141).  Most of {{RaftServerProxy#close}} is executed only once due to lifecycle state check.  Thus this thread fasttracks to shutting down and waiting for {{RaftServerProxy#executor}}.

The deadlock is that {{StateMachineUpdater}} thread can only exit after {{RaftServerProxy#executor}} is shutdown, but the {{RaftServerImpl#close}} running in that executor is waiting for the {{StateMachineUpdater}} to exit.

Prior to RATIS-1656, {{RaftServerImpl#close}} was invoked in the JDK's {{asyncPool}} executor.  Thus {{RaftServerProxy#executor}} could shutdown and {{StateMachineUpdater}} could exit.

This may be fixed by reverting this part of RATIS-1656 (going back to {{asyncPool}}, but I would like to consider moving shutdown of {{RaftServerProxy#executor}} inside the block which is executed after lifecycle check, i.e. make it "only once".

Threads from Ozone integration test:

{code:title=main}
"Listener at 127.0.0.1/58734" #1 prio=5 os_prio=31 cpu=8643.94ms elapsed=257.10s tid=0x00007fb3ae809800 nid=0x2103 waiting on condition  [0x000070000c89f000]
   java.lang.Thread.State: TIMED_WAITING (parking)
	at jdk.internal.misc.Unsafe.park(java.base@11.0.7/Native Method)
	- parking to wait for  <0x00000007818beda8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
	at java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.7/LockSupport.java:234)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(java.base@11.0.7/AbstractQueuedSynchronizer.java:2123)
	at java.util.concurrent.ThreadPoolExecutor.awaitTermination(java.base@11.0.7/ThreadPoolExecutor.java:1454)
	at org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:128)
	at org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:120)
	at org.apache.ratis.server.impl.RaftServerProxy.close(RaftServerProxy.java:421)
	at org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl.stop(SCMRatisServerImpl.java:238)
	at org.apache.hadoop.hdds.scm.ha.SCMHAManagerImpl.stop(SCMHAManagerImpl.java:339)
	at org.apache.hadoop.hdds.scm.server.StorageContainerManager.stop(StorageContainerManager.java:1655)
	at org.apache.hadoop.ozone.MiniOzoneHAClusterImpl.stop(MiniOzoneHAClusterImpl.java:321)
{code}

{code:title=StateMachineUpdater}
"ece6ad7b-20b9-4fbc-a0f7-4977d31a566d@group-9AF6C29D7499-StateMachineUpdater" #66 daemon prio=5 os_prio=31 cpu=97.47ms elapsed=244.74s tid=0x00007fb3ac86e800 nid=0x14503 waiting on condition  [0x00007000100d9000]
   java.lang.Thread.State: TIMED_WAITING (parking)
	at jdk.internal.misc.Unsafe.park(java.base@11.0.7/Native Method)
	- parking to wait for  <0x00000007818beda8> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
	at java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.7/LockSupport.java:234)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(java.base@11.0.7/AbstractQueuedSynchronizer.java:2123)
	at java.util.concurrent.ThreadPoolExecutor.awaitTermination(java.base@11.0.7/ThreadPoolExecutor.java:1454)
	at org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:128)
	at org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:120)
	at org.apache.ratis.server.impl.RaftServerProxy.close(RaftServerProxy.java:421)
	at org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl.stop(SCMRatisServerImpl.java:238)
	at org.apache.hadoop.hdds.scm.ha.SCMHAManagerImpl.stop(SCMHAManagerImpl.java:339)
	at org.apache.hadoop.hdds.scm.server.StorageContainerManager.stop(StorageContainerManager.java:1655)
	at org.apache.hadoop.hdds.scm.server.StorageContainerManager.shutDown(StorageContainerManager.java:1679)
	at org.apache.hadoop.hdds.scm.ha.SCMStateMachine.close(SCMStateMachine.java:403)
	at org.apache.ratis.server.impl.StateMachineUpdater.stop(StateMachineUpdater.java:134)
	at org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:187)
{code}

{code:title=RaftServerProxy#executor}
"ece6ad7b-20b9-4fbc-a0f7-4977d31a566d-impl-thread1" #41 prio=5 os_prio=31 cpu=20.09ms elapsed=246.40s tid=0x00007fb3af1de000 nid=0x8713 in Object.wait()  [0x000070000e67c000]
   java.lang.Thread.State: WAITING (on object monitor)
	at java.lang.Object.wait(java.base@11.0.7/Native Method)
	- waiting on <no object reference available>
	at java.lang.Thread.join(java.base@11.0.7/Thread.java:1305)
	- waiting to re-lock in wait() <0x00000007819b1f10> (a org.apache.ratis.util.Daemon)
	at java.lang.Thread.join(java.base@11.0.7/Thread.java:1379)
	at org.apache.ratis.server.impl.StateMachineUpdater.stopAndJoin(StateMachineUpdater.java:155)
	at org.apache.ratis.server.impl.ServerState.close(ServerState.java:461)
	at org.apache.ratis.server.impl.RaftServerImpl.lambda$close$4(RaftServerImpl.java:473)
	at org.apache.ratis.server.impl.RaftServerImpl$$Lambda$766/0x000000080041c440.run(Unknown Source)
	at org.apache.ratis.util.LifeCycle.lambda$checkStateAndClose$4(LifeCycle.java:299)
	at org.apache.ratis.util.LifeCycle$$Lambda$733/0x000000080040d440.get(Unknown Source)
	at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:319)
	at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:297)
	at org.apache.ratis.server.impl.RaftServerImpl.close(RaftServerImpl.java:450)
	at org.apache.ratis.server.impl.RaftServerProxy$ImplMap.close(RaftServerProxy.java:132)
	at org.apache.ratis.server.impl.RaftServerProxy$ImplMap.lambda$close$0(RaftServerProxy.java:119)
	at org.apache.ratis.server.impl.RaftServerProxy$ImplMap$$Lambda$764/0x000000080041bc40.accept(Unknown Source)
	at org.apache.ratis.util.ConcurrentUtils.accept(ConcurrentUtils.java:173)
	at org.apache.ratis.util.ConcurrentUtils.lambda$null$3(ConcurrentUtils.java:165)
	at org.apache.ratis.util.ConcurrentUtils$$Lambda$502/0x00000008002e4840.run(Unknown Source)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.7/ThreadPoolExecutor.java:1128)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.7/ThreadPoolExecutor.java:628)
	at java.lang.Thread.run(java.base@11.0.7/Thread.java:834)
{code}



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