Ver código fonte

HDDS-1029. Allow option for force in DeleteContainerCommand. Contributed by Bharat Viswanadham.

Yiqun Lin 6 anos atrás
pai
commit
3efa168e1f
16 arquivos alterados com 174 adições e 102 exclusões
  1. 0 1
      hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
  2. 0 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
  3. 1 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
  4. 3 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java
  5. 2 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java
  6. 2 3
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
  7. 12 15
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
  8. 1 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
  9. 5 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java
  10. 26 2
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteContainerCommand.java
  11. 1 0
      hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
  12. 2 2
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
  13. 1 1
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
  14. 2 2
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
  15. 112 44
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java
  16. 4 23
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java

+ 0 - 1
hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto

@@ -126,7 +126,6 @@ enum Result {
   PUT_SMALL_FILE_ERROR = 20;
   GET_SMALL_FILE_ERROR = 21;
   CLOSED_CONTAINER_IO = 22;
-  ERROR_CONTAINER_NOT_EMPTY = 23;
   ERROR_IN_COMPACT_DB = 24;
   UNCLOSED_CONTAINER_IO = 25;
   DELETE_ON_OPEN_CONTAINER = 26;

+ 0 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java

@@ -126,7 +126,6 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
     case CONTAINER_UNHEALTHY:
     case CLOSED_CONTAINER_IO:
     case DELETE_ON_OPEN_CONTAINER:
-    case ERROR_CONTAINER_NOT_EMPTY:
       return true;
     default:
       return false;

+ 1 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java

@@ -50,10 +50,9 @@ public interface Container<CONTAINERDATA extends ContainerData> extends RwLock {
   /**
    * Deletes the container.
    *
-   * @param forceDelete   - whether this container should be deleted forcibly.
    * @throws StorageContainerException
    */
-  void delete(boolean forceDelete) throws StorageContainerException;
+  void delete() throws StorageContainerException;
 
   /**
    * Update the container.

+ 3 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java

@@ -151,9 +151,11 @@ public abstract class Handler {
    * Deletes the given container.
    *
    * @param container container to be deleted
+   * @param force if this is set to true, we delete container without checking
+   * state of the container.
    * @throws IOException
    */
-  public abstract void deleteContainer(Container container)
+  public abstract void deleteContainer(Container container, boolean force)
       throws IOException;
 
   public void setScmID(String scmId) {

+ 2 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java

@@ -54,7 +54,8 @@ public class DeleteContainerCommandHandler implements CommandHandler {
       final DeleteContainerCommand deleteContainerCommand =
           (DeleteContainerCommand) command;
       final ContainerController controller = ozoneContainer.getController();
-      controller.deleteContainer(deleteContainerCommand.getContainerID());
+      controller.deleteContainer(deleteContainerCommand.getContainerID(),
+          deleteContainerCommand.isForce());
       updateCommandStatus(context, command,
           (cmdStatus) -> cmdStatus.setStatus(true), LOG);
     } catch (IOException e) {

+ 2 - 3
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java

@@ -253,11 +253,10 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
 
 
   @Override
-  public void delete(boolean forceDelete)
-      throws StorageContainerException {
+  public void delete() throws StorageContainerException {
     long containerId = containerData.getContainerID();
     try {
-      KeyValueContainerUtil.removeContainer(containerData, config, forceDelete);
+      KeyValueContainerUtil.removeContainer(containerData, config);
     } catch (StorageContainerException ex) {
       throw ex;
     } catch (IOException ex) {

+ 12 - 15
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java

@@ -928,26 +928,23 @@ public class KeyValueHandler extends Handler {
   }
 
   @Override
-  public void deleteContainer(Container container) throws IOException {
-    deleteInternal(container, true);
+  public void deleteContainer(Container container, boolean force)
+      throws IOException {
+    deleteInternal(container, force);
   }
 
   private void deleteInternal(Container container, boolean force)
       throws StorageContainerException {
     container.writeLock();
     try {
-      // Check if container is open
-      if (container.getContainerData().isOpen()) {
-        throw new StorageContainerException(
-            "Deletion of Open Container is not allowed.",
-            DELETE_ON_OPEN_CONTAINER);
-      }
-      if (!force && container.getContainerData().getKeyCount() > 0) {
-        // If the container is not empty and cannot be deleted forcibly,
-        // then throw a SCE to stop deleting.
-        throw new StorageContainerException(
-            "Container cannot be deleted because it is not empty.",
-            ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY);
+    // If force is false, we check container state.
+      if (!force) {
+        // Check if container is open
+        if (container.getContainerData().isOpen()) {
+          throw new StorageContainerException(
+              "Deletion of Open Container is not allowed.",
+              DELETE_ON_OPEN_CONTAINER);
+        }
       }
       long containerId = container.getContainerData().getContainerID();
       containerSet.removeContainer(containerId);
@@ -955,6 +952,6 @@ public class KeyValueHandler extends Handler {
       container.writeUnlock();
     }
     // Avoid holding write locks for disk operations
-    container.delete(force);
+    container.delete();
   }
 }

+ 1 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java

@@ -104,11 +104,10 @@ public final class KeyValueContainerUtil {
    *
    * @param containerData - Data of the container to remove.
    * @param conf - configuration of the cluster.
-   * @param forceDelete - whether this container should be deleted forcibly.
    * @throws IOException
    */
   public static void removeContainer(KeyValueContainerData containerData,
-                                     Configuration conf, boolean forceDelete)
+                                     Configuration conf)
       throws IOException {
     Preconditions.checkNotNull(containerData);
     File containerMetaDataPath = new File(containerData

+ 5 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerController.java

@@ -116,11 +116,14 @@ public class ContainerController {
   /**
    * Deletes a container given its Id.
    * @param containerId Id of the container to be deleted
+   * @param force if this is set to true, we delete container without checking
+   * state of the container.
    * @throws IOException
    */
-  public void deleteContainer(final long containerId) throws IOException {
+  public void deleteContainer(final long containerId, boolean force)
+      throws IOException {
     final Container container = containerSet.getContainer(containerId);
-    getHandler(container).deleteContainer(container);
+    getHandler(container).deleteContainer(container, force);
   }
 
   /**

+ 26 - 2
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/DeleteContainerCommand.java

@@ -31,9 +31,28 @@ public class DeleteContainerCommand extends
     SCMCommand<DeleteContainerCommandProto> {
 
   private final long containerId;
+  private final boolean force;
 
+  /**
+   * DeleteContainerCommand, to send a command for datanode to delete a
+   * container.
+   * @param containerId
+   */
   public DeleteContainerCommand(long containerId) {
+    this(containerId, false);
+  }
+
+  /**
+   * DeleteContainerCommand, to send a command for datanode to delete a
+   * container.
+   * @param containerId
+   * @param forceFlag if this is set to true, we delete container without
+   * checking state of the container.
+   */
+
+  public DeleteContainerCommand(long containerId, boolean forceFlag) {
     this.containerId = containerId;
+    this.force = forceFlag;
   }
 
   @Override
@@ -46,7 +65,7 @@ public class DeleteContainerCommand extends
     DeleteContainerCommandProto.Builder builder =
         DeleteContainerCommandProto.newBuilder();
     builder.setCmdId(getId())
-        .setContainerID(getContainerID());
+        .setContainerID(getContainerID()).setForce(force);
     return builder.build();
   }
 
@@ -54,9 +73,14 @@ public class DeleteContainerCommand extends
     return containerId;
   }
 
+  public boolean isForce() {
+    return force;
+  }
+
   public static DeleteContainerCommand getFromProtobuf(
       DeleteContainerCommandProto protoMessage) {
     Preconditions.checkNotNull(protoMessage);
-    return new DeleteContainerCommand(protoMessage.getContainerID());
+    return new DeleteContainerCommand(protoMessage.getContainerID(),
+        protoMessage.getForce());
   }
 }

+ 1 - 0
hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto

@@ -303,6 +303,7 @@ This command asks the datanode to delete a specific container.
 message DeleteContainerCommandProto {
   required int64 containerID = 1;
   required int64 cmdId = 2;
+  required bool force = 3 [default = false];
 }
 
 /**

+ 2 - 2
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java

@@ -213,7 +213,7 @@ public class TestKeyValueContainer {
     }
 
     //delete the original one
-    keyValueContainer.delete(true);
+    keyValueContainer.delete();
 
     //create a new one
     KeyValueContainerData containerData =
@@ -294,7 +294,7 @@ public class TestKeyValueContainer {
     keyValueContainer = new KeyValueContainer(
         keyValueContainerData, conf);
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
-    keyValueContainer.delete(true);
+    keyValueContainer.delete();
 
     String containerMetaDataPath = keyValueContainerData
         .getMetadataPath();

+ 1 - 1
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java

@@ -216,7 +216,7 @@ public class ReplicationManager implements Runnable {
               final int toDelete = Math.min(listOfReplica.size() - 1,
                   numberOfReplicasToDelete);
               final DeleteContainerCommand deleteContainer =
-                  new DeleteContainerCommand(containerID.getId());
+                  new DeleteContainerCommand(containerID.getId(), true);
               for (int i = 0; i < toDelete; i++) {
                 LOG.info("Container {} is over replicated." +
                     " Expected replica count is {}, but found {}." +

+ 2 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java

@@ -231,7 +231,7 @@ public class TestContainerPersistence {
     Assert.assertTrue(containerSet.getContainerMap()
         .containsKey(testContainerID2));
 
-    container1.delete(false);
+    container1.delete();
     containerSet.removeContainer(testContainerID1);
     Assert.assertFalse(containerSet.getContainerMap()
         .containsKey(testContainerID1));
@@ -253,7 +253,7 @@ public class TestContainerPersistence {
     exception.expect(StorageContainerException.class);
     exception.expectMessage(
         "Container cannot be deleted because it is not empty.");
-    container2.delete(false);
+    container2.delete();
     Assert.assertTrue(containerSet.getContainerMap()
         .containsKey(testContainerID1));
   }

+ 112 - 44
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
@@ -43,7 +44,9 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.util.HashMap;
+import java.util.UUID;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
 
@@ -55,14 +58,22 @@ public class TestDeleteContainerHandler {
 
   private static MiniOzoneCluster cluster;
   private static OzoneConfiguration conf;
+  private static ObjectStore objectStore;
+  private static String volumeName = UUID.randomUUID().toString();
+  private static String bucketName = UUID.randomUUID().toString();
 
   @BeforeClass
   public static void setup() throws Exception {
     conf = new OzoneConfiguration();
     conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB");
     cluster = MiniOzoneCluster.newBuilder(conf)
-        .setNumDatanodes(3).build();
+        .setNumDatanodes(1).build();
     cluster.waitForClusterToBeReady();
+
+    OzoneClient client = OzoneClientFactory.getClient(conf);
+    objectStore = client.getObjectStore();
+    objectStore.createVolume(volumeName);
+    objectStore.getVolume(volumeName).createBucket(bucketName);
   }
 
   @AfterClass
@@ -77,41 +88,27 @@ public class TestDeleteContainerHandler {
   }
 
   @Test(timeout = 60000)
-  public void testDeleteContainerRequestHandler() throws Exception {
+  public void testDeleteContainerRequestHandlerOnClosedContainer()
+      throws Exception {
 
     //the easiest way to create an open container is creating a key
-    OzoneClient client = OzoneClientFactory.getClient(conf);
-    ObjectStore objectStore = client.getObjectStore();
-    objectStore.createVolume("test");
-    objectStore.getVolume("test").createBucket("test");
-    OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
-        .createKey("test", 1024, ReplicationType.RATIS,
-            ReplicationFactor.THREE, new HashMap<>());
-    key.write("test".getBytes());
-    key.close();
 
-    //get the name of a valid container
-    OmKeyArgs keyArgs =
-        new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
-            .setType(HddsProtos.ReplicationType.RATIS)
-            .setFactor(HddsProtos.ReplicationFactor.THREE).setDataSize(1024)
-            .setKeyName("test").build();
+    String keyName = UUID.randomUUID().toString();
 
-    OmKeyLocationInfo omKeyLocationInfo =
-        cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
-            .get(0).getBlocksLatestVersionOnly().get(0);
+    // create key
+    createKey(keyName);
+
+    // get containerID of the key
+    ContainerID containerId = getContainerID(keyName);
 
-    ContainerID containerId = ContainerID.valueof(
-        omKeyLocationInfo.getContainerID());
     ContainerInfo container = cluster.getStorageContainerManager()
         .getContainerManager().getContainer(containerId);
+
     Pipeline pipeline = cluster.getStorageContainerManager()
         .getPipelineManager().getPipeline(container.getPipelineID());
 
-    // Take first node from the datanode list, and close the container (As we
-    // have only three, the container will be created on three nodes)
     // We need to close the container because delete container only happens
-    // on closed containers.
+    // on closed containers with force flag set to false.
 
     HddsDatanodeService hddsDatanodeService =
         cluster.getHddsDatanodes().get(0);
@@ -121,9 +118,11 @@ public class TestDeleteContainerHandler {
 
     DatanodeDetails datanodeDetails = hddsDatanodeService.getDatanodeDetails();
 
+    NodeManager nodeManager =
+        cluster.getStorageContainerManager().getScmNodeManager();
+
     //send the order to close the container
-    cluster.getStorageContainerManager().getScmNodeManager()
-        .addDatanodeCommand(datanodeDetails.getUuid(),
+    nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
             new CloseContainerCommand(containerId.getId(), pipeline.getId()));
 
     GenericTestUtils.waitFor(() ->
@@ -138,44 +137,113 @@ public class TestDeleteContainerHandler {
     Assert.assertFalse(isContainerDeleted(hddsDatanodeService,
         containerId.getId()));
 
-    // send delete container to one of the datanode
-    cluster.getStorageContainerManager().getScmNodeManager()
-        .addDatanodeCommand(datanodeDetails.getUuid(),
-            new DeleteContainerCommand(containerId.getId()));
+    // send delete container to the datanode
+    nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
+            new DeleteContainerCommand(containerId.getId(), false));
 
     GenericTestUtils.waitFor(() ->
             isContainerDeleted(hddsDatanodeService, containerId.getId()),
         500, 5 * 1000);
 
-    // On another node, where container is open try to delete container
-    HddsDatanodeService hddsDatanodeService1 =
-        cluster.getHddsDatanodes().get(1);
-    DatanodeDetails datanodeDetails1 =
-        hddsDatanodeService1.getDatanodeDetails();
+    Assert.assertTrue(isContainerDeleted(hddsDatanodeService,
+        containerId.getId()));
 
-    cluster.getStorageContainerManager().getScmNodeManager()
-        .addDatanodeCommand(datanodeDetails1.getUuid(),
-            new DeleteContainerCommand(containerId.getId()));
+  }
+
+
+  @Test
+  public void testDeleteContainerRequestHandlerOnOpenContainer()
+      throws Exception {
+
+    //the easiest way to create an open container is creating a key
+    String keyName = UUID.randomUUID().toString();
+
+    // create key
+    createKey(keyName);
+
+    // get containerID of the key
+    ContainerID containerId = getContainerID(keyName);
+
+    HddsDatanodeService hddsDatanodeService =
+        cluster.getHddsDatanodes().get(0);
+    DatanodeDetails datanodeDetails =
+        hddsDatanodeService.getDatanodeDetails();
+
+    NodeManager nodeManager =
+        cluster.getStorageContainerManager().getScmNodeManager();
+
+    // Send delete container command with force flag set to false.
+    nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
+        new DeleteContainerCommand(containerId.getId(), false));
 
     // Here it should not delete it, and the container should exist in the
     // containerset
-
     int count = 1;
-    // Checking for 10 seconds, whether it is containerSet, as after command
+    // Checking for 5 seconds, whether it is containerSet, as after command
     // is issued, giving some time for it to process.
-    while (!isContainerDeleted(hddsDatanodeService1, containerId.getId())) {
+    while (!isContainerDeleted(hddsDatanodeService, containerId.getId())) {
       Thread.sleep(1000);
       count++;
-      if (count == 10) {
+      if (count == 5) {
         break;
       }
     }
 
-    Assert.assertFalse(isContainerDeleted(hddsDatanodeService1,
+    Assert.assertFalse(isContainerDeleted(hddsDatanodeService,
+        containerId.getId()));
+
+
+    // Now delete container with force flag set to true. now it should delete
+    // container
+
+    nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
+        new DeleteContainerCommand(containerId.getId(), true));
+
+    GenericTestUtils.waitFor(() ->
+            isContainerDeleted(hddsDatanodeService, containerId.getId()),
+        500, 5 * 1000);
+
+    Assert.assertTrue(isContainerDeleted(hddsDatanodeService,
         containerId.getId()));
 
   }
 
+  /**
+   * create a key with specified name.
+   * @param keyName
+   * @throws IOException
+   */
+  private void createKey(String keyName) throws IOException {
+    OzoneOutputStream key = objectStore.getVolume(volumeName)
+        .getBucket(bucketName)
+        .createKey(keyName, 1024, ReplicationType.STAND_ALONE,
+            ReplicationFactor.ONE, new HashMap<>());
+    key.write("test".getBytes());
+    key.close();
+  }
+
+  /**
+   * Return containerID of the key.
+   * @param keyName
+   * @return ContainerID
+   * @throws IOException
+   */
+  private ContainerID getContainerID(String keyName) throws IOException {
+    OmKeyArgs keyArgs =
+        new OmKeyArgs.Builder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setType(HddsProtos.ReplicationType.STAND_ALONE)
+            .setFactor(HddsProtos.ReplicationFactor.ONE)
+            .setKeyName(keyName).build();
+
+    OmKeyLocationInfo omKeyLocationInfo =
+        cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
+            .get(0).getBlocksLatestVersionOnly().get(0);
+
+    return ContainerID.valueof(
+        omKeyLocationInfo.getContainerID());
+  }
+
   /**
    * Checks whether is closed or not on a datanode.
    * @param hddsDatanodeService

+ 4 - 23
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java

@@ -410,37 +410,18 @@ public class TestOzoneContainer {
       Assert.assertTrue(
           putBlockRequest.getTraceID().equals(response.getTraceID()));
 
-      // Container cannot be deleted forcibly because
-      // the container is not closed.
-      request = ContainerTestHelper.getDeleteContainer(
-          client.getPipeline(), containerID, true);
-      response = client.sendCommand(request);
-
-      Assert.assertNotNull(response);
-      Assert.assertEquals(ContainerProtos.Result.DELETE_ON_OPEN_CONTAINER,
-          response.getResult());
-      Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
-
-      // Close the container.
-      request = ContainerTestHelper.getCloseContainer(
-          client.getPipeline(), containerID);
-      response = client.sendCommand(request);
-      Assert.assertNotNull(response);
-      Assert.assertEquals(ContainerProtos.Result.SUCCESS, response.getResult());
-      Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
-
-      // Container cannot be deleted because the container is not empty.
+      // Container cannot be deleted because force flag is set to false and
+      // the container is still open
       request = ContainerTestHelper.getDeleteContainer(
           client.getPipeline(), containerID, false);
       response = client.sendCommand(request);
 
       Assert.assertNotNull(response);
-      Assert.assertEquals(ContainerProtos.Result.ERROR_CONTAINER_NOT_EMPTY,
+      Assert.assertEquals(ContainerProtos.Result.DELETE_ON_OPEN_CONTAINER,
           response.getResult());
       Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
 
-      // Container can be deleted forcibly because
-      // it is closed and non-empty.
+      // Container can be deleted, by setting force flag, even with out closing
       request = ContainerTestHelper.getDeleteContainer(
           client.getPipeline(), containerID, true);
       response = client.sendCommand(request);