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

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

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

 ##########
 File path: 
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
 ##########
 @@ -674,30 +681,60 @@ 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(
+              () -> runCommand(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 {} logIndex {} msg : "
+                  + "{} Container Result: {}", gid, r.getCmdType(), index,
+              r.getMessage(), 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);
 
 Review comment:
   Addressed in the latest patch.
 
----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 294634)
    Time Spent: 5h  (was: 4h 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: 5h
>  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: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to