Details
-
Sub-task
-
Status: Open
-
Major
-
Resolution: Unresolved
-
None
-
None
-
None
-
None
Description
Cluster shutdown is waiting for executor to shutdown:
"main" java.lang.Thread.State: TIMED_WAITING at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078) at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1475) at org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:144) at org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:136) at org.apache.ratis.server.impl.RaftServerProxy.lambda$close$9(RaftServerProxy.java:438) at org.apache.ratis.server.impl.RaftServerProxy$$Lambda$1625/1179968371.run(Unknown Source) at org.apache.ratis.util.LifeCycle.lambda$checkStateAndClose$7(LifeCycle.java:306) at org.apache.ratis.util.LifeCycle$$Lambda$1127/1377650693.get(Unknown Source) at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:326) at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:304) at org.apache.ratis.server.impl.RaftServerProxy.close(RaftServerProxy.java:415) at org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.stop(OzoneManagerRatisServer.java:597) at org.apache.hadoop.ozone.om.OzoneManager.stop(OzoneManager.java:2226) at org.apache.hadoop.ozone.MiniOzoneClusterImpl.stopOM(MiniOzoneClusterImpl.java:557) at org.apache.hadoop.ozone.MiniOzoneHAClusterImpl.stop(MiniOzoneHAClusterImpl.java:311) at org.apache.hadoop.ozone.MiniOzoneClusterImpl.shutdown(MiniOzoneClusterImpl.java:453) at org.apache.hadoop.ozone.om.TestOzoneManagerHA.shutdown(TestOzoneManagerHA.java:209)
Ratis might wait up to 1 day:
ConcurrentUtils.shutdownAndWait(executor.get());
static void shutdownAndWait(ExecutorService executor) {
shutdownAndWait(TimeDuration.ONE_DAY, executor, timeout -> {
The executor is running a task that's also waiting:
"omNode-3-impl-thread1" java.lang.Thread.State: WAITING at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1707) at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323) at java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1742) at java.util.concurrent.CompletableFuture.join(CompletableFuture.java:1947) at org.apache.ratis.server.impl.RaftServerImpl.lambda$close$3(RaftServerImpl.java:543) at org.apache.ratis.server.impl.RaftServerImpl$$Lambda$1628/1400495203.run(Unknown Source) at org.apache.ratis.util.LifeCycle.lambda$checkStateAndClose$7(LifeCycle.java:306) at org.apache.ratis.util.LifeCycle$$Lambda$1127/1377650693.get(Unknown Source) at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:326) at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:304) at org.apache.ratis.server.impl.RaftServerImpl.close(RaftServerImpl.java:525) at org.apache.ratis.server.impl.RaftServerProxy$ImplMap.close(RaftServerProxy.java:136) at org.apache.ratis.server.impl.RaftServerProxy$ImplMap.lambda$close$0(RaftServerProxy.java:123) at org.apache.ratis.server.impl.RaftServerProxy$ImplMap$$Lambda$1626/677696333.accept(Unknown Source) at org.apache.ratis.util.ConcurrentUtils.accept(ConcurrentUtils.java:203) at org.apache.ratis.util.ConcurrentUtils.lambda$null$4(ConcurrentUtils.java:182) at org.apache.ratis.util.ConcurrentUtils$$Lambda$790/801783477.run(Unknown Source) 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:750)
role.shutdownLeaderState(true).join();
log
2024-01-20 18:19:01,239 [main] INFO ratis.OzoneManagerRatisServer (OzoneManagerRatisServer.java:stop(594)) - Stopping org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer@299dd381 at port 15058 2024-01-20 18:19:01,239 [omNode-3@group-523986131536-LeaderElection22] INFO segmented.SegmentedRaftLogWorker (SegmentedRaftLogWorker.java:rollLogSegment(435)) - omNode-3@group-523986131536-SegmentedRaftLogWorker: Rolling segment log-47_48 to index:48 2024-01-20 18:19:01,239 [main] INFO server.RaftServer (RaftServerProxy.java:lambda$close$9(416)) - omNode-3: close 2024-01-20 18:19:01,239 [main] INFO server.GrpcService (GrpcService.java:closeImpl(311)) - omNode-3: shutdown server GrpcServerProtocolService now 2024-01-20 18:19:01,239 [omNode-3@group-523986131536-LeaderElection22] INFO server.RaftServer$Division (ServerState.java:setRaftConf(386)) - omNode-3@group-523986131536: set configuration 49: peers:[omNode-3|localhost:15058, omNode-1|localhost:15050, omNode-2|localhost:15054]|listeners:[], old=null 2024-01-20 18:19:01,239 [omNode-3@group-523986131536-SegmentedRaftLogWorker] INFO segmented.SegmentedRaftLogWorker (SegmentedRaftLogWorker.java:execute(591)) - omNode-3@group-523986131536-SegmentedRaftLogWorker: Rolled log segment from /home/runner/work/ozone/ozone/hadoop-ozone/integration-test/target/test-dir/MiniOzoneClusterImpl-9c8bbe94-a664-4f6d-892b-2c9911aeb6c1/omNode-3/ratis/c9bc4cf4-3bc3-3c60-a66b-523986131536/current/log_inprogress_47 to /home/runner/work/ozone/ozone/hadoop-ozone/integration-test/target/test-dir/MiniOzoneClusterImpl-9c8bbe94-a664-4f6d-892b-2c9911aeb6c1/omNode-3/ratis/c9bc4cf4-3bc3-3c60-a66b-523986131536/current/log_47-48 2024-01-20 18:19:01,239 [omNode-3-impl-thread1] INFO server.RaftServer$Division (RaftServerImpl.java:lambda$close$3(526)) - omNode-3@group-523986131536: shutdown 2024-01-20 18:19:01,240 [omNode-3@group-523986131536-SegmentedRaftLogWorker] INFO segmented.BufferedWriteChannel (BufferedWriteChannel.java:open(62)) - open log_inprogress_49 at position 0 2024-01-20 18:19:01,240 [omNode-3-impl-thread1] INFO util.JmxRegister (JmxRegister.java:unregister(73)) - Successfully un-registered JMX Bean with object name Ratis:service=RaftServer,group=group-523986131536,id=omNode-3 2024-01-20 18:19:01,240 [omNode-3-impl-thread1] INFO impl.RoleInfo (RoleInfo.java:shutdownLeaderState(94)) - omNode-3: shutdown omNode-3@group-523986131536-LeaderStateImpl 2024-01-20 18:19:01,240 [omNode-3-impl-thread1] INFO impl.PendingRequests (PendingRequests.java:sendNotLeaderResponses(289)) - omNode-3@group-523986131536-PendingRequests: sendNotLeaderResponses 2024-01-20 18:19:01,247 [omNode-3@group-523986131536-SegmentedRaftLogWorker] INFO segmented.SegmentedRaftLogWorker (SegmentedRaftLogWorker.java:execute(634)) - omNode-3@group-523986131536-SegmentedRaftLogWorker: created new log segment /home/runner/work/ozone/ozone/hadoop-ozone/integration-test/target/test-dir/MiniOzoneClusterImpl-9c8bbe94-a664-4f6d-892b-2c9911aeb6c1/omNode-3/ratis/c9bc4cf4-3bc3-3c60-a66b-523986131536/current/log_inprogress_49 2024-01-20 18:19:01,260 [Thread-4860] INFO server.GrpcServerProtocolClient (GrpcServerProtocolClient.java:close(103)) - omNode-1 Close channels 2024-01-20 18:19:01,261 [Thread-4861] INFO server.GrpcServerProtocolClient (GrpcServerProtocolClient.java:close(103)) - omNode-2 Close channels 2024-01-20 18:19:01,262 [main] INFO server.GrpcService (GrpcService.java:closeImpl(320)) - omNode-3: shutdown server GrpcServerProtocolService successfully
Attachments
Attachments
Issue Links
- relates to
-
HDDS-5990 TestOzoneManagerPrepare occasionally times out after 120s
- Open
-
RATIS-1928 Join the LogAppenders when closing the server
- Resolved