[
https://issues.apache.org/jira/browse/HDDS-10643?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Ivan Andika updated HDDS-10643:
-------------------------------
Target Version/s: 2.0.0, 1.4.2
> SCM fails to stop gracefully
> ----------------------------
>
> Key: HDDS-10643
> URL: https://issues.apache.org/jira/browse/HDDS-10643
> Project: Apache Ozone
> Issue Type: Bug
> Reporter: Ashish Kumar
> Assignee: Ashish Kumar
> Priority: Major
> Labels: pull-request-available
> Fix For: 2.0.0
>
>
> When SCM stop is called, SCM first stops raft server which internally invokes
> state machine to
> [close|https://github.com/apache/ozone/blob/3467db1b1cc581a21caeb8648587fcbf35bbfdfa/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMStateMachine.java#L444].
>
> Since this call is from ratis, statemachine tries to
> [terminate|https://github.com/apache/ozone/blob/3467db1b1cc581a21caeb8648587fcbf35bbfdfa/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java#L1796]
> SCM by invoking [System.exit
> |https://github.com/apache/ratis/blob/f40424422b692349b5496ee30e24335c8186093b/ratis-common/src/main/java/org/apache/ratis/util/ExitUtils.java#L138]call.
>
> But System.exit waits for shutdown-hook to gets completed. Shutdown-hook is
> waiting for main thread(waiting for raft server close) to complete. So both
> threads are waiting for each other and leads to deadlock situation.
> It depends on the timeout for shutdown-hook after that it interrupts the
> thread abruptly.
> Currently after this SCM shuts down which is not graceful exit.
> Below is the stack:
> StateMachineUpdater:
> {code:java}
> "fa8d8513-6849-46f1-8088-e091f305efda@group-4D11F4CFC172-StateMachineUpdater"
> #58 daemon prio=5 os_prio=0 cpu=520.27ms elapsed=820.33s
> tid=0x00007fdc96449000 nid=0x188160 waiting for monitor entry
> [0x00007fdc60bd6000]
> java.lang.Thread.State: BLOCKED (on object monitor)
> at java.lang.Shutdown.exit([email protected]/Shutdown.java:173)
> waiting to lock <0x000000072d15c608> (a java.lang.Class for
> java.lang.Shutdown)
> at java.lang.Runtime.exit([email protected]/Runtime.java:115)
> at java.lang.System.exit([email protected]/System.java:1746)
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:138)
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:151)
> at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:155)
> at
> org.apache.hadoop.hdds.scm.server.StorageContainerManager.shutDown(StorageContainerManager.java:1758)
> at
> org.apache.hadoop.hdds.scm.ha.SCMStateMachine.close(SCMStateMachine.java:439)
> at
> org.apache.ratis.server.impl.StateMachineUpdater.stop(StateMachineUpdater.java:134)
> at
> org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:189)
> at java.lang.Thread.run([email protected]/Thread.java:834) {code}
> -
>
> Shutdown-Hook:
> {code:java}
> "SIGTERM handler" #468 daemon prio=9 os_prio=0 cpu=1.24ms elapsed=20.21s
> tid=0x0000000000e99000 nid=0x18af9c in Object.wait() [0x00007fdc4fd00000]
> java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait([email protected]/Native Method)
> - waiting on <no object reference available>
> at java.lang.Thread.join([email protected]/Thread.java:1305)
> - waiting to re-lock in wait() <0x000000071165a3f8> (a
> org.apache.hadoop.ozone.util.ShutdownHookManager$1)
> at java.lang.Thread.join([email protected]/Thread.java:1379)
> at
> java.lang.ApplicationShutdownHooks.runHooks([email protected]/ApplicationShutdownHooks.java:107)
> at
> java.lang.ApplicationShutdownHooks$1.run([email protected]/ApplicationShutdownHooks.java:46)
> at java.lang.Shutdown.runHooks([email protected]/Shutdown.java:130)
> at java.lang.Shutdown.exit([email protected]/Shutdown.java:174)
> - locked <0x000000072d15c608> (a java.lang.Class for java.lang.Shutdown)
> at java.lang.Terminator$1.handle([email protected]/Terminator.java:51)
> at
> sun.misc.Signal$SunMiscHandler.handle([email protected]/Signal.java:228)
> at
> org.apache.hadoop.hdds.utils.SignalLogger$Handler.handle(SignalLogger.java:62)
> at
> sun.misc.Signal$InternalMiscHandler.handle([email protected]/Signal.java:198)
> at jdk.internal.misc.Signal$1.run([email protected]/Signal.java:220)
> at java.lang.Thread.run([email protected]/Thread.java:834){code}
>
> Raftserver stop:
> {code:java}
> "shutdown-hook-0" #469 daemon prio=5 os_prio=0 cpu=59.00ms elapsed=20.19s
> tid=0x00007fdc795a6800 nid=0x18afa0 waiting on condition [0x00007fdc46579000]
> java.lang.Thread.State: TIMED_WAITING (parking)
> at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
> - parking to wait for <0x000000072c928090> (a
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at
> java.util.concurrent.locks.LockSupport.parkNanos([email protected]/LockSupport.java:234)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos([email protected]/AbstractQueuedSynchronizer.java:2123)
> at
> java.util.concurrent.ThreadPoolExecutor.awaitTermination([email protected]/ThreadPoolExecutor.java:1454)
> at
> org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:143)
> at
> org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:135)
> at
> org.apache.ratis.server.impl.RaftServerProxy.lambda$close$6(RaftServerProxy.java:432)
> at
> org.apache.ratis.server.impl.RaftServerProxy$$Lambda$1122/0x0000000800a7e440.run(Unknown
> Source)
> at
> org.apache.ratis.util.LifeCycle.lambda$checkStateAndClose$4(LifeCycle.java:299)
> at
> org.apache.ratis.util.LifeCycle$$Lambda$892/0x0000000800960c40.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.RaftServerProxy.close(RaftServerProxy.java:415)
> at
> org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl.stop(SCMRatisServerImpl.java:255)
> at
> org.apache.hadoop.hdds.scm.ha.SCMHAManagerImpl.stop(SCMHAManagerImpl.java:392)
> at
> org.apache.hadoop.hdds.scm.server.StorageContainerManager.stop(StorageContainerManager.java:1705)
> at
> org.apache.hadoop.hdds.scm.server.StorageContainerManagerStarter$SCMStarterHelper.lambda$start$0(StorageContainerManagerStarter.java:175)
> at
> org.apache.hadoop.hdds.scm.server.StorageContainerManagerStarter$SCMStarterHelper$$Lambda$726/0x0000000800817840.run(Unknown
> Source)
> at
> java.util.concurrent.Executors$RunnableAdapter.call([email protected]/Executors.java:515)
> at
> java.util.concurrent.FutureTask.run([email protected]/FutureTask.java:264)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1128)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:628)
> at java.lang.Thread.run([email protected]/Thread.java:834){code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]