HDDS-709. Modify Close Container handling sequence on datanodes. Contributed by 
Shashikant Banerjee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f944f338
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f944f338
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f944f338

Branch: refs/heads/HDFS-13891
Commit: f944f3383246450a1aa2b34f55f99a9e86e10c42
Parents: 1f9c4f3
Author: Jitendra Pandey <jiten...@apache.org>
Authored: Mon Nov 12 14:08:39 2018 -0800
Committer: Jitendra Pandey <jiten...@apache.org>
Committed: Mon Nov 12 14:08:39 2018 -0800

----------------------------------------------------------------------
 .../helpers/ContainerNotOpenException.java      |  36 +++++++
 .../helpers/InvalidContainerStateException.java |  35 ++++++
 .../main/proto/DatanodeContainerProtocol.proto  |   1 +
 .../container/common/impl/HddsDispatcher.java   | 106 ++++++++++++++++---
 .../common/interfaces/ContainerDispatcher.java  |  10 ++
 .../CloseContainerCommandHandler.java           |  28 +++--
 .../server/ratis/ContainerStateMachine.java     |  11 ++
 .../container/keyvalue/KeyValueHandler.java     |  33 +++---
 .../ozone/client/io/ChunkGroupOutputStream.java |  14 ++-
 .../rpc/TestCloseContainerHandlingByClient.java |   2 +-
 .../rpc/TestContainerStateMachineFailures.java  |   6 +-
 .../transport/server/ratis/TestCSMMetrics.java  |   6 ++
 .../container/server/TestContainerServer.java   |   6 ++
 13 files changed, 255 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerNotOpenException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerNotOpenException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerNotOpenException.java
