[
https://issues.apache.org/jira/browse/HDDS-11124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17864370#comment-17864370
]
Hemant Kumar edited comment on HDDS-11124 at 7/9/24 8:35 PM:
-------------------------------------------------------------
There is a deadlock between the DoubleBuffer and the StateMachine. Because of
snapshot dir is not getting created because it happens inside the double buffer
flush thread.
1. The double buffer flush thread is waiting on Delete_Table lock
[OmSnapshotManager#createOmSnapshotCheckpoint()|https://github.com/apache/ozone/blob/3e97d8fd6aa21521b4b22b43e9873376fd0a47c4/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java#L461].
{code:java}
"om229-OMDoubleBufferFlushThread" #8341 daemon prio=5 os_prio=0
tid=0x00007f7ec4683000 nid=0x8c31 waiting on condition [0x00007f7ecd6e9000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f7fd60cc5e8> (a
java.util.concurrent.locks.ReentrantReadWriteLock$FairSync)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
at
java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943)
at
org.apache.hadoop.ozone.om.OmSnapshotManager.createOmSnapshotCheckpoint(OmSnapshotManager.java:462)
at
org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotCreateResponse.addToDBBatch(OMSnapshotCreateResponse.java:81)
at
org.apache.hadoop.ozone.om.response.OMClientResponse.checkAndUpdateDB(OMClientResponse.java:66)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.lambda$8(OzoneManagerDoubleBuffer.java:408)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer$$Lambda$674/1034433827.run(Unknown
Source)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.addToBatchWithTrace(OzoneManagerDoubleBuffer.java:253)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.addToBatch(OzoneManagerDoubleBuffer.java:407)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.flushBatch(OzoneManagerDoubleBuffer.java:353)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.flushCurrentBuffer(OzoneManagerDoubleBuffer.java:328)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.flushTransactions(OzoneManagerDoubleBuffer.java:295)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer$$Lambda$331/996688653.run(Unknown
Source){code}
2. Delete_Table lock is currently held by
[KeyDeletingService#call|https://github.com/apache/ozone/blob/2f2234c7b61714404399ada8f31b3fb4772b613a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java#L209]
and it is waiting for future completion.
{code:java}
"om229-KeyDeletingService#0" #8902 daemon prio=5 os_prio=0
tid=0x00007f8f62669800 nid=0x8c7f waiting on condition [0x00007f7ec0dfb000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f7fe1909168> (a
java.util.concurrent.CompletableFuture$Signaller)
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.get(CompletableFuture.java:1908)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequestToRatisImpl(OzoneManagerRatisServer.java:320)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.lambda$3(OzoneManagerRatisServer.java:292)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer$$Lambda$1088/1425223398.get(Unknown
Source)
at org.apache.hadoop.util.MetricUtil.captureLatencyNs(MetricUtil.java:45)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequestToRatis(OzoneManagerRatisServer.java:290)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequest(OzoneManagerRatisServer.java:312)
at
org.apache.hadoop.ozone.om.service.AbstractKeyDeletingService.submitPurgeKeysRequest(AbstractKeyDeletingService.java:252)
at
org.apache.hadoop.ozone.om.service.AbstractKeyDeletingService.processKeyDeletes(AbstractKeyDeletingService.java:124)
at
org.apache.hadoop.ozone.om.service.KeyDeletingService$KeyDeletingTask.call(KeyDeletingService.java:209)
at
org.apache.hadoop.hdds.utils.BackgroundService$PeriodicalTask.lambda$run$0(BackgroundService.java:140)
at
org.apache.hadoop.hdds.utils.BackgroundService$PeriodicalTask$$Lambda$245/1676298842.run(Unknown
Source)
at
java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1640)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
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:748)
{code}
3. Finally StateMachine is waiting for the double buffer to flush before
applying the transaction:
{code:java}
"om229@group-ACB43EDE43F3-StateMachineUpdater" #8361 daemon prio=5 os_prio=0
tid=0x00007f8f62fb9800 nid=0x8c5f waiting on condition [0x00007f7ecb3c4000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f7fd7dd50e0> (a
java.util.concurrent.Semaphore$NonfairSync)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
at java.util.concurrent.Semaphore.acquire(Semaphore.java:467)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.acquireUnFlushedTransactions(OzoneManagerDoubleBuffer.java:234)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine.applyTransaction(OzoneManagerStateMachine.java:355)
at
org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:1878)
at
org.apache.ratis.server.impl.StateMachineUpdater.applyLog(StateMachineUpdater.java:244)
at
org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:186)
at java.lang.Thread.run(Thread.java:748)
{code}
4. StateMachince pause is causing all the request handler threads to wait.
{code:java}
"IPC Server handler 99 on 9862" #9927 daemon prio=5 os_prio=0
tid=0x00007f7ec4039800 nid=0x8de7 waiting on condition [0x00007f7eb5098000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f7fe69f0788> (a
java.util.concurrent.CompletableFuture$Signaller)
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.get(CompletableFuture.java:1908)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequestToRatisImpl(OzoneManagerRatisServer.java:320)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.lambda$3(OzoneManagerRatisServer.java:292)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer$$Lambda$1088/1425223398.get(Unknown
Source)
at org.apache.hadoop.util.MetricUtil.captureLatencyNs(MetricUtil.java:45)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequestToRatis(OzoneManagerRatisServer.java:290)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequest(OzoneManagerRatisServer.java:262)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequestToRatis(OzoneManagerProtocolServerSideTranslatorPB.java:252)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.internalProcessRequest(OzoneManagerProtocolServerSideTranslatorPB.java:226)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:161)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB$$Lambda$1005/219824061.apply(Unknown
Source)
at
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:89)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:152)
at
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos$OzoneManagerService$2.callBlockingMethod(OzoneManagerProtocolProtos.java)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:533)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1070)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:994)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:922)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2899){code}
Attached the complete jstack and analysis:
[https://fastthread.io/threads.jsp?traceId=30&pc&dumpId=1&type=rs&ts=2024-07-09T13-28-44]
Original change which introduced the lock: HDDS-8067
was (Author: JIRAUSER297350):
There is a deadlock between the DoubleBuffer and the StateMachine. Because of
snapshot dir is not getting created because it happens inside the double buffer
flush thread.
1. The double buffer flush thread is waiting on Delete_Table lock
[OmSnapshotManager#createOmSnapshotCheckpoint()|https://github.com/apache/ozone/blob/3e97d8fd6aa21521b4b22b43e9873376fd0a47c4/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java#L461].
{code:java}
"om229-OMDoubleBufferFlushThread" #8341 daemon prio=5 os_prio=0
tid=0x00007f7ec4683000 nid=0x8c31 waiting on condition [0x00007f7ecd6e9000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f7fd60cc5e8> (a
java.util.concurrent.locks.ReentrantReadWriteLock$FairSync)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
at
java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943)
at
org.apache.hadoop.ozone.om.OmSnapshotManager.createOmSnapshotCheckpoint(OmSnapshotManager.java:462)
at
org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotCreateResponse.addToDBBatch(OMSnapshotCreateResponse.java:81)
at
org.apache.hadoop.ozone.om.response.OMClientResponse.checkAndUpdateDB(OMClientResponse.java:66)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.lambda$8(OzoneManagerDoubleBuffer.java:408)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer$$Lambda$674/1034433827.run(Unknown
Source)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.addToBatchWithTrace(OzoneManagerDoubleBuffer.java:253)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.addToBatch(OzoneManagerDoubleBuffer.java:407)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.flushBatch(OzoneManagerDoubleBuffer.java:353)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.flushCurrentBuffer(OzoneManagerDoubleBuffer.java:328)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.flushTransactions(OzoneManagerDoubleBuffer.java:295)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer$$Lambda$331/996688653.run(Unknown
Source){code}
2. Delete_Table lock is currently held by
[KeyDeletingService#call|https://github.com/apache/ozone/blob/2f2234c7b61714404399ada8f31b3fb4772b613a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java#L209]
and it is waiting for future completion.
{code:java}
"om229-KeyDeletingService#0" #8902 daemon prio=5 os_prio=0
tid=0x00007f8f62669800 nid=0x8c7f waiting on condition [0x00007f7ec0dfb000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f7fe1909168> (a
java.util.concurrent.CompletableFuture$Signaller)
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.get(CompletableFuture.java:1908)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequestToRatisImpl(OzoneManagerRatisServer.java:320)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.lambda$3(OzoneManagerRatisServer.java:292)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer$$Lambda$1088/1425223398.get(Unknown
Source)
at org.apache.hadoop.util.MetricUtil.captureLatencyNs(MetricUtil.java:45)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequestToRatis(OzoneManagerRatisServer.java:290)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequest(OzoneManagerRatisServer.java:312)
at
org.apache.hadoop.ozone.om.service.AbstractKeyDeletingService.submitPurgeKeysRequest(AbstractKeyDeletingService.java:252)
at
org.apache.hadoop.ozone.om.service.AbstractKeyDeletingService.processKeyDeletes(AbstractKeyDeletingService.java:124)
at
org.apache.hadoop.ozone.om.service.KeyDeletingService$KeyDeletingTask.call(KeyDeletingService.java:209)
at
org.apache.hadoop.hdds.utils.BackgroundService$PeriodicalTask.lambda$run$0(BackgroundService.java:140)
at
org.apache.hadoop.hdds.utils.BackgroundService$PeriodicalTask$$Lambda$245/1676298842.run(Unknown
Source)
at
java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1640)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
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:748)
{code}
3. Finally StateMachine is waiting for the double buffer to flush before
applying the transaction:
{code:java}
"om229@group-ACB43EDE43F3-StateMachineUpdater" #8361 daemon prio=5 os_prio=0
tid=0x00007f8f62fb9800 nid=0x8c5f waiting on condition [0x00007f7ecb3c4000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f7fd7dd50e0> (a
java.util.concurrent.Semaphore$NonfairSync)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
at java.util.concurrent.Semaphore.acquire(Semaphore.java:467)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerDoubleBuffer.acquireUnFlushedTransactions(OzoneManagerDoubleBuffer.java:234)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine.applyTransaction(OzoneManagerStateMachine.java:355)
at
org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:1878)
at
org.apache.ratis.server.impl.StateMachineUpdater.applyLog(StateMachineUpdater.java:244)
at
org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:186)
at java.lang.Thread.run(Thread.java:748)
{code}
4. StateMachince pause is causing all the request handler threads to wait.
{code:java}
"IPC Server handler 99 on 9862" #9927 daemon prio=5 os_prio=0
tid=0x00007f7ec4039800 nid=0x8de7 waiting on condition [0x00007f7eb5098000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00007f7fe69f0788> (a
java.util.concurrent.CompletableFuture$Signaller)
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.get(CompletableFuture.java:1908)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequestToRatisImpl(OzoneManagerRatisServer.java:320)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.lambda$3(OzoneManagerRatisServer.java:292)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer$$Lambda$1088/1425223398.get(Unknown
Source)
at org.apache.hadoop.util.MetricUtil.captureLatencyNs(MetricUtil.java:45)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequestToRatis(OzoneManagerRatisServer.java:290)
at
org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.submitRequest(OzoneManagerRatisServer.java:262)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequestToRatis(OzoneManagerProtocolServerSideTranslatorPB.java:252)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.internalProcessRequest(OzoneManagerProtocolServerSideTranslatorPB.java:226)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:161)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB$$Lambda$1005/219824061.apply(Unknown
Source)
at
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:89)
at
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:152)
at
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos$OzoneManagerService$2.callBlockingMethod(OzoneManagerProtocolProtos.java)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:533)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1070)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:994)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:922)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2899){code}
Attached the complete jstack and analysis:
https://fastthread.io/threads.jsp?traceId=30&pc&dumpId=1&type=rs&ts=2024-07-09T13-28-44
> Snapshot create requests failing with OM failover error in a system with
> 30000 snapshots
> ----------------------------------------------------------------------------------------
>
> Key: HDDS-11124
> URL: https://issues.apache.org/jira/browse/HDDS-11124
> Project: Apache Ozone
> Issue Type: Bug
> Components: OM, Snapshot
> Reporter: Jyotirmoy Sinha
> Priority: Major
> Labels: ozone-snapshot
> Attachments: vc0729-jstack.txt
>
>
> Snapshot create requests failing with OM failover error in a system with
> 30000 snapshots
> Console error -
> {code:java}
> # ozone sh snapshot create voltest21719585314/buck1 snaptest2
> 24/07/09 01:56:23 INFO retry.RetryInvocationHandler:
> com.google.protobuf.ServiceException: java.net.SocketTimeoutException: Call
> From vc0725.halxg.cloudera.com/10.17.213.35 to vc0729.halxg.cloudera.com:9862
> failed on socket timeout exception: java.net.SocketTimeoutException: 60000
> millis timeout while waiting for channel to be ready for read. ch :
> java.nio.channels.SocketChannel[connected local=/10.17.213.35:45466
> remote=vc0729.halxg.cloudera.com/10.17.213.39:9862]; For more details see:
> http://wiki.apache.org/hadoop/SocketTimeout, while invoking
> $Proxy19.submitRequest over
> nodeId=om229,nodeAddress=vc0729.halxg.cloudera.com:9862. Trying to failover
> after sleeping for 2000ms.
> 24/07/09 01:56:25 INFO retry.RetryInvocationHandler:
> com.google.protobuf.ServiceException:
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException):
> OM:om228 is not the leader. Suggested leader is
> OM:om229[vc0729.halxg.cloudera.com/10.17.213.39].
> at
> org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.newOMNotLeaderException(OzoneManagerRatisServer.java:780)
> at
> org.apache.hadoop.ozone.om.OzoneManager.checkLeaderStatus(OzoneManager.java:4156)
> at
> org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils.checkLeaderStatus(OzoneManagerRatisUtils.java:488)
> at
> org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.internalProcessRequest(OzoneManagerProtocolServerSideTranslatorPB.java:207)
> at
> org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:161)
> at
> org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:89)
> at
> org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:152)
> at
> org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos$OzoneManagerService$2.callBlockingMethod(OzoneManagerProtocolProtos.java)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:533)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1070)
> at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:994)
> at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:922)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2899)
> , while invoking $Proxy19.submitRequest over
> nodeId=om228,nodeAddress=vc0724.halxg.cloudera.com:9862 after 1 failover
> attempts. Trying to failover immediately.
> 24/07/09 01:56:25 INFO retry.RetryInvocationHandler:
> com.google.protobuf.ServiceException:
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException):
> OM:om230 is not the leader. Suggested leader is
> OM:om229[vc0729.halxg.cloudera.com/10.17.213.39].
> at
> org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer.newOMNotLeaderException(OzoneManagerRatisServer.java:780)
> at
> org.apache.hadoop.ozone.om.OzoneManager.checkLeaderStatus(OzoneManager.java:4156)
> at
> org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils.checkLeaderStatus(OzoneManagerRatisUtils.java:488)
> at
> org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.internalProcessRequest(OzoneManagerProtocolServerSideTranslatorPB.java:207)
> at
> org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:161)
> at
> org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:89)
> at
> org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:152)
> at
> org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos$OzoneManagerService$2.callBlockingMethod(OzoneManagerProtocolProtos.java)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:533)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1070)
> at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:994)
> at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:922)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2899)
> , while invoking $Proxy19.submitRequest over
> nodeId=om230,nodeAddress=vc0725.halxg.cloudera.com:9862 after 2 failover
> attempts. Trying to failover immediately. {code}
> On checking OM Logs during this command there are intermittent errors of
> "Unable to load snapshot" -
> {code:java}
> 2024-07-09 01:44:36,632 INFO
> [om229-KeyDeletingService#0]-org.apache.hadoop.hdds.utils.db.RDBCheckpointUtils:
> Checkpoint directory: 60 didn't get created in
> /var/lib/hadoop-ozone/om/data/db.snapshots/checkpointState/om.db-e3987edf-029d-41be-b2ae-f2deca2fabb0
> secs.
> 2024-07-09 01:44:36,632 ERROR
> [om229-KeyDeletingService#0]-org.apache.hadoop.ozone.om.OmSnapshotManager:
> Failed to retrieve snapshot: /voltest21719509823/buck1/snap980
> TIMEOUT org.apache.hadoop.ozone.om.exceptions.OMException: Unable to load
> snapshot. Snapshot checkpoint directory
> '/var/lib/hadoop-ozone/om/data/db.snapshots/checkpointState/om.db-e3987edf-029d-41be-b2ae-f2deca2fabb0'
> does not exist yet. Please wait a few more seconds before retrying
> at
> org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.checkSnapshotDirExist(SnapshotUtils.java:113)
> at
> org.apache.hadoop.ozone.om.OmMetadataManagerImpl.<init>(OmMetadataManagerImpl.java:413)
> at
> org.apache.hadoop.ozone.om.OmSnapshotManager$1.load(OmSnapshotManager.java:371)
> at
> org.apache.hadoop.ozone.om.OmSnapshotManager$1.load(OmSnapshotManager.java:1)
> at
> org.apache.hadoop.ozone.om.snapshot.SnapshotCache.lambda$2(SnapshotCache.java:158)
> at
> java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1892)
> at
> org.apache.hadoop.ozone.om.snapshot.SnapshotCache.get(SnapshotCache.java:154)
> at
> org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshot(OmSnapshotManager.java:690)
> at
> org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshot(OmSnapshotManager.java:678)
> at
> org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshot(OmSnapshotManager.java:661)
> at
> org.apache.hadoop.ozone.om.OmMetadataManagerImpl.getLatestActiveSnapshot(OmMetadataManagerImpl.java:1668)
> at
> org.apache.hadoop.ozone.om.OmMetadataManagerImpl.getPendingDeletionKeys(OmMetadataManagerImpl.java:1523)
> at
> org.apache.hadoop.ozone.om.KeyManagerImpl.getPendingDeletionKeys(KeyManagerImpl.java:684)
> at
> org.apache.hadoop.ozone.om.service.KeyDeletingService$KeyDeletingTask.call(KeyDeletingService.java:205)
> at
> org.apache.hadoop.hdds.utils.BackgroundService$PeriodicalTask.lambda$run$0(BackgroundService.java:140)
> at
> java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1640)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> 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:748)
> 2024-07-09 01:44:36,633 WARN
> [om229-KeyDeletingService#0]-org.apache.hadoop.hdds.utils.BackgroundService:
> Background task execution failed {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]