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

Siyao Meng commented on HDDS-9876:
----------------------------------

Note to others that might be hitting the same issue and for searchability:

{{OMLeaderNotReadyException}} can also happen because of this bug due to OM 
leader not being able to apply the latest transactions. Even after a failover 
(i.e. killing the leader node and let one of the follower become the new 
leader), the new leader is still stuck applying the transaction.

Example of a symptom:

{code:title=Ozone client output}
24/06/20 11:57:12 INFO retry.RetryInvocationHandler: 
com.google.protobuf.ServiceException: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException):
 om1 is Leader but not ready to process request yet. at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderNotReadyException(OzoneManagerProtocolServerSideTranslatorPB.java:265)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderErrorException(OzoneManagerProtocolServerSideTranslatorPB.java:237)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitReadRequestToOM(OzoneManagerProtocolServerSideTranslatorPB.java:228)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:180)
 at 
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:149)
 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 
$Proxy18.submitRequest over 
nodeId=om1,nodeAddress=dfmwr125b01t3.nam.nsroot.net:9862 after 1 failover 
attempts. Trying to failover after sleeping for 4000ms. 24/06/20 11:57:16 INFO 
retry.RetryInvocationHandler: com.google.protobuf.ServiceException: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException):
 om1 is Leader but not ready to process request yet. at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderNotReadyException(OzoneManagerProtocolServerSideTranslatorPB.java:265)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderErrorException(OzoneManagerProtocolServerSideTranslatorPB.java:237)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitReadRequestToOM(OzoneManagerProtocolServerSideTranslatorPB.java:228)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:180)
 at 
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:149)
 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 
$Proxy18.submitRequest over 
nodeId=om1,nodeAddress=dfmwr125b01t3.nam.nsroot.net:9862 after 2 failover 
attempts. Trying to failover after sleeping for 6000ms. 24/06/20 11:57:22 INFO 
retry.RetryInvocationHandler: com.google.protobuf.ServiceException: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException):
 om1 is Leader but not ready to process request yet. at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderNotReadyException(OzoneManagerProtocolServerSideTranslatorPB.java:265)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderErrorException(OzoneManagerProtocolServerSideTranslatorPB.java:237)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitReadRequestToOM(OzoneManagerProtocolServerSideTranslatorPB.java:228)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:180)
 at 
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:149)
 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 
$Proxy18.submitRequest over 
nodeId=om1,nodeAddress=dfmwr125b01t3.nam.nsroot.net:9862 after 3 failover 
attempts. Trying to failover after sleeping for 8000ms. 24/06/20 11:57:30 INFO 
retry.RetryInvocationHandler: com.google.protobuf.ServiceException: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException):
 om1 is Leader but not ready to process request yet. at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderNotReadyException(OzoneManagerProtocolServerSideTranslatorPB.java:265)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderErrorException(OzoneManagerProtocolServerSideTranslatorPB.java:237)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitReadRequestToOM(OzoneManagerProtocolServerSideTranslatorPB.java:228)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:180)
 at 
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:149)
 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 
$Proxy18.submitRequest over 
nodeId=om1,nodeAddress=dfmwr125b01t3.nam.nsroot.net:9862 after 4 failover 
attempts. Trying to failover after sleeping for 10000ms. 24/06/20 11:57:40 INFO 
retry.RetryInvocationHandler: com.google.protobuf.ServiceException: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException):
 om1 is Leader but not ready to process request yet. at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderNotReadyException(OzoneManagerProtocolServerSideTranslatorPB.java:265)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderErrorException(OzoneManagerProtocolServerSideTranslatorPB.java:237)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitReadRequestToOM(OzoneManagerProtocolServerSideTranslatorPB.java:228)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:180)
 at 
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:149)
 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 
$Proxy18.submitRequest over 
nodeId=om1,nodeAddress=dfmwr125b01t3.nam.nsroot.net:9862 after 5 failover 
attempts. Trying to failover after sleeping for 12000ms. 24/06/20 11:57:52 INFO 
retry.RetryInvocationHandler: com.google.protobuf.ServiceException: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException):
 om1 is Leader but not ready to process request yet. at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderNotReadyException(OzoneManagerProtocolServerSideTranslatorPB.java:265)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderErrorException(OzoneManagerProtocolServerSideTranslatorPB.java:237)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitReadRequestToOM(OzoneManagerProtocolServerSideTranslatorPB.java:228)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:180)
 at 
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:149)
 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 