new file mode 100644
index 0000000..4e406e6
--- /dev/null
+++ 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerNotOpenException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.common.helpers;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+
+/**
+ * Exceptions thrown when a write/update opearation is done on non-open
+ * container.
+ */
+public class ContainerNotOpenException extends StorageContainerException {
+
+  /**
+   * Constructs an {@code IOException} with the specified detail message.
+   *
+   * @param message The detail message (which is saved for later retrieval by
+   * the {@link #getMessage()} method)
+   */
+  public ContainerNotOpenException(String message) {
+    super(message, ContainerProtos.Result.CONTAINER_NOT_OPEN);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/InvalidContainerStateException.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/InvalidContainerStateException.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/InvalidContainerStateException.java
new file mode 100644
index 0000000..1378d1a
--- /dev/null
+++ 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/InvalidContainerStateException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.common.helpers;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+
+/**
+ * Exceptions thrown when a container is in invalid state while doing a I/O.
+ */
+public class InvalidContainerStateException extends StorageContainerException {
+
+  /**
+   * Constructs an {@code IOException} with the specified detail message.
+   *
+   * @param message The detail message (which is saved for later retrieval by
+   * the {@link #getMessage()} method)
+   */
+  public InvalidContainerStateException(String message) {
+    super(message, ContainerProtos.Result.INVALID_CONTAINER_STATE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto 
b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index df26f24..5eecdcb 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -142,6 +142,7 @@ enum Result {
   CONTAINER_UNHEALTHY = 36;
   UNKNOWN_BCSID = 37;
   BCSID_MISMATCH = 38;
+  CONTAINER_NOT_OPEN = 39;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index 27dd8ba..c52d973 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -28,6 +28,8 @@ import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerDataProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerAction;
+import 
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
+import 
org.apache.hadoop.hdds.scm.container.common.helpers.InvalidContainerStateException;
 import 
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
@@ -42,6 +44,9 @@ import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerType;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
+    ContainerDataProto.State;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -99,6 +104,25 @@ public class HddsDispatcher implements ContainerDispatcher {
     volumeSet.shutdown();
   }
 
+  /**
+   * Returns true for exceptions which can be ignored for marking the container
+   * unhealthy.
+   * @param result ContainerCommandResponse error code.
+   * @return true if exception can be ignored, false otherwise.
+   */
+  private boolean canIgnoreException(Result result) {
+    switch (result) {
+    case SUCCESS:
+    case CONTAINER_UNHEALTHY:
+    case CLOSED_CONTAINER_IO:
+    case DELETE_ON_OPEN_CONTAINER:
+    case ERROR_CONTAINER_NOT_EMPTY:
+      return true;
+    default:
+      return false;
+    }
+  }
+
   @Override
   public ContainerCommandResponseProto dispatch(
       ContainerCommandRequestProto msg) {
@@ -160,19 +184,30 @@ public class HddsDispatcher implements 
ContainerDispatcher {
       // failed. All subsequent transactions on the container should fail and
       // hence replica will be marked unhealthy here. In this case, a close
       // container action will be sent to SCM to close the container.
-      if (!HddsUtils.isReadOnly(msg)
-          && responseProto.getResult() != ContainerProtos.Result.SUCCESS) {
-        // If the container is open and the container operation has failed,
-        // it should be first marked unhealthy and the initiate the close
-        // container action. This also implies this is the first transaction
-        // which has failed, so the container is marked unhealthy right here.
+
+      // ApplyTransaction called on closed Container will fail with Closed
+      // container exception. In such cases, ignore the exception here
+      // If the container is already marked unhealthy, no need to change the
+      // state here.
+
+      Result result = responseProto.getResult();
+      if (!HddsUtils.isReadOnly(msg) && !canIgnoreException(result)) {
+        // If the container is open/closing and the container operation
+        // has failed, it should be first marked unhealthy and the initiate the
+        // close container action. This also implies this is the first
+        // transaction which has failed, so the container is marked unhealthy
+        // right here.
         // Once container is marked unhealthy, all the subsequent write
         // transactions will fail with UNHEALTHY_CONTAINER exception.
-        if (container.getContainerState() == ContainerDataProto.State.OPEN) {
-          container.getContainerData()
-              .setState(ContainerDataProto.State.UNHEALTHY);
-          sendCloseContainerActionIfNeeded(container);
-        }
+
+        // For container to be moved to unhealthy state here, the container can
+        // only be in open or closing state.
+        State containerState = container.getContainerData().getState();
+        Preconditions.checkState(
+            containerState == State.OPEN || containerState == State.CLOSING);
+        container.getContainerData()
+            .setState(ContainerDataProto.State.UNHEALTHY);
+        sendCloseContainerActionIfNeeded(container);
       }
       return responseProto;
     } else {
@@ -207,6 +242,54 @@ public class HddsDispatcher implements ContainerDispatcher 
{
   }
 
   /**
+   * This will be called as a part of creating the log entry during
+   * startTransaction in Ratis on the leader node. In such cases, if the
+   * container is not in open state for writing we should just fail.
+   * Leader will propagate the exception to client.
+   * @param msg  container command proto
+   * @throws StorageContainerException In case container state is open for 
write
+   *         requests and in invalid state for read requests.
+   */
+  @Override
+  public void validateContainerCommand(
+      ContainerCommandRequestProto msg) throws StorageContainerException {
+    ContainerType containerType = msg.getCreateContainer().getContainerType();
+    Handler handler = getHandler(containerType);
+    if (handler == null) {
+      StorageContainerException ex = new StorageContainerException(
+          "Invalid " + "ContainerType " + containerType,
+          ContainerProtos.Result.CONTAINER_INTERNAL_ERROR);
+      throw ex;
+    }
+    ContainerProtos.Type cmdType = msg.getCmdType();
+    long containerID = msg.getContainerID();
+    Container container;
+    container = getContainer(containerID);
+    if (container != null) {
+      State containerState = container.getContainerState();
+      if (!HddsUtils.isReadOnly(msg) && containerState != State.OPEN) {
+        switch (cmdType) {
+        case CreateContainer:
+          // Create Container is idempotent. There is nothing to validate.
+          break;
+        case CloseContainer:
+          // If the container is unhealthy, closeContainer will be rejected
+          // while execution. Nothing to validate here.
+          break;
+        default:
+          // if the container is not open, no updates can happen. Just throw
+          // an exception
+          throw new ContainerNotOpenException(
+              "Container " + containerID + " in " + containerState + " state");
+        }
+      } else if (HddsUtils.isReadOnly(msg) && containerState == State.INVALID) 
{
+        throw new InvalidContainerStateException(
+            "Container " + containerID + " in " + containerState + " state");
+      }
+    }
+  }
+
+  /**
    * If the container usage reaches the close threshold or the container is
    * marked unhealthy we send Close ContainerAction to SCM.
    * @param container current state of container
@@ -264,7 +347,6 @@ public class HddsDispatcher implements ContainerDispatcher {
     }
   }
 
-  @VisibleForTesting
   public Container getContainer(long containerID) {
     return containerSet.getContainer(containerID);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
index 18644bb..7a22143 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDispatcher.java
@@ -23,6 +23,7 @@ import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
+import 
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 
 /**
  * Dispatcher acts as the bridge between the transport layer and
@@ -41,6 +42,15 @@ public interface ContainerDispatcher {
   ContainerCommandResponseProto dispatch(ContainerCommandRequestProto msg);
 
   /**
+   * Validates whether the container command should be executed on the pipeline
+   * or not. Will be invoked by the leader node in the Ratis pipeline
+   * @param msg containerCommand
+   * @throws StorageContainerException
+   */
+  void validateContainerCommand(
+      ContainerCommandRequestProto msg) throws StorageContainerException;
+
+  /**
    * Initialize the Dispatcher.
    */
   void init();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
index a7d855b..0838be2 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java
@@ -24,6 +24,9 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
+    ContainerDataProto.State;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.statemachine
     .SCMConnectionManager;
@@ -84,8 +87,18 @@ public class CloseContainerCommandHandler implements 
CommandHandler {
         cmdExecuted = false;
         return;
       }
-      if (!container.getContainerData().isClosed()) {
+      ContainerData containerData = container.getContainerData();
+      State containerState = container.getContainerData().getState();
+      if (containerState != State.CLOSED) {
         LOG.debug("Closing container {}.", containerID);
+        // when a closeContainerCommand arrives at a Datanode and if the
+        // container is open, each replica will be moved to closing state 
first.
+        if (containerState == State.OPEN) {
+          containerData.setState(State.CLOSING);
+        }
+
+        // if the container is already closed, it will be just ignored.
+        // ICR will get triggered to change the replica state in SCM.
         HddsProtos.PipelineID pipelineID = closeContainerProto.getPipelineID();
         HddsProtos.ReplicationType replicationType =
             closeContainerProto.getReplicationType();
@@ -100,14 +113,13 @@ public class CloseContainerCommandHandler implements 
CommandHandler {
         request.setDatanodeUuid(
             context.getParent().getDatanodeDetails().getUuidString());
         // submit the close container request for the XceiverServer to handle
-        ozoneContainer.submitContainerRequest(
-            request.build(), replicationType, pipelineID);
+        ozoneContainer.submitContainerRequest(request.build(), replicationType,
+            pipelineID);
         // Since the container is closed, we trigger an ICR
-        IncrementalContainerReportProto icr = IncrementalContainerReportProto
-            .newBuilder()
-            .addReport(ozoneContainer.getContainerSet()
-                .getContainer(containerID).getContainerReport())
-            .build();
+        IncrementalContainerReportProto icr =
+            IncrementalContainerReportProto.newBuilder().addReport(
+                ozoneContainer.getContainerSet().getContainer(containerID)
+                    .getContainerReport()).build();
         context.addReport(icr);
         context.getParent().triggerHeartbeat();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
index 38b789e..3899bde 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
@@ -205,6 +205,17 @@ public class ContainerStateMachine extends 
BaseStateMachine {
     final ContainerCommandRequestProto proto =
         getRequestProto(request.getMessage().getContent());
     Preconditions.checkArgument(request.getRaftGroupId().equals(gid));
+    try {
+      dispatcher.validateContainerCommand(proto);
+    } catch (IOException ioe) {
+      TransactionContext ctxt = TransactionContext.newBuilder()
+          .setClientRequest(request)
+          .setStateMachine(this)
+          .setServerRole(RaftPeerRole.LEADER)
+          .build();
+      ctxt.setException(ioe);
+      return ctxt;
+    }
     if (proto.getCmdType() == Type.WriteChunk) {
       final WriteChunkRequestProto write = proto.getWriteChunk();
       // create the state machine data proto

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 1271d99..d8c23bf 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -386,28 +386,26 @@ public class KeyValueHandler extends Handler {
     }
 
     long containerID = kvContainer.getContainerData().getContainerID();
-    ContainerDataProto.State containerState = kvContainer.getContainerState();
-
     try {
-      if (containerState == ContainerDataProto.State .CLOSED) {
-        LOG.debug("Container {} is already closed.", containerID);
-        return ContainerUtils.getSuccessResponse(request);
-      } else if (containerState == ContainerDataProto.State .INVALID) {
-        LOG.debug("Invalid container data. ContainerID: {}", containerID);
-        throw new StorageContainerException("Invalid container data. " +
-            "ContainerID: " + containerID, INVALID_CONTAINER_STATE);
-      }
-
+      checkContainerOpen(kvContainer);
       KeyValueContainerData kvData = kvContainer.getContainerData();
 
       // remove the container from open block map once, all the blocks
       // have been committed and the container is closed
-      kvData.setState(ContainerDataProto.State.CLOSING);
       commitPendingBlocks(kvContainer);
+
+      // TODO : The close command should move the container to either quasi
+      // closed/closed depending upon how the closeContainer gets executed.
+      // If it arrives by Standalone, it will be moved to Quasi Closed or
+      // otherwise moved to Closed state if it gets executed via Ratis.
       kvContainer.close();
       // make sure the the container open keys from BlockMap gets removed
       openContainerBlockMap.removeContainer(kvData.getContainerID());
     } catch (StorageContainerException ex) {
+      if (ex.getResult() == CLOSED_CONTAINER_IO) {
+        LOG.debug("Container {} is already closed.", containerID);
+        return ContainerUtils.getSuccessResponse(request);
+      }
       return ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ex) {
       return ContainerUtils.logAndReturnError(LOG,
@@ -799,14 +797,21 @@ public class KeyValueHandler extends Handler {
 
     ContainerDataProto.State containerState = kvContainer.getContainerState();
 
-    if (containerState == ContainerDataProto.State.OPEN) {
+    /**
+     * In a closing state, follower will receive transactions from leader.
+     * Once the leader is put to closing state, it will reject further requests
+     * from clients. Only the transactions which happened before the container
+     * in the leader goes to closing state, will arrive here even the container
+     * might already be in closing state here.
+     */
+    if (containerState == ContainerDataProto.State.OPEN
+        || containerState == ContainerDataProto.State.CLOSING) {
       return;
     } else {
       String msg = "Requested operation not allowed as ContainerState is " +
           containerState;
       ContainerProtos.Result result = null;
       switch (containerState) {
-      case CLOSING:
       case CLOSED:
         result = CLOSED_CONTAINER_IO;
         break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
----------------------------------------------------------------------
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
index 2cfc7cf..450e2dc 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
+++ 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
 import org.apache.hadoop.hdds.client.BlockID;
+import 
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
 import 
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
@@ -429,13 +430,24 @@ public class ChunkGroupOutputStream extends OutputStream {
   }
 
   private boolean checkIfContainerIsClosed(IOException ioe) {
-    return Optional.of(ioe.getCause())
+    return checkIfContainerNotOpenException(ioe) || Optional.of(ioe.getCause())
         .filter(e -> e instanceof StorageContainerException)
         .map(e -> (StorageContainerException) e)
         .filter(sce -> sce.getResult() == Result.CLOSED_CONTAINER_IO)
         .isPresent();
   }
 
+  private boolean checkIfContainerNotOpenException(IOException ioe) {
+    Throwable t = ioe.getCause();
+    while (t != null) {
+      if (t instanceof ContainerNotOpenException) {
+        return true;
+      }
+      t = t.getCause();
+    }
+    return false;
+  }
+
   private long getKeyLength() {
     return streamEntries.parallelStream().mapToLong(e -> e.currentPosition)
         .sum();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
----------------------------------------------------------------------
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
index 576801d..43517ae 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
@@ -559,7 +559,7 @@ public class TestCloseContainerHandlingByClient {
       if (datanodes.get(0).equals(datanodeService.getDatanodeDetails())) {
         datanodeService.getDatanodeStateMachine().getContainer()
             .getContainerSet().getContainer(containerID).getContainerData()
-            .setState(ContainerProtos.ContainerDataProto.State.CLOSING);
+            .setState(ContainerProtos.ContainerDataProto.State.CLOSED);
       }
     }
     dataString = fixedLengthString(keyString, (chunkSize * 1 / 2));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java
----------------------------------------------------------------------
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java
index bffbd6e..73bff6f 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java
@@ -162,9 +162,9 @@ public class TestContainerStateMachineFailures {
       key.close();
       Assert.fail("Expected exception not thrown");
     } catch (IOException ioe) {
-      Assert.assertTrue(ioe.getCause() instanceof StorageContainerException);
-      Assert.assertTrue(((StorageContainerException) 
ioe.getCause()).getResult()
-          == ContainerProtos.Result.CONTAINER_UNHEALTHY);
+      Assert.assertTrue(ioe instanceof StorageContainerException);
+      Assert.assertTrue(((StorageContainerException) ioe).getResult()
+          == ContainerProtos.Result.BLOCK_NOT_COMMITTED);
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java
----------------------------------------------------------------------
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java
index 67db7ff..ab2ddf0 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/TestCSMMetrics.java
@@ -33,6 +33,7 @@ import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
 import org.apache.hadoop.hdds.scm.*;
+import 
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -173,6 +174,11 @@ public class TestCSMMetrics {
     }
 
     @Override
+    public void validateContainerCommand(
+        ContainerCommandRequestProto msg) throws StorageContainerException {
+    }
+
+    @Override
     public void init() {
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f944f338/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
----------------------------------------------------------------------
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
index 33e3e1a..9a3fa1b 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.container.server;
 
+import 
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.interfaces.Handler;
@@ -229,6 +230,11 @@ public class TestContainerServer {
     }
 
     @Override
+    public void validateContainerCommand(
+        ContainerCommandRequestProto msg) throws StorageContainerException {
+    }
+
+    @Override
     public void shutdown() {
     }
     @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-commits-h...@hadoop.apache.org

Reply via email to