[ 
https://issues.apache.org/jira/browse/HDDS-1610?focusedWorklogId=291053&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-291053
 ]

ASF GitHub Bot logged work on HDDS-1610:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 08/Aug/19 07:49
            Start Date: 08/Aug/19 07:49
    Worklog Time Spent: 10m 
      Work Description: supratimdeka commented on pull request #1226: 
HDDS-1610. applyTransaction failure should not be lost on restart.
URL: https://github.com/apache/hadoop/pull/1226#discussion_r311898006
 
 

 ##########
 File path: 
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
 ##########
 @@ -674,30 +674,54 @@ public void notifyIndexUpdate(long term, long index) {
       if (cmdType == Type.WriteChunk || cmdType ==Type.PutSmallFile) {
         builder.setCreateContainerSet(createContainerSet);
       }
+      CompletableFuture<Message> applyTransactionFuture =
+          new CompletableFuture<>();
       // Ensure the command gets executed in a separate thread than
       // stateMachineUpdater thread which is calling applyTransaction here.
-      CompletableFuture<Message> future = CompletableFuture
-          .supplyAsync(() -> runCommand(requestProto, builder.build()),
+      CompletableFuture<ContainerCommandResponseProto> future =
+          CompletableFuture.supplyAsync(
+              () -> runCommandGetResponse(requestProto, builder.build()),
               getCommandExecutor(requestProto));
-
-      future.thenAccept(m -> {
+      future.thenApply(r -> {
         if (trx.getServerRole() == RaftPeerRole.LEADER) {
           long startTime = (long) trx.getStateMachineContext();
           metrics.incPipelineLatency(cmdType,
               Time.monotonicNowNanos() - startTime);
         }
-
-        final Long previous =
-            applyTransactionCompletionMap
-                .put(index, trx.getLogEntry().getTerm());
-        Preconditions.checkState(previous == null);
-        if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) {
-          metrics.incNumBytesCommittedCount(
+        if (r.getResult() != ContainerProtos.Result.SUCCESS) {
+          StorageContainerException sce =
+              new StorageContainerException(r.getMessage(), r.getResult());
+          LOG.error(gid + ": ApplyTransaction failed: cmd " + r.getCmdType()
+              + " logIndex " + index + " Error message: " + r.getMessage()
+              + " Container Result: " + r.getResult());
+          metrics.incNumApplyTransactionsFails();
+          ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole());
+          // Since the applyTransaction now is completed exceptionally,
+          // before any further snapshot is taken , the exception will be
+          // caught in stateMachineUpdater in Ratis and ratis server will
+          // shutdown.
+          applyTransactionFuture.completeExceptionally(sce);
+        } else {
+          metrics.incNumBytesWrittenCount(
               requestProto.getWriteChunk().getChunkData().getLen());
+          LOG.debug(gid + ": ApplyTransaction completed: cmd " + r.getCmdType()
+              + " logIndex " + index + " Error message: " + r.getMessage()
+              + " Container Result: " + r.getResult());
+          applyTransactionFuture.complete(r::toByteString);
+          if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) {
+            metrics.incNumBytesCommittedCount(
+                requestProto.getWriteChunk().getChunkData().getLen());
+          }
         }
+
+        final Long previous = applyTransactionCompletionMap
+            .put(index, trx.getLogEntry().getTerm());
+        Preconditions.checkState(previous == null);
         updateLastApplied();
-      }).whenComplete((r, t) -> applyTransactionSemaphore.release());
-      return future;
+        applyTransactionSemaphore.release();
 
 Review comment:
   I think we should keep the whenComplete() stage at the end.
   releasing the semaphore from a whenComplete() stage guarantees that the 
semaphore will be released even if the processing inside thenApply() stage hits 
an exception. This seems to me to be a good practice.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 291053)
    Time Spent: 4h  (was: 3h 50m)

> applyTransaction failure should not be lost on restart
> ------------------------------------------------------
>
>                 Key: HDDS-1610
>                 URL: https://issues.apache.org/jira/browse/HDDS-1610
>             Project: Hadoop Distributed Data Store
>          Issue Type: Bug
>            Reporter: Shashikant Banerjee
>            Assignee: Shashikant Banerjee
>            Priority: Blocker
>              Labels: pull-request-available
>          Time Spent: 4h
>  Remaining Estimate: 0h
>
> If the applyTransaction fails in the containerStateMachine, then the 
> container should not accept new writes on restart,.
> This can occur if
> # chunk write applyTransaction fails
> # container state update to UNHEALTHY also fails
> # Ratis snapshot is taken
> # Node restarts
> # container accepts new transactions



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

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

Reply via email to