$Proxy18.submitRequest over 
nodeId=om1,nodeAddress=dfmwr125b01t3.nam.nsroot.net:9862 after 6 failover 
attempts. Trying to failover after sleeping for 14000ms. 24/06/20 11:58:06 INFO 
retry.RetryInvocationHandler: com.google.protobuf.ServiceException: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException):
 om1 is Leader but not ready to process request yet. at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderNotReadyException(OzoneManagerProtocolServerSideTranslatorPB.java:265)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.createLeaderErrorException(OzoneManagerProtocolServerSideTranslatorPB.java:237)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitReadRequestToOM(OzoneManagerProtocolServerSideTranslatorPB.java:228)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:180)
 at 
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
 at 
org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:149)
 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 
$Proxy18.submitRequest over 
nodeId=om1,nodeAddress=dfmwr125b01t3.nam.nsroot.net:9862 after 7 failover 
attempts. Trying to failover after sleeping for 16000ms.
{code}

The permanent solution is to patch the cluster's OMs with HDDS-9876.

Or if patching is not an immediate option, a temporary solution is fix their OM 
DB by applying the txs using another Ozone cluster with the patch:
1. Grab the affected cluster's *om.db {color:red}and Raft logs{color}*, 
transfer both to a second cluster with HDDS-9876 patched.
1.5. Confirm the transactionInfoTable has a lower index than the latest Raft 
logs:
{code:bash|title=Check transactionInfoTable index. Index is 253086177 in this 
case}
# ozone debug ldb --db=/var/lib/hadoop-ozone/om/data/om.db scan 
--column_family=transactionInfoTable
...
{ "#TRANSACTIONINFO": {
  "transactionInfoString": "2862#253086177"
} }
{code}
{code:bash|title=Check OM Raft logs. Latest index is 253086398}
# ls -l /var/lib/hadoop-ozone/om/ratis/*/current/
total 3320
-rw-r--r-- 1 hdfs hdfs 3343710 Jun 23 13:54 log_253079988-253086377
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 14:14 log_253086378-253086379
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 14:24 log_253086380-253086381
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 15:18 log_253086382-253086383
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 15:20 log_253086384-253086385
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 15:58 log_253086386-253086387
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 16:01 log_253086388-253086389
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 17:27 log_253086390-253086391
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 17:39 log_253086392-253086393
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 17:50 log_253086394-253086395
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 18:20 log_253086396-253086397
-rw-r--r-- 1 hdfs hdfs     189 Jun 23 21:32 log_inprogress_253086398
-rw-r--r-- 1 hdfs hdfs      64 Jun 23 18:24 raft-meta
-rw-r--r-- 1 hdfs hdfs     155 Jun 24 04:05 raft-meta.conf
{code}
2. Start OM on the second cluster with configs applied (e.g. OM ID, service ID, 
SCM ID or [patch it out in the 
code|https://github.com/apache/ozone/blob/27c1513fd92346a3e91d3705d057a07b10d7abad/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java#L624-L625]
 so that OM's Raft server will boot up)
3. Tail the OM log. Wait for Raft server to start and (attempt to) start an 
election. The Ratis election won't actually progress because we only have one 
OM node, but that is perfectly fine for our purpose because we just need the 
Raft server and OzoneManagerStateMachine to apply the transactions. No need to 
allow the OM to actually serve any client requests during the repair.
{code}
tail -F /var/log/hadoop-ozone/ozone-om.log
{code}
4. Run {{ozone debug ldb}} on {{transactionInfoTable}} again to check the index 
applied to OM DB. Index is 253086398 in this case, which caught up to the 
latest Raft logs, indicating that the previously "stuck" transactions are now 
applied to OM DB. That is exactly what we want. Now we got the OM DB "fixed".
{code:bash|title=Check transactionInfoTable again. Index is 253086398 now. Yay}
# ozone debug ldb --db=/var/lib/hadoop-ozone/om/data/om.db scan 
--column_family=transactionInfoTable
24/06/24 22:21:17 INFO helpers.OmKeyInfo: OmKeyInfo.getCodec ignorePipeline = 
true
...
{ "#TRANSACTIONINFO": {
  "transactionInfoString": "2873#253086398"
} }
{code}
5. Shut down OM on the second cluster. Copy the fixed OM DB back to the first 
cluster and bootstrap the OMs. Raft logs should no longer be useful now that 
all transactions are applied.

> OzoneManagerStateMachine should add response to OzoneManagerDoubleBuffer for 
> every write request
> ------------------------------------------------------------------------------------------------
>
>                 Key: HDDS-9876
>                 URL: https://issues.apache.org/jira/browse/HDDS-9876
>             Project: Apache Ozone
>          Issue Type: Bug
>          Components: OM
>            Reporter: Sammi Chen
>            Assignee: Sammi Chen
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 1.4.0, 1.5.0
>
>
> This task is to resolve the issues in HDDS-9342.
> HDDS-2680 introduced a logic in OzoneManagerStateMachine to calculate the 
> lastAppliedTermIndex based on two maps, applyTransactionMap and 
> ratisTransactionMap. Any write request from RATIS through applyTransaction 
> will add its trxLogIndex into applyTransactionMap. And any write request 
> which is flushed by OzoneManagerDoubleBuffer#flushBatch will have its 
> trxLogIndex removed from applyTransactionMap during flushBatch call 
> ozoneManagerRatisSnapShot.updateLastAppliedIndex(flushedEpochs).
> If any write request from RATIS not going through 
> OzoneManagerDoubleBuffer#flushBatch, then its trxLogIndex will be left in the
> applyTransactionMap forever. Since lastApplicedIndex can only be updated 
> incrementally, any trxLogIndex not confirmed by OzoneManagerDoubleBuffer 
> flush will make the lastApplicedIndex grow stops before it, and although 
> write requests after that unconfirmed one could be flushed, but their 
> trxLogIndex will be added to the ratisTransactionMap, which causes the 
> ratisTransactionMap grow bigger and bigger. 
> How a write request will not be confirmed by OzoneManagerDoubleBuffer flush? 
> Here is one case reproduced locally.
> T1: create bucket1
> T2: client1 sends delete bucket "bucket1" request to OM. OM verify bucket1 
> exists, then send request to RATIS to handle the request.
> T3: client2 sends create key "bucket1/key1" request to OM. OM verify bucket2 
> exists, then send request to RATIS
> T4: OzoneManagerStateMachine executes delete bucket "bucket1" success, return 
> response to client1
> T5: OzoneManagerStateMachine executes create key "bucket1/key1" request, 
> "bucket1" cannot be found, execution fails, return failure to client2
> In T5, the failure stack is
> {code:java}
> 2023-10-18 19:04:10,131 [OM StateMachine ApplyTransaction Thread - 0] WARN 
> org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine: Failed to write, 
> Exception occurred 
> BUCKET_NOT_FOUND org.apache.hadoop.ozone.om.exceptions.OMException: Bucket 
> not found: s3v/prod-voyager
> at 
> org.apache.hadoop.ozone.om.OzoneManagerUtils.reportNotFound(OzoneManagerUtils.java:87)
> at 
> org.apache.hadoop.ozone.om.OzoneManagerUtils.getBucketInfo(OzoneManagerUtils.java:72)
> at 
> org.apache.hadoop.ozone.om.OzoneManagerUtils.resolveBucketInfoLink(OzoneManagerUtils.java:148)
> at 
> org.apache.hadoop.ozone.om.OzoneManagerUtils.getResolvedBucketInfo(OzoneManagerUtils.java:124)
> at 
> org.apache.hadoop.ozone.om.OzoneManagerUtils.getBucketLayout(OzoneManagerUtils.java:106)
> at 
> org.apache.hadoop.ozone.om.request.BucketLayoutAwareOMKeyRequestFactory.createRequest(BucketLayoutAwareOMKeyRequestFactory.java:230)
> at 
> org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils.createClientRequest(OzoneManagerRatisUtils.java:336)
> at 
> org.apache.hadoop.ozone.protocolPB.OzoneManagerRequestHandler.handleWriteRequest(OzoneManagerRequestHandler.java:380)
> at 
> org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine.runCommand(OzoneManagerStateMachine.java:572)
> at 
> org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine.lambda$1(OzoneManagerStateMachine.java:362)
> at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745){code}
> In OzoneManagerStateMachine.runCommand, when IOException is throw out from 
> OzoneManagerRequestHandler.handleWriteRequest, it constructs and returns 
> OMResponse to client, it doesn't add the response into 
> OzoneManagerDoubleBuffer, so OzoneManagerDoubleBuffer doesn't aware of this 
> request and its trxLogIndex. The consequence is this trxLogIndex will be stay 
> in applyTransactionMap forever.



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to