Parcourir la source

HDFS-11675. Ozone: SCM CLI: Implement delete container command. Contributed by Weiwei Yang.

Anu Engineer il y a 8 ans
Parent
commit
6b3da448eb
18 fichiers modifiés avec 338 ajouts et 30 suppressions
  1. 23 7
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/ContainerOperationClient.java
  2. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/ScmClient.java
  3. 10 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/protocol/StorageContainerLocationProtocol.java
  4. 20 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
  5. 26 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/storage/ContainerProtocolCalls.java
  6. 13 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/StorageContainerLocationProtocol.proto
  7. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/storage/StorageManager.java
  8. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  9. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
  10. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java
  11. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
  12. 23 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/OzoneCommandHandler.java
  13. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/SCMCLI.java
  14. 14 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/container/ContainerCommandHandler.java
  15. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/container/CreateContainerHandler.java
  16. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/container/DeleteContainerHandler.java
  17. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/util/MockStorageClient.java
  18. 82 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java

+ 23 - 7
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/ContainerOperationClient.java

@@ -82,9 +82,11 @@ public class ContainerOperationClient implements ScmClient {
       client = xceiverClientManager.acquireClient(pipeline);
       String traceID = UUID.randomUUID().toString();
       ContainerProtocolCalls.createContainer(client, traceID);
-      LOG.info("Created container " + containerId +
-          " leader:" + pipeline.getLeader() +
-          " machines:" + pipeline.getMachines());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Created container " + containerId
+            + " leader:" + pipeline.getLeader()
+            + " machines:" + pipeline.getMachines());
+      }
       return pipeline;
     } finally {
       if (client != null) {
@@ -128,11 +130,26 @@ public class ContainerOperationClient implements ScmClient {
   /**
    * Delete the container, this will release any resource it uses.
    * @param pipeline - Pipeline that represents the container.
+   * @param force - True to forcibly delete the container.
    * @throws IOException
    */
   @Override
-  public void deleteContainer(Pipeline pipeline) throws IOException {
-    // TODO
+  public void deleteContainer(Pipeline pipeline, boolean force)
+      throws IOException {
+    XceiverClientSpi client = null;
+    try {
+      client = xceiverClientManager.acquireClient(pipeline);
+      String traceID = UUID.randomUUID().toString();
+      ContainerProtocolCalls.deleteContainer(client, force, traceID);
+      LOG.info("Deleted container {}, leader: {}, machines: {} ",
+          pipeline.getContainerName(),
+          pipeline.getLeader(),
+          pipeline.getMachines());
+    } finally {
+      if (client != null) {
+        xceiverClientManager.releaseClient(client);
+      }
+    }
   }
 
   /**
@@ -144,8 +161,7 @@ public class ContainerOperationClient implements ScmClient {
   @Override
   public Pipeline getContainer(String containerId) throws
       IOException {
-    // TODO
-    return null;
+    return storageContainerLocationClient.getContainer(containerId);
   }
 
   /**

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/ScmClient.java

@@ -53,9 +53,10 @@ public interface ScmClient {
   /**
    * Delets an existing container.
    * @param pipeline - Pipeline that represents the container.
+   * @param force - true to forcibly delete the container.
    * @throws IOException
    */
-  void deleteContainer(Pipeline pipeline) throws IOException;
+  void deleteContainer(Pipeline pipeline, boolean force) throws IOException;
 
   /**
    * Gets the container size -- Computed by SCM from Container Reports.

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/protocol/StorageContainerLocationProtocol.java

@@ -62,4 +62,14 @@ public interface StorageContainerLocationProtocol {
   Pipeline allocateContainer(String containerName,
       ScmClient.ReplicationFactor replicationFactor) throws IOException;
 
+  /**
+   * Ask SCM the location of the container. SCM responds with a group of
+   * nodes where this container and its replicas are located.
+   *
+   * @param containerName - Name of the container.
+   * @return Pipeline - the pipeline where container locates.
+   * @throws IOException
+   */
+  Pipeline getContainer(String containerName) throws IOException;
+
 }

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java

@@ -35,6 +35,8 @@ import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolPr
 import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.GetStorageContainerLocationsRequestProto;
 import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.GetStorageContainerLocationsResponseProto;
 import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.LocatedContainerProto;
+import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerRequestProto;
+import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerResponseProto;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 
 import java.io.Closeable;
@@ -146,6 +148,24 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
     return Pipeline.getFromProtoBuf(response.getPipeline());
   }
 
+  public Pipeline getContainer(String containerName) throws IOException {
+    Preconditions.checkNotNull(containerName,
+        "Container Name cannot be Null");
+    Preconditions.checkState(!containerName.isEmpty(),
+        "Container name cannot be empty");
+    GetContainerRequestProto request = GetContainerRequestProto
+        .newBuilder()
+        .setContainerName(containerName)
+        .build();
+    try {
+      GetContainerResponseProto response =
+          rpcProxy.getContainer(NULL_RPC_CONTROLLER, request);
+      return Pipeline.getFromProtoBuf(response.getPipeline());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   public Object getUnderlyingProxyObject() {
     return rpcProxy;

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/storage/ContainerProtocolCalls.java

@@ -245,6 +245,32 @@ public final class ContainerProtocolCalls {
     validateContainerResponse(response);
   }
 
+  /**
+   * Deletes a container from a pipeline.
+   *
+   * @param client
+   * @param force whether or not to forcibly delete the container.
+   * @param traceID
+   * @throws IOException
+   */
+  public static void deleteContainer(XceiverClientSpi client,
+      boolean force, String traceID) throws IOException {
+    ContainerProtos.DeleteContainerRequestProto.Builder deleteRequest =
+        ContainerProtos.DeleteContainerRequestProto.newBuilder();
+    deleteRequest.setName(client.getPipeline().getContainerName());
+    deleteRequest.setPipeline(client.getPipeline().getProtobufMessage());
+    deleteRequest.setForceDelete(force);
+
+    ContainerCommandRequestProto.Builder request =
+        ContainerCommandRequestProto.newBuilder();
+    request.setCmdType(ContainerProtos.Type.DeleteContainer);
+    request.setDeleteContainer(deleteRequest);
+    request.setTraceID(traceID);
+    ContainerCommandResponseProto response =
+        client.sendCommand(request.build());
+    validateContainerResponse(response);
+  }
+
   /**
    * Reads the data given the container name and key.
    *

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/StorageContainerLocationProtocol.proto

@@ -84,6 +84,14 @@ message ContainerResponseProto {
   optional string errorMessage = 3;
 }
 
+message GetContainerRequestProto {
+  required string containerName = 1;
+}
+
+message GetContainerResponseProto {
+  required hadoop.hdfs.ozone.Pipeline pipeline = 1;
+}
+
 // SCM Block protocol
 /**
  * keys - batch of block keys to find
@@ -146,10 +154,14 @@ service StorageContainerLocationProtocolService {
       returns(GetStorageContainerLocationsResponseProto);
 
   /**
-   Creates a container entry in SCM.
+   * Creates a container entry in SCM.
    */
   rpc allocateContainer(ContainerRequestProto) returns (ContainerResponseProto);
 
+  /**
+   * Returns the pipeline for a given container.
+   */
+  rpc getContainer(GetContainerRequestProto) returns (GetContainerResponseProto);
 
   /**
    * Find the set of nodes that currently host the block, as

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/storage/StorageManager.java

@@ -224,7 +224,7 @@ public class StorageManager {
     for (String containerID : volume.getContainerIDsList()) {
       try {
         Pipeline pipeline = storageClient.getContainer(containerID);
-        storageClient.deleteContainer(pipeline);
+        storageClient.deleteContainer(pipeline, force);
       } catch (IOException e) {
         LOGGER.error("Error deleting container Container:{} error:{}",
             containerID, e);

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -111,6 +111,7 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker;
 import org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.client.BlockReportOptions;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -1574,7 +1575,12 @@ public class DataNode extends ReconfigurableBase
     
     registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
   }
-  
+
+  @VisibleForTesting
+  public OzoneContainer getOzoneContainerManager() {
+    return this.datanodeStateMachine.getContainer();
+  }
+
   /**
    * After the block pool has contacted the NN, registers that block pool
    * with the secret manager, updating it with the secrets provided by the NN.

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java

@@ -17,6 +17,7 @@
 
 package org.apache.hadoop.ozone.container.ozoneimpl;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -193,4 +194,9 @@ public class OzoneContainer {
   public List<ContainerData> getContainerReports() throws IOException {
     return this.manager.getContainerReports();
   }
+
+  @VisibleForTesting
+  public ContainerManager getContainerManager() {
+    return this.manager;
+  }
 }

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java

@@ -54,6 +54,10 @@ import org.apache.hadoop.ozone.protocol.proto
     .StorageContainerLocationProtocolProtos.GetScmBlockLocationsRequestProto;
 import org.apache.hadoop.ozone.protocol.proto
     .StorageContainerLocationProtocolProtos.GetScmBlockLocationsResponseProto;
+import org.apache.hadoop.ozone.protocol.proto
+    .StorageContainerLocationProtocolProtos.GetContainerRequestProto;
+import org.apache.hadoop.ozone.protocol.proto
+    .StorageContainerLocationProtocolProtos.GetContainerResponseProto;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolPB;
 
@@ -130,6 +134,20 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB
     }
   }
 
+  @Override
+  public GetContainerResponseProto getContainer(
+      RpcController controller, GetContainerRequestProto request)
+      throws ServiceException {
+    try {
+      Pipeline pipeline = impl.getContainer(request.getContainerName());
+      return GetContainerResponseProto.newBuilder()
+          .setPipeline(pipeline.getProtobufMessage())
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   @Override
   public GetScmBlockLocationsResponseProto getScmBlockLocations(
       RpcController controller, GetScmBlockLocationsRequestProto req)

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java

@@ -365,8 +365,11 @@ public class StorageContainerManager
         ScmClient.ReplicationFactor.ONE);
   }
 
-  @VisibleForTesting
-  Pipeline getContainer(String containerName) throws IOException {
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Pipeline getContainer(String containerName) throws IOException {
     return scmContainerManager.getContainer(containerName);
   }
 

+ 23 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/OzoneCommandHandler.java

@@ -19,10 +19,9 @@ package org.apache.hadoop.ozone.scm.cli;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.hadoop.scm.client.ScmClient;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.io.PrintStream;
 
 /**
  * The abstract class of all SCM CLI commands.
@@ -30,8 +29,8 @@ import java.io.IOException;
 public abstract class OzoneCommandHandler {
 
   private ScmClient scmClient;
-  protected static final Logger LOG =
-      LoggerFactory.getLogger(OzoneCommandHandler.class);
+  protected PrintStream out = System.out;
+  protected PrintStream err = System.err;
 
   /**
    * Constructs a handler object.
@@ -44,6 +43,26 @@ public abstract class OzoneCommandHandler {
     return scmClient;
   }
 
+  /**
+   * Sets customized output stream to redirect the stdout to somewhere else.
+   * @param out
+   */
+  public void setOut(PrintStream out) {
+    this.out = out;
+  }
+
+  /**
+   * Sets customized error stream to redirect the stderr to somewhere else.
+   * @param err
+   */
+  public void setErr(PrintStream err) {
+    this.err = err;
+  }
+
+  public void logOut(String msg, String ... variable) {
+    this.out.println(String.format(msg, variable));
+  }
+
   /**
    * Executes the Client command.
    *

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/SCMCLI.java

@@ -38,8 +38,6 @@ import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSi
 import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ToolRunner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.PrintStream;
@@ -63,8 +61,6 @@ import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
  */
 public class SCMCLI extends OzoneBaseCLI {
 
-  private static final Logger LOG = LoggerFactory.getLogger(SCMCLI.class);
-
   public static final String HELP_OP = "help";
   public static final int CMD_WIDTH = 80;
 
@@ -203,7 +199,7 @@ public class SCMCLI extends OzoneBaseCLI {
       BasicParser parser = new BasicParser();
       return parser.parse(opts, argv);
     } catch (ParseException ex) {
-      LOG.error(ex.getMessage());
+      err.println(ex.getMessage());
     }
     return null;
   }
@@ -216,6 +212,7 @@ public class SCMCLI extends OzoneBaseCLI {
       if (cmd.hasOption(CONTAINER_CMD)) {
         handler = new ContainerCommandHandler(scmClient);
       }
+
       if (handler == null) {
         if (cmd.hasOption(HELP_OP)) {
           displayHelp();
@@ -226,6 +223,9 @@ public class SCMCLI extends OzoneBaseCLI {
           return UNRECOGNIZED_CMD;
         }
       } else {
+        // Redirect stdout and stderr if necessary.
+        handler.setOut(this.out);
+        handler.setErr(this.err);
         handler.execute(cmd);
         return SUCCESS;
       }

+ 14 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/container/ContainerCommandHandler.java

@@ -29,7 +29,10 @@ import java.util.Arrays;
 
 import static org.apache.hadoop.ozone.scm.cli.SCMCLI.CMD_WIDTH;
 import static org.apache.hadoop.ozone.scm.cli.SCMCLI.HELP_OP;
-import static org.apache.hadoop.ozone.scm.cli.container.CreateContainerHandler.CONTAINER_CREATE;
+import static org.apache.hadoop.ozone.scm.cli.container
+    .CreateContainerHandler.CONTAINER_CREATE;
+import static org.apache.hadoop.ozone.scm.cli.container
+    .DeleteContainerHandler.CONTAINER_DELETE;
 
 /**
  * The handler class of container-specific commands, e.g. createContainer.
@@ -52,10 +55,15 @@ public class ContainerCommandHandler extends OzoneCommandHandler {
     OzoneCommandHandler handler = null;
     if (cmd.hasOption(CONTAINER_CREATE)) {
       handler = new CreateContainerHandler(getScmClient());
+    } else if (cmd.hasOption(CONTAINER_DELETE)) {
+      handler = new DeleteContainerHandler(getScmClient());
     }
+
     // execute the sub command, throw exception if no sub command found
     // unless -help option is given.
     if (handler != null) {
+      handler.setOut(this.out);
+      handler.setErr(this.err);
       handler.execute(cmd);
     } else {
       displayHelp();
@@ -79,7 +87,11 @@ public class ContainerCommandHandler extends OzoneCommandHandler {
   private static void addCommandsOption(Options options) {
     Option createContainer =
         new Option(CONTAINER_CREATE, false, "Create container");
+    Option deleteContainer =
+        new Option(CONTAINER_DELETE, true, "Delete container");
+
     options.addOption(createContainer);
+    options.addOption(deleteContainer);
     // TODO : add other options such as delete, close etc.
   }
 
@@ -87,6 +99,7 @@ public class ContainerCommandHandler extends OzoneCommandHandler {
     addCommandsOption(options);
     // for create container options.
     CreateContainerHandler.addOptions(options);
+    DeleteContainerHandler.addOptions(options);
     // TODO : add other options such as delete, close etc.
   }
 }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/container/CreateContainerHandler.java

@@ -56,9 +56,10 @@ public class CreateContainerHandler extends OzoneCommandHandler {
       }
     }
     String pipelineID = cmd.getOptionValue(PIPELINE_ID);
-    LOG.info("Create container : {}", pipelineID);
+
+    logOut("Creating container : %s.", pipelineID);
     getScmClient().createContainer(pipelineID);
-    LOG.debug("Container creation returned");
+    logOut("Container created.");
   }
 
   @Override

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/cli/container/DeleteContainerHandler.java

@@ -0,0 +1,78 @@
+/**
+ * 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.ozone.scm.cli.container;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.ozone.scm.cli.OzoneCommandHandler;
+import org.apache.hadoop.scm.client.ScmClient;
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.scm.cli.SCMCLI.CMD_WIDTH;
+
+/**
+ * This is the handler that process delete container command.
+ */
+public class DeleteContainerHandler extends OzoneCommandHandler {
+
+  protected static final String CONTAINER_DELETE = "del";
+  protected static final String OPT_FORCE = "f";
+
+  public DeleteContainerHandler(ScmClient scmClient) {
+    super(scmClient);
+  }
+
+  @Override
+  public void execute(CommandLine cmd) throws IOException {
+    Preconditions.checkArgument(cmd.hasOption(CONTAINER_DELETE),
+        "Expecting command del");
+
+    String containerName = cmd.getOptionValue(CONTAINER_DELETE);
+
+    Pipeline pipeline = getScmClient().getContainer(containerName);
+    if (pipeline == null) {
+      throw new IOException("Cannot delete an non-exist container "
+          + containerName);
+    }
+
+    logOut("Deleting container : %s.", containerName);
+    getScmClient().deleteContainer(pipeline, cmd.hasOption(OPT_FORCE));
+    logOut("Container %s deleted.", containerName);
+  }
+
+  @Override public void displayHelp() {
+    Options options = new Options();
+    addOptions(options);
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -del <option>",
+        "where <option> is", options, "");
+  }
+
+  public static void addOptions(Options options) {
+    Option forceOpt = new Option(OPT_FORCE,
+        false,
+        "forcibly delete a container");
+    options.addOption(forceOpt);
+  }
+}

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/util/MockStorageClient.java

@@ -55,7 +55,8 @@ public class MockStorageClient implements ScmClient {
    * @throws IOException
    */
   @Override
-  public void deleteContainer(Pipeline pipeline) throws IOException {
+  public void deleteContainer(Pipeline pipeline, boolean force)
+      throws IOException {
 
   }
 

+ 82 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.ozone.scm;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
+import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
 import org.apache.hadoop.ozone.scm.cli.ResultCode;
 import org.apache.hadoop.ozone.scm.cli.SCMCLI;
 import org.apache.hadoop.scm.XceiverClientManager;
@@ -27,7 +30,7 @@ import org.apache.hadoop.scm.client.ContainerOperationClient;
 import org.apache.hadoop.scm.client.ScmClient;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
-import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -52,6 +55,7 @@ public class TestSCMCli {
       storageContainerLocationClient;
 
   private static StorageContainerManager scm;
+  private static ContainerManager containerManager;
 
   private static ByteArrayOutputStream outContent;
   private static PrintStream outStream;
@@ -73,10 +77,29 @@ public class TestSCMCli {
     errStream = new PrintStream(errContent);
     cli = new SCMCLI(client, outStream, errStream);
     scm = cluster.getStorageContainerManager();
+    containerManager = cluster.getDataNodes().get(0)
+        .getOzoneContainerManager().getContainerManager();
   }
 
-  @After
-  public void shutdown() throws InterruptedException {
+  private int runCommandAndGetOutput(String[] cmd,
+      ByteArrayOutputStream out,
+      ByteArrayOutputStream err) throws Exception {
+    PrintStream cmdOutStream = System.out;
+    PrintStream cmdErrStream = System.err;
+    if(out != null) {
+      cmdOutStream = new PrintStream(out);
+    }
+    if (err != null) {
+      cmdErrStream = new PrintStream(err);
+    }
+    ScmClient client = new ContainerOperationClient(
+        storageContainerLocationClient, new XceiverClientManager(conf));
+    SCMCLI scmCLI = new SCMCLI(client, cmdOutStream, cmdErrStream);
+    return scmCLI.run(cmd);
+  }
+
+  @AfterClass
+  public static void shutdown() throws InterruptedException {
     IOUtils.cleanup(null, storageContainerLocationClient, cluster);
   }
 
@@ -98,6 +121,60 @@ public class TestSCMCli {
     assertEquals(containerName, container.getContainerName());
   }
 
+
+  @Test
+  public void testDeleteContainer() throws Exception {
+    final String cname1 = "cname1";
+    final String cname2 = "cname2";
+
+    // ****************************************
+    // 1. Test to delete a non-empty container.
+    // ****************************************
+    // Create an non-empty container
+    Pipeline pipeline1 = scm.allocateContainer(cname1);
+    ContainerData data1 = new ContainerData(cname1);
+    containerManager.createContainer(pipeline1, data1);
+    ContainerData cdata = containerManager.readContainer(cname1);
+    KeyUtils.getDB(cdata, conf).put(cname1.getBytes(),
+        "someKey".getBytes());
+
+    // Gracefully delete a container should fail because it is not empty.
+    String[] del1 = {"-container", "-del", cname1};
+    ByteArrayOutputStream testErr1 = new ByteArrayOutputStream();
+    int exitCode1 = runCommandAndGetOutput(del1, null, testErr1);
+    assertEquals(ResultCode.EXECUTION_ERROR, exitCode1);
+    assertTrue(testErr1.toString()
+        .contains("Container cannot be deleted because it is not empty."));
+
+    // Delete should fail when attempts to delete an open container.
+    // Even with the force tag.
+    String[] del2 = {"-container", "-del", cname1, "-f"};
+    ByteArrayOutputStream testErr2 = new ByteArrayOutputStream();
+    int exitCode2 = runCommandAndGetOutput(del2, null, testErr2);
+    assertEquals(ResultCode.EXECUTION_ERROR, exitCode2);
+    assertTrue(testErr2.toString()
+        .contains("Attempting to force delete an open container."));
+
+    // Close the container and try force delete again.
+    containerManager.closeContainer(cname1);
+    int exitCode3 = runCommandAndGetOutput(del2, null, null);
+    assertEquals(ResultCode.SUCCESS, exitCode3);
+
+
+    // ****************************************
+    // 2. Test to delete an empty container.
+    // ****************************************
+    // Create an empty container
+    Pipeline pipeline2 = scm.allocateContainer(cname2);
+    ContainerData data2 = new ContainerData(cname2);
+    containerManager.createContainer(pipeline2, data2);
+
+    // Successfully delete an empty container.
+    String[] del3 = {"-container", "-del", cname2};
+    int exitCode4 = runCommandAndGetOutput(del3, null, null);
+    assertEquals(ResultCode.SUCCESS, exitCode4);
+  }
+
   @Test
   public void testNonExistCommand() throws Exception {
     PrintStream init = System.out;
@@ -139,7 +216,8 @@ public class TestSCMCli {
     String expected1 =
         "usage: hdfs scm -container <commands> <options>\n" +
         "where <commands> can be one of the following\n" +
-        " -create   Create container\n";
+        " -create      Create container\n" +
+        " -del <arg>   Delete container\n";
     assertEquals(expected1, testContent.toString());
     testContent.reset();