Browse Source

HDDS-379. Simplify and improve the cli arg parsing of ozone scmcli.
Contributed by Elek, Marton.

Anu Engineer 6 years ago
parent
commit
76bae4ccb1
23 changed files with 576 additions and 1436 deletions
  1. 4 0
      hadoop-hdds/common/dev-support/findbugsExcludeFile.xml
  2. 82 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
  3. 35 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/HddsVersionProvider.java
  4. 2 11
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/package-info.java
  5. 5 0
      hadoop-hdds/pom.xml
  6. 0 43
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java
  7. 0 87
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneCommandHandler.java
  8. 80 166
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
  9. 0 85
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java
  10. 54 0
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java
  11. 0 128
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java
  12. 0 67
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java
  13. 65 0
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java
  14. 0 95
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java
  15. 60 0
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java
  16. 0 114
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
  17. 94 0
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
  18. 0 117
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java
  19. 83 0
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
  20. 3 0
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/package-info.java
  21. 8 4
      hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/package-info.java
  22. 1 1
      hadoop-ozone/common/src/main/bin/ozone
  23. 0 518
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java

+ 4 - 0
hadoop-hdds/common/dev-support/findbugsExcludeFile.xml

@@ -21,4 +21,8 @@
   <Match>
     <Package name="org.apache.hadoop.hdds.protocol.datanode.proto"/>
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdds.cli.GenericCli"></Class>
+    <Bug pattern="DM_EXIT" />
+  </Match>
 </FindBugsFilter>

+ 82 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java

@@ -0,0 +1,82 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+
+import picocli.CommandLine;
+import picocli.CommandLine.ExecutionException;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.ParameterException;
+import picocli.CommandLine.RunLast;
+
+/**
+ * This is a generic parent class for all the ozone related cli tools.
+ */
+public class GenericCli implements Callable<Void> {
+
+  @Option(names = {"--verbose"},
+      description = "More verbose output. Show the stack trace of the errors.")
+  private boolean verbose;
+
+  @Option(names = {"-D", "--set"})
+  private Map<String, String> configurationOverrides = new HashMap<>();
+
+  private final CommandLine cmd;
+
+  public GenericCli() {
+    cmd = new CommandLine(this);
+  }
+
+  public void run(String[] argv) {
+    try {
+      cmd.parseWithHandler(new RunLast(), argv);
+    } catch (ExecutionException ex) {
+      printError(ex.getCause());
+      System.exit(-1);
+    }
+  }
+
+  private void printError(Throwable error) {
+    if (verbose) {
+      error.printStackTrace(System.err);
+    } else {
+      System.err.println(error.getMessage().split("\n")[0]);
+    }
+  }
+
+  @Override
+  public Void call() throws Exception {
+    throw new ParameterException(cmd, "Please choose a subcommand");
+  }
+
+  public OzoneConfiguration createOzoneConfiguration() {
+    OzoneConfiguration ozoneConf = new OzoneConfiguration();
+    if (configurationOverrides != null) {
+      for (Entry<String, String> entry : configurationOverrides.entrySet()) {
+        ozoneConf
+            .set(entry.getKey(), configurationOverrides.get(entry.getValue()));
+      }
+    }
+    return ozoneConf;
+  }
+}

+ 35 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/HddsVersionProvider.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli;
+
+import org.apache.hadoop.utils.HddsVersionInfo;
+
+import picocli.CommandLine.IVersionProvider;
+
+/**
+ * Version provider for the CLI interface.
+ */
+public class HddsVersionProvider implements IVersionProvider {
+  @Override
+  public String[] getVersion() throws Exception {
+    String[] result = new String[] {
+        HddsVersionInfo.getBuildVersion()
+    };
+    return result;
+  }
+}

+ 2 - 11
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ResultCode.java → hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/package-info.java

@@ -15,17 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdds.scm.cli;
 
 /**
- * The possible result code of SCM CLI.
+ * Generic helper class to make instantiate picocli based cli tools.
  */
-public final class ResultCode {
-  public static final int SUCCESS = 1;
-
-  public static final int UNRECOGNIZED_CMD = 2;
-
-  public static final int EXECUTION_ERROR = 3;
-
-  private ResultCode() {}
-}
+package org.apache.hadoop.hdds.cli;

+ 5 - 0
hadoop-hdds/pom.xml

@@ -68,6 +68,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
+    <dependency>
+      <groupId>info.picocli</groupId>
+      <artifactId>picocli</artifactId>
+      <version>3.5.2</version>
+    </dependency>
     <dependency>
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>

+ 0 - 43
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java

@@ -1,43 +0,0 @@
-/**
- * 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.cli;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.util.Tool;
-
-import java.io.IOException;
-import java.net.URISyntaxException;
-
-/**
- * This class is the base CLI for scm, om and scmadm.
- */
-public abstract class OzoneBaseCLI extends Configured implements Tool {
-
-  protected abstract int dispatch(CommandLine cmd, Options opts)
-      throws IOException, URISyntaxException;
-
-  protected abstract CommandLine parseArgs(String[] argv, Options opts)
-      throws ParseException;
-
-  protected abstract Options getOptions();
-
-  protected abstract void displayHelp();
-}

+ 0 - 87
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneCommandHandler.java

@@ -1,87 +0,0 @@
-/**
- * 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.cli;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-import java.io.PrintStream;
-
-/**
- * The abstract class of all SCM CLI commands.
- */
-public abstract class OzoneCommandHandler {
-
-  private ScmClient scmClient;
-  private PrintStream out = System.out;
-  private PrintStream err = System.err;
-
-  /**
-   * Constructs a handler object.
-   */
-  public OzoneCommandHandler(ScmClient scmClient) {
-    this.scmClient = scmClient;
-  }
-
-  protected ScmClient getScmClient() {
-    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.
-   *
-   * @param cmd - CommandLine.
-   * @throws IOException throws exception.
-   */
-  public abstract void execute(CommandLine cmd) throws IOException;
-
-  /**
-   * Display a help message describing the options the command takes.
-   * TODO : currently only prints to standard out, may want to change this.
-   */
-  public abstract void displayHelp();
-
-  public PrintStream getOut() {
-    return out;
-  }
-
-  public PrintStream getErr() {
-    return err;
-  }
-}

+ 80 - 166
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java

@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
@@ -17,69 +17,73 @@
  */
 package org.apache.hadoop.hdds.scm.cli;
 
-import org.apache.commons.cli.BasicParser;
-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.commons.cli.ParseException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
 import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.cli.GenericCli;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.cli.container.ContainerCommandHandler;
-import org.apache.hadoop.hdds.scm.cli.container.CreateContainerHandler;
+import org.apache.hadoop.hdds.scm.cli.container.CloseSubcommand;
+import org.apache.hadoop.hdds.scm.cli.container.CreateSubcommand;
+import org.apache.hadoop.hdds.scm.cli.container.DeleteSubcommand;
+import org.apache.hadoop.hdds.scm.cli.container.InfoSubcommand;
+import org.apache.hadoop.hdds.scm.cli.container.ListSubcommand;
 import org.apache.hadoop.hdds.scm.client.ContainerOperationClient;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.protocolPB
     .StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.ToolRunner;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.InetSocketAddress;
-import java.net.URISyntaxException;
-import java.util.Arrays;
+import org.apache.hadoop.util.NativeCodeLoader;
 
+import org.apache.commons.lang3.StringUtils;
+import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
+    .OZONE_SCM_CLIENT_ADDRESS_KEY;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE;
-import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
-import static org.apache.hadoop.hdds.scm.cli.ResultCode.EXECUTION_ERROR;
-import static org.apache.hadoop.hdds.scm.cli.ResultCode.SUCCESS;
-import static org.apache.hadoop.hdds.scm.cli.ResultCode.UNRECOGNIZED_CMD;
+    .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
+import org.apache.log4j.ConsoleAppender;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
 
 /**
  * This class is the CLI of SCM.
  */
-public class SCMCLI extends OzoneBaseCLI {
-
-  public static final String HELP_OP = "help";
-  public static final int CMD_WIDTH = 80;
 
-  private final ScmClient scmClient;
-  private final PrintStream out;
-  private final PrintStream err;
-
-  private final Options options;
-
-  public SCMCLI(ScmClient scmClient) {
-    this(scmClient, System.out, System.err);
-  }
-
-  public SCMCLI(ScmClient scmClient, PrintStream out, PrintStream err) {
-    this.scmClient = scmClient;
-    this.out = out;
-    this.err = err;
-    this.options = getOptions();
-  }
+/**
+ * Container subcommand.
+ */
+@Command(name = "ozone scmcli", hidden = true, description =
+    "Developer tools to handle SCM specific "
+        + "operations.",
+    versionProvider = HddsVersionProvider.class,
+    subcommands = {
+        ListSubcommand.class,
+        InfoSubcommand.class,
+        DeleteSubcommand.class,
+        CreateSubcommand.class,
+        CloseSubcommand.class
+    },
+    mixinStandardHelpOptions = true)
+public class SCMCLI extends GenericCli {
+
+  @Option(names = {"--scm"}, description = "The destination scm (host:port)")
+  private String scm = "";
 
   /**
    * Main for the scm shell Command handling.
@@ -88,30 +92,40 @@ public class SCMCLI extends OzoneBaseCLI {
    * @throws Exception
    */
   public static void main(String[] argv) throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
-    ScmClient scmClient = getScmClient(conf);
-    SCMCLI shell = new SCMCLI(scmClient);
-    conf.setQuietMode(false);
-    shell.setConf(conf);
-    int res = 0;
-    try {
-      res = ToolRunner.run(shell, argv);
-    } catch (Exception ex) {
-      System.exit(1);
-    }
-    System.exit(res);
+
+    LogManager.resetConfiguration();
+    Logger.getRootLogger().setLevel(Level.INFO);
+    Logger.getRootLogger()
+        .addAppender(new ConsoleAppender(new PatternLayout("%m%n")));
+    Logger.getLogger(NativeCodeLoader.class).setLevel(Level.ERROR);
+
+    new SCMCLI().run(argv);
   }
 
-  private static ScmClient getScmClient(OzoneConfiguration ozoneConf)
+  public ScmClient createScmClient()
       throws IOException {
+
+    OzoneConfiguration ozoneConf = createOzoneConfiguration();
+    if (StringUtils.isNotEmpty(scm)) {
+      ozoneConf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, scm);
+    }
+    if (!HddsUtils.getHostNameFromConfigKeys(ozoneConf,
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY).isPresent()) {
+
+      throw new IllegalArgumentException(
+          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY
+              + " should be set in ozone-site.xml or with the --scm option");
+    }
+
     long version = RPC.getProtocolVersion(
         StorageContainerLocationProtocolPB.class);
     InetSocketAddress scmAddress =
         getScmAddressForClients(ozoneConf);
-    int containerSizeGB = (int)ozoneConf.getStorageSize(
+    int containerSizeGB = (int) ozoneConf.getStorageSize(
         OZONE_SCM_CONTAINER_SIZE, OZONE_SCM_CONTAINER_SIZE_DEFAULT,
         StorageUnit.GB);
-    ContainerOperationClient.setContainerSizeB(containerSizeGB*OzoneConsts.GB);
+    ContainerOperationClient
+        .setContainerSizeB(containerSizeGB * OzoneConsts.GB);
 
     RPC.setProtocolEngine(ozoneConf, StorageContainerLocationProtocolPB.class,
         ProtobufRpcEngine.class);
@@ -121,116 +135,16 @@ public class SCMCLI extends OzoneBaseCLI {
                 scmAddress, UserGroupInformation.getCurrentUser(), ozoneConf,
                 NetUtils.getDefaultSocketFactory(ozoneConf),
                 Client.getRpcTimeout(ozoneConf)));
-    ScmClient storageClient = new ContainerOperationClient(
+    return new ContainerOperationClient(
         client, new XceiverClientManager(ozoneConf));
-    return storageClient;
-  }
-
-  /**
-   * Adds ALL the options that hdfs scm command supports. Given the hierarchy
-   * of commands, the options are added in a cascading manner, e.g.:
-   * {@link SCMCLI} asks {@link ContainerCommandHandler} to add it's options,
-   * which then asks it's sub command, such as
-   * {@link CreateContainerHandler}
-   * to add it's own options.
-   *
-   * We need to do this because {@link BasicParser} need to take all the options
-   * when paring args.
-   * @return ALL the options supported by this CLI.
-   */
-  @Override
-  protected Options getOptions() {
-    Options newOptions = new Options();
-    // add the options
-    addTopLevelOptions(newOptions);
-    ContainerCommandHandler.addOptions(newOptions);
-    // TODO : add pool, node and pipeline commands.
-    addHelpOption(newOptions);
-    return newOptions;
-  }
-
-  private static void addTopLevelOptions(Options options) {
-    Option containerOps =
-        new Option(ContainerCommandHandler.CONTAINER_CMD, false,
-            "Container related options");
-    options.addOption(containerOps);
-    // TODO : add pool, node and pipeline commands.
-  }
-
-  private static void addHelpOption(Options options) {
-    Option helpOp = new Option(HELP_OP, false, "display help message");
-    options.addOption(helpOp);
   }
 
-  @Override
-  protected void displayHelp() {
-    HelpFormatter helpFormatter = new HelpFormatter();
-    Options topLevelOptions = new Options();
-    addTopLevelOptions(topLevelOptions);
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scmcli <commands> [<options>]",
-        "where <commands> can be one of the following",
-        topLevelOptions, "");
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    CommandLine cmd = parseArgs(args, options);
-    if (cmd == null) {
-      err.println("Unrecognized options:" + Arrays.asList(args));
-      displayHelp();
-      return UNRECOGNIZED_CMD;
-    }
-    return dispatch(cmd, options);
-  }
-
-  /**
-   * This function parses all command line arguments
-   * and returns the appropriate values.
-   *
-   * @param argv - Argv from main
-   *
-   * @return CommandLine
-   */
-  @Override
-  protected CommandLine parseArgs(String[] argv, Options opts)
-      throws ParseException {
-    try {
-      BasicParser parser = new BasicParser();
-      return parser.parse(opts, argv);
-    } catch (ParseException ex) {
-      err.println(ex.getMessage());
+  public void checkContainerExists(ScmClient scmClient, long containerId)
+      throws IOException {
+    ContainerInfo container = scmClient.getContainer(containerId);
+    if (container == null) {
+      throw new IllegalArgumentException("No such container " + containerId);
     }
-    return null;
   }
 
-  @Override
-  protected int dispatch(CommandLine cmd, Options opts)
-      throws IOException, URISyntaxException {
-    OzoneCommandHandler handler = null;
-    try {
-      if (cmd.hasOption(ContainerCommandHandler.CONTAINER_CMD)) {
-        handler = new ContainerCommandHandler(scmClient);
-      }
-
-      if (handler == null) {
-        if (cmd.hasOption(HELP_OP)) {
-          displayHelp();
-          return SUCCESS;
-        } else {
-          displayHelp();
-          err.println("Unrecognized command: " + Arrays.asList(cmd.getArgs()));
-          return UNRECOGNIZED_CMD;
-        }
-      } else {
-        // Redirect stdout and stderr if necessary.
-        handler.setOut(this.out);
-        handler.setErr(this.err);
-        handler.execute(cmd);
-        return SUCCESS;
-      }
-    } catch (IOException ioe) {
-      err.println("Error executing command:" + ioe);
-      return EXECUTION_ERROR;
-    }
-  }
 }

+ 0 - 85
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java

@@ -1,85 +0,0 @@
-/**
- * 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.cli.container;
-
-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.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.cli.SCMCLI;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-
-/**
- * The handler of close container command.
- */
-public class CloseContainerHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_CLOSE = "close";
-  public static final String OPT_CONTAINER_ID = "c";
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    if (!cmd.hasOption(CONTAINER_CLOSE)) {
-      throw new IOException("Expecting container close");
-    }
-    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
-      displayHelp();
-      if (!cmd.hasOption(SCMCLI.HELP_OP)) {
-        throw new IOException("Expecting container id");
-      } else {
-        return;
-      }
-    }
-    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
-
-    ContainerWithPipeline container = getScmClient().
-        getContainerWithPipeline(Long.parseLong(containerID));
-    if (container == null) {
-      throw new IOException("Cannot close an non-exist container "
-          + containerID);
-    }
-    logOut("Closing container : %s.", containerID);
-    getScmClient()
-        .closeContainer(container.getContainerInfo().getContainerID());
-    logOut("Container closed.");
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addOptions(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter
-        .printHelp(SCMCLI.CMD_WIDTH, "hdfs scm -container -close <option>",
-            "where <option> is", options, "");
-  }
-
-  public static void addOptions(Options options) {
-    Option containerNameOpt = new Option(OPT_CONTAINER_ID,
-        true, "Specify container ID");
-    options.addOption(containerNameOpt);
-  }
-
-  CloseContainerHandler(ScmClient client) {
-    super(client);
-  }
-}

+ 54 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java

@@ -0,0 +1,54 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli.container;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * The handler of close container command.
+ */
+@Command(
+    name = "close",
+    description = "close container",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class CloseSubcommand implements Callable<Void> {
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Parameters(description = "Id of the container to close")
+  private long containerId;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+      parent.checkContainerExists(scmClient, containerId);
+      scmClient.closeContainer(containerId);
+      return null;
+    }
+  }
+}

+ 0 - 128
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java

@@ -1,128 +0,0 @@
-/**
- * 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.cli.container;
-
-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.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-import static org.apache.hadoop.hdds.scm.cli.container.CloseContainerHandler
-    .CONTAINER_CLOSE;
-import static org.apache.hadoop.hdds.scm.cli.container.CreateContainerHandler
-    .CONTAINER_CREATE;
-import static org.apache.hadoop.hdds.scm.cli.container.DeleteContainerHandler
-    .CONTAINER_DELETE;
-import static org.apache.hadoop.hdds.scm.cli.container.InfoContainerHandler
-    .CONTAINER_INFO;
-import static org.apache.hadoop.hdds.scm.cli.container.ListContainerHandler
-    .CONTAINER_LIST;
-
-/**
- * The handler class of container-specific commands, e.g. addContainer.
- */
-public class ContainerCommandHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_CMD = "container";
-
-  public ContainerCommandHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    // all container commands should contain -container option
-    if (!cmd.hasOption(CONTAINER_CMD)) {
-      throw new IOException("Expecting container cmd");
-    }
-    // check which each the sub command it is
-    OzoneCommandHandler handler = null;
-    if (cmd.hasOption(CONTAINER_CREATE)) {
-      handler = new CreateContainerHandler(getScmClient());
-    } else if (cmd.hasOption(CONTAINER_DELETE)) {
-      handler = new DeleteContainerHandler(getScmClient());
-    } else if (cmd.hasOption(CONTAINER_INFO)) {
-      handler = new InfoContainerHandler(getScmClient());
-    } else if (cmd.hasOption(CONTAINER_LIST)) {
-      handler = new ListContainerHandler(getScmClient());
-    } else if (cmd.hasOption(CONTAINER_CLOSE)) {
-      handler = new CloseContainerHandler(getScmClient());
-    }
-
-    // execute the sub command, throw exception if no sub command found
-    // unless -help option is given.
-    if (handler != null) {
-      handler.setOut(this.getOut());
-      handler.setErr(this.getErr());
-      handler.execute(cmd);
-    } else {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Unrecognized command "
-            + Arrays.asList(cmd.getArgs()));
-      }
-    }
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addCommandsOption(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH,
-        "hdfs scm -container <commands> <options>",
-        "where <commands> can be one of the following", options, "");
-  }
-
-  private static void addCommandsOption(Options options) {
-    Option createContainer =
-        new Option(CONTAINER_CREATE, false, "Create container");
-    Option infoContainer =
-        new Option(CONTAINER_INFO, false, "Info container");
-    Option deleteContainer =
-        new Option(CONTAINER_DELETE, false, "Delete container");
-    Option listContainer =
-        new Option(CONTAINER_LIST, false, "List container");
-    Option closeContainer =
-        new Option(CONTAINER_CLOSE, false, "Close container");
-
-    options.addOption(createContainer);
-    options.addOption(deleteContainer);
-    options.addOption(infoContainer);
-    options.addOption(listContainer);
-    options.addOption(closeContainer);
-    // Every new option should add it's option here.
-  }
-
-  public static void addOptions(Options options) {
-    addCommandsOption(options);
-    // for create container options.
-    DeleteContainerHandler.addOptions(options);
-    InfoContainerHandler.addOptions(options);
-    ListContainerHandler.addOptions(options);
-    CloseContainerHandler.addOptions(options);
-    // Every new option should add it's option here.
-  }
-}

+ 0 - 67
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java

@@ -1,67 +0,0 @@
-/**
- * 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.cli.container;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-
-/**
- * This is the handler that process container creation command.
- */
-public class CreateContainerHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_CREATE = "create";
-  public static final String CONTAINER_OWNER = "OZONE";
-  // TODO Support an optional -p <pipelineID> option to create
-  // container on given datanodes.
-
-  public CreateContainerHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    if (cmd.hasOption(HELP_OP)) {
-      displayHelp();
-    }
-
-    if (!cmd.hasOption(CONTAINER_CREATE)) {
-      throw new IOException("Expecting container create");
-    }
-
-    logOut("Creating container...");
-    getScmClient().createContainer(CONTAINER_OWNER);
-    logOut("Container created.");
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -create",
-        null, options, null);
-  }
-}

+ 65 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java

@@ -0,0 +1,65 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli.container;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .ContainerWithPipeline;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process container creation command.
+ */
+@Command(
+    name = "create",
+    description = "Create container",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class CreateSubcommand implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CreateSubcommand.class);
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Option(description = "Owner of the new container", defaultValue = "OZONE",
+      required = false, names = {
+      "-o", "--owner"})
+
+  private String owner;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+      ContainerWithPipeline container = scmClient.createContainer(owner);
+      LOG.info("Container {} is created.",
+          container.getContainerInfo().getContainerID());
+      return null;
+    }
+  }
+}

+ 0 - 95
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java

@@ -1,95 +0,0 @@
-/**
- * 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.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.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-
-/**
- * This is the handler that process delete container command.
- */
-public class DeleteContainerHandler extends OzoneCommandHandler {
-
-  protected static final String CONTAINER_DELETE = "delete";
-  protected static final String OPT_FORCE = "f";
-  protected static final String OPT_CONTAINER_ID = "c";
-
-  public DeleteContainerHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    Preconditions.checkArgument(cmd.hasOption(CONTAINER_DELETE),
-        "Expecting command delete");
-    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Expecting container name");
-      } else {
-        return;
-      }
-    }
-
-    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
-
-    ContainerWithPipeline container = getScmClient().getContainerWithPipeline(
-        Long.parseLong(containerID));
-    if (container == null) {
-      throw new IOException("Cannot delete an non-exist container "
-          + containerID);
-    }
-
-    logOut("Deleting container : %s.", containerID);
-    getScmClient()
-        .deleteContainer(container.getContainerInfo().getContainerID(),
-            container.getPipeline(), cmd.hasOption(OPT_FORCE));
-    logOut("Container %s deleted.", containerID);
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addOptions(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -delete <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);
-    Option containerNameOpt = new Option(OPT_CONTAINER_ID,
-        true, "Specify container id");
-    options.addOption(containerNameOpt);
-  }
-}

+ 60 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java

@@ -0,0 +1,60 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli.container;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process delete container command.
+ */
+@Command(
+    name = "delete",
+    description = "Delete container",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class DeleteSubcommand implements Callable<Void> {
+
+  @Parameters(description = "Id of the container to close")
+  private long containerId;
+
+  @Option(names = {"-f",
+      "--force"}, description = "forcibly delete the container")
+  private boolean force;
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+      parent.checkContainerExists(scmClient, containerId);
+      scmClient.deleteContainer(containerId, force);
+      return null;
+    }
+  }
+}

+ 0 - 114
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java

@@ -1,114 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.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.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerData;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerLifeCycleState;
-
-import java.io.IOException;
-import java.util.stream.Collectors;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-
-/**
- * This is the handler that process container info command.
- */
-public class InfoContainerHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_INFO = "info";
-  protected static final String OPT_CONTAINER_ID = "c";
-
-  /**
-   * Constructs a handler object.
-   *
-   * @param scmClient scm client.
-   */
-  public InfoContainerHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    if (!cmd.hasOption(CONTAINER_INFO)) {
-      throw new IOException("Expecting container info");
-    }
-    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Expecting container name");
-      } else {
-        return;
-      }
-    }
-    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
-    ContainerWithPipeline container = getScmClient().
-        getContainerWithPipeline(Long.parseLong(containerID));
-    Preconditions.checkNotNull(container, "Container cannot be null");
-
-    ContainerData containerData = getScmClient().readContainer(container
-        .getContainerInfo().getContainerID(), container.getPipeline());
-
-    // Print container report info.
-    logOut("Container id: %s", containerID);
-    String openStatus =
-        containerData.getState() == ContainerLifeCycleState.OPEN ? "OPEN" :
-            "CLOSED";
-    logOut("Container State: %s", openStatus);
-    logOut("Container Path: %s", containerData.getContainerPath());
-
-    // Output meta data.
-    String metadataStr = containerData.getMetadataList().stream().map(
-        p -> p.getKey() + ":" + p.getValue()).collect(Collectors.joining(", "));
-    logOut("Container Metadata: {%s}", metadataStr);
-
-    // Print pipeline of an existing container.
-    logOut("LeaderID: %s", container.getPipeline()
-        .getLeader().getHostName());
-    String machinesStr = container.getPipeline()
-        .getMachines().stream().map(
-        DatanodeDetails::getHostName).collect(Collectors.joining(","));
-    logOut("Datanodes: [%s]", machinesStr);
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addOptions(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -info <option>",
-        "where <option> is", options, "");
-  }
-
-  public static void addOptions(Options options) {
-    Option containerIdOpt = new Option(OPT_CONTAINER_ID,
-        true, "Specify container id");
-    options.addOption(containerIdOpt);
-  }
-}

+ 94 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java

@@ -0,0 +1,94 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli.container;
+
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerLifeCycleState;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .ContainerWithPipeline;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process container info command.
+ */
+@Command(
+    name = "info",
+    description = "Show information about a specific container",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class InfoSubcommand implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(InfoSubcommand.class);
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Parameters(description = "Decimal id of the container.")
+  private long containerID;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+      ContainerWithPipeline container = scmClient.
+          getContainerWithPipeline(containerID);
+      Preconditions.checkNotNull(container, "Container cannot be null");
+
+      ContainerData containerData = scmClient.readContainer(container
+          .getContainerInfo().getContainerID(), container.getPipeline());
+
+      // Print container report info.
+      LOG.info("Container id: {}", containerID);
+      String openStatus =
+          containerData.getState() == ContainerLifeCycleState.OPEN ? "OPEN" :
+              "CLOSED";
+      LOG.info("Container State: {}", openStatus);
+      LOG.info("Container Path: {}", containerData.getContainerPath());
+
+      // Output meta data.
+      String metadataStr = containerData.getMetadataList().stream().map(
+          p -> p.getKey() + ":" + p.getValue())
+          .collect(Collectors.joining(", "));
+      LOG.info("Container Metadata: {}", metadataStr);
+
+      // Print pipeline of an existing container.
+      LOG.info("LeaderID: {}", container.getPipeline()
+          .getLeader().getHostName());
+      String machinesStr = container.getPipeline()
+          .getMachines().stream().map(
+              DatanodeDetails::getHostName).collect(Collectors.joining(","));
+      LOG.info("Datanodes: [{}]", machinesStr);
+      return null;
+    }
+  }
+}

+ 0 - 117
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java

@@ -1,117 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.cli.container;
-
-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.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
-import org.apache.hadoop.ozone.web.utils.JsonUtils;
-
-import java.io.IOException;
-import java.util.List;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-
-/**
- * This is the handler that process container list command.
- */
-public class ListContainerHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_LIST = "list";
-  public static final String OPT_START_CONTAINER = "start";
-  public static final String OPT_COUNT = "count";
-
-  /**
-   * Constructs a handler object.
-   *
-   * @param scmClient scm client
-   */
-  public ListContainerHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    if (!cmd.hasOption(CONTAINER_LIST)) {
-      throw new IOException("Expecting container list");
-    }
-    if (cmd.hasOption(HELP_OP)) {
-      displayHelp();
-      return;
-    }
-
-    if (!cmd.hasOption(OPT_COUNT)) {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Expecting container count");
-      } else {
-        return;
-      }
-    }
-
-    String startID = cmd.getOptionValue(OPT_START_CONTAINER);
-    int count = 0;
-
-    if (cmd.hasOption(OPT_COUNT)) {
-      count = Integer.parseInt(cmd.getOptionValue(OPT_COUNT));
-      if (count < 0) {
-        displayHelp();
-        throw new IOException("-count should not be negative");
-      }
-    }
-
-    List<ContainerInfo> containerList =
-        getScmClient().listContainer(
-            Long.parseLong(startID), count);
-
-    // Output data list
-    for (ContainerInfo container : containerList) {
-      outputContainerInfo(container);
-    }
-  }
-
-  private void outputContainerInfo(ContainerInfo containerInfo)
-      throws IOException {
-    // Print container report info.
-    logOut("%s", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        containerInfo.toJsonString()));
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addOptions(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -list <option>",
-        "where <option> can be the following", options, "");
-  }
-
-  public static void addOptions(Options options) {
-    Option startContainerOpt = new Option(OPT_START_CONTAINER,
-        true, "Specify start container id");
-    Option countOpt = new Option(OPT_COUNT, true,
-        "Specify count number, required");
-    options.addOption(countOpt);
-    options.addOption(startContainerOpt);
-  }
-}

+ 83 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java

@@ -0,0 +1,83 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli.container;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.ozone.web.utils.JsonUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Help.Visibility;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process container list command.
+ */
+@Command(
+    name = "list",
+    description = "List containers",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class ListSubcommand implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ListSubcommand.class);
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Option(names = {"-s", "--start"},
+      description = "Container id to start the iteration", required = true)
+  private long startId;
+
+  @Option(names = {"-c", "--count"},
+      description = "Maximum number of containers to list",
+      defaultValue = "20", showDefaultValue = Visibility.ALWAYS)
+  private int count = 20;
+
+  private void outputContainerInfo(ContainerInfo containerInfo)
+      throws IOException {
+    // Print container report info.
+    LOG.info("{}", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+        containerInfo.toJsonString()));
+  }
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+
+      List<ContainerInfo> containerList =
+          scmClient.listContainer(startId, count);
+
+      // Output data list
+      for (ContainerInfo container : containerList) {
+        outputContainerInfo(container);
+      }
+      return null;
+    }
+  }
+}

+ 3 - 0
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/package-info.java

@@ -16,4 +16,7 @@
  * limitations under the License.
  */
 
+/**
+ * Contains all of the container related scm commands.
+ */
 package org.apache.hadoop.hdds.scm.cli.container;

+ 8 - 4
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/package-info.java

@@ -6,14 +6,18 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
+ * <p>
+ * SCM related cli tools.
+ */
+/**
+ * SCM related cli tools.
  */
-
 package org.apache.hadoop.hdds.scm.cli;

+ 1 - 1
hadoop-ozone/common/src/main/bin/ozone

@@ -116,7 +116,7 @@ function ozonecmd_case
       HADOOP_CLASSNAME=org.apache.hadoop.fs.FsShell
     ;;
     scmcli)
-      HADOOP_CLASSNAME=org.apache.hadoop.ozone.scm.cli.SCMCLI
+      HADOOP_CLASSNAME=org.apache.hadoop.hdds.scm.cli.SCMCLI
     ;;
     version)
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.util.OzoneVersionInfo

+ 0 - 518
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java

@@ -1,518 +0,0 @@
-/**
- * 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;
-
-import com.google.common.primitives.Longs;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.scm.cli.ResultCode;
-import org.apache.hadoop.hdds.scm.cli.SCMCLI;
-import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.client.ContainerOperationClient;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
-
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.OPEN;
-
-import static org.apache.hadoop.hdds.scm.cli.ResultCode.EXECUTION_ERROR;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-
-/**
- * This class tests the CLI of SCM.
- */
-@Ignore ("Needs to be fixed for new SCM and Storage design")
-public class TestSCMCli {
-  private static SCMCLI cli;
-
-  private static MiniOzoneCluster cluster;
-  private static OzoneConfiguration conf;
-  private static StorageContainerLocationProtocolClientSideTranslatorPB
-      storageContainerLocationClient;
-
-  private static StorageContainerManager scm;
-  private static ScmClient containerOperationClient;
-
-  private static ByteArrayOutputStream outContent;
-  private static PrintStream outStream;
-  private static ByteArrayOutputStream errContent;
-  private static PrintStream errStream;
-  private static XceiverClientManager xceiverClientManager;
-  private static String containerOwner = "OZONE";
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(30000);
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    conf = new OzoneConfiguration();
-    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
-    cluster.waitForClusterToBeReady();
-    xceiverClientManager = new XceiverClientManager(conf);
-    storageContainerLocationClient =
-        cluster.getStorageContainerLocationClient();
-    containerOperationClient = new ContainerOperationClient(
-        storageContainerLocationClient, new XceiverClientManager(conf));
-    outContent = new ByteArrayOutputStream();
-    outStream = new PrintStream(outContent);
-    errContent = new ByteArrayOutputStream();
-    errStream = new PrintStream(errContent);
-    cli = new SCMCLI(containerOperationClient, outStream, errStream);
-    scm = cluster.getStorageContainerManager();
-  }
-
-  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 {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-    IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
-  }
-
-  @Test
-  public void testCreateContainer() throws Exception {
-    ByteArrayOutputStream testContent = new ByteArrayOutputStream();
-    PrintStream testPrintOut = new PrintStream(testContent);
-    System.setOut(testPrintOut);
-    String[] args = {"-container", "-create"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args));
-    assertEquals("", testContent.toString());
-  }
-
-  private boolean containerExist(long containerID) {
-    try {
-      ContainerInfo container = scm.getClientProtocolServer()
-          .getContainerWithPipeline(containerID).getContainerInfo();
-      return container != null
-          && containerID == container.getContainerID();
-    } catch (IOException e) {
-      return false;
-    }
-  }
-
-  @Test
-  public void testDeleteContainer() throws Exception {
-    String containerName;
-    ContainerData containerData;
-    Pipeline pipeline;
-    String[] delCmd;
-    ByteArrayOutputStream testErr;
-    int exitCode;
-
-    // ****************************************
-    // 1. Test to delete a non-empty container.
-    // ****************************************
-    // Create an non-empty container
-    ContainerWithPipeline container = containerOperationClient
-        .createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    KeyValueContainerData kvData = KeyValueContainerData
-        .getFromProtoBuf(containerOperationClient.readContainer(
-            container.getContainerInfo().getContainerID(), container
-                .getPipeline()));
-    KeyUtils.getDB(kvData, conf)
-        .put(Longs.toByteArray(container.getContainerInfo().getContainerID()),
-            "someKey".getBytes());
-    Assert.assertTrue(containerExist(container.getContainerInfo()
-        .getContainerID()));
-
-    // Gracefully delete a container should fail because it is open.
-    delCmd = new String[]{"-container", "-delete", "-c",
-        Long.toString(container.getContainerInfo().getContainerID())};
-    testErr = new ByteArrayOutputStream();
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    exitCode = runCommandAndGetOutput(delCmd, out, testErr);
-    assertEquals(EXECUTION_ERROR, exitCode);
-    assertTrue(testErr.toString()
-        .contains("Deleting an open container is not allowed."));
-    Assert.assertTrue(
-        containerExist(container.getContainerInfo().getContainerID()));
-
-    // Close the container
-    containerOperationClient.closeContainer(
-        container.getContainerInfo().getContainerID());
-
-    // Gracefully delete a container should fail because it is not empty.
-    testErr = new ByteArrayOutputStream();
-    int exitCode2 = runCommandAndGetOutput(delCmd, out, testErr);
-    assertEquals(EXECUTION_ERROR, exitCode2);
-    assertTrue(testErr.toString()
-        .contains("Container cannot be deleted because it is not empty."));
-    Assert.assertTrue(
-        containerExist(container.getContainerInfo().getContainerID()));
-
-    // Try force delete again.
-    delCmd = new String[]{"-container", "-delete", "-c",
-        Long.toString(container.getContainerInfo().getContainerID()), "-f"};
-    exitCode = runCommandAndGetOutput(delCmd, out, null);
-    assertEquals("Expected success, found:", ResultCode.SUCCESS, exitCode);
-    assertFalse(containerExist(container.getContainerInfo().getContainerID()));
-
-    // ****************************************
-    // 2. Test to delete an empty container.
-    // ****************************************
-    // Create an empty container
-    ContainerWithPipeline emptyContainer = containerOperationClient
-        .createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    containerOperationClient
-        .closeContainer(emptyContainer.getContainerInfo().getContainerID());
-    Assert.assertTrue(
-        containerExist(emptyContainer.getContainerInfo().getContainerID()));
-
-    // Successfully delete an empty container.
-    delCmd = new String[]{"-container", "-delete", "-c",
-        Long.toString(emptyContainer.getContainerInfo().getContainerID())};
-    exitCode = runCommandAndGetOutput(delCmd, out, null);
-    assertEquals(ResultCode.SUCCESS, exitCode);
-    assertFalse(
-        containerExist(emptyContainer.getContainerInfo().getContainerID()));
-
-    // After the container is deleted,
-    // another container can now be recreated.
-    ContainerWithPipeline newContainer = containerOperationClient.
-        createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    Assert.assertTrue(
-        containerExist(newContainer.getContainerInfo().getContainerID()));
-
-    // ****************************************
-    // 3. Test to delete a non-exist container.
-    // ****************************************
-    long nonExistContainerID = ContainerTestHelper.getTestContainerID();
-    delCmd = new String[]{"-container", "-delete", "-c",
-        Long.toString(nonExistContainerID)};
-    testErr = new ByteArrayOutputStream();
-    exitCode = runCommandAndGetOutput(delCmd, out, testErr);
-    assertEquals(EXECUTION_ERROR, exitCode);
-    assertTrue(testErr.toString()
-        .contains("Specified key does not exist."));
-  }
-
-  @Test
-  public void testInfoContainer() throws Exception {
-    // The cluster has one Datanode server.
-    DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails();
-    String formatStr =
-        "Container id: %s\n" +
-        "Container State: %s\n" +
-        "Container DB Path: %s\n" +
-        "Container Path: %s\n" +
-        "Container Metadata: {%s}\n" +
-        "LeaderID: %s\n" +
-        "Datanodes: [%s]\n";
-
-    // Test a non-exist container
-    String containerID =
-        Long.toString(ContainerTestHelper.getTestContainerID());
-    String[] info = {"-container", "-info", containerID};
-    int exitCode = runCommandAndGetOutput(info, null, null);
-    assertEquals("Expected Execution Error, Did not find that.",
-        EXECUTION_ERROR, exitCode);
-
-    // Create an empty container.
-    ContainerWithPipeline container = containerOperationClient
-        .createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    KeyValueContainerData data = KeyValueContainerData
-        .getFromProtoBuf(containerOperationClient.
-            readContainer(container.getContainerInfo().getContainerID(),
-                container.getPipeline()));
-    info = new String[]{"-container", "-info", "-c",
-        Long.toString(container.getContainerInfo().getContainerID())};
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    exitCode = runCommandAndGetOutput(info, out, null);
-    assertEquals("Expected Success, did not find it.", ResultCode.SUCCESS,
-        exitCode);
-
-    String openStatus = data.isOpen() ? "OPEN" : "CLOSED";
-    String expected =
-        String.format(formatStr, container.getContainerInfo().getContainerID
-                (), openStatus, data.getDbFile().getPath(), data
-                .getContainerPath(), "", datanodeDetails.getHostName(),
-            datanodeDetails.getHostName());
-
-    assertEquals(expected, out.toString());
-
-    out.reset();
-
-    // Create an non-empty container
-    container = containerOperationClient
-        .createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    data = KeyValueContainerData
-        .getFromProtoBuf(containerOperationClient.readContainer(
-            container.getContainerInfo().getContainerID(), container
-                .getPipeline()));
-    KeyUtils.getDB(data, conf)
-        .put(containerID.getBytes(), "someKey".getBytes());
-
-    info = new String[]{"-container", "-info", "-c",
-        Long.toString(container.getContainerInfo().getContainerID())};
-    exitCode = runCommandAndGetOutput(info, out, null);
-    assertEquals(ResultCode.SUCCESS, exitCode);
-
-    openStatus = data.isOpen() ? "OPEN" : "CLOSED";
-
-    expected = String.format(formatStr, container.getContainerInfo()
-            .getContainerID(), openStatus, data.getDbFile().getPath(), data
-            .getContainerPath(), "", datanodeDetails.getHostName(),
-        datanodeDetails.getHostName());
-    assertEquals(expected, out.toString());
-
-    out.reset();
-
-    // Close last container and test info again.
-    containerOperationClient
-        .closeContainer(container.getContainerInfo().getContainerID());
-
-    info = new String[]{"-container", "-info", "-c",
-        Long.toString(container.getContainerInfo().getContainerID())};
-    exitCode = runCommandAndGetOutput(info, out, null);
-    assertEquals(ResultCode.SUCCESS, exitCode);
-    data = KeyValueContainerData
-        .getFromProtoBuf(containerOperationClient.readContainer(
-            container.getContainerInfo().getContainerID(), container
-                .getPipeline()));
-
-    openStatus = data.isOpen() ? "OPEN" : "CLOSED";
-    expected = String
-        .format(formatStr, container.getContainerInfo().getContainerID(),
-            openStatus, data.getDbFile().getPath(), data.getContainerPath(), "",
-            datanodeDetails.getHostName(), datanodeDetails.getHostName());
-    assertEquals(expected, out.toString());
-  }
-
-  @Test
-  public void testNonExistCommand() throws Exception {
-    PrintStream init = System.out;
-    ByteArrayOutputStream testContent = new ByteArrayOutputStream();
-    PrintStream testPrintOut = new PrintStream(testContent);
-    System.setOut(testPrintOut);
-    String[] args = {"-nothingUseful"};
-    assertEquals(ResultCode.UNRECOGNIZED_CMD, cli.run(args));
-    assertTrue(errContent.toString()
-        .contains("Unrecognized options:[-nothingUseful]"));
-    String expectedOut =
-        "usage: hdfs scmcli <commands> [<options>]\n" +
-        "where <commands> can be one of the following\n" +
-        " -container   Container related options\n";
-    assertEquals(expectedOut, testContent.toString());
-    System.setOut(init);
-  }
-
-  @Test
-  public void testListContainerCommand() throws Exception {
-    // Create 20 containers for testing.
-    List<ContainerInfo> containers = new ArrayList<>();
-    for (int index = 0; index < 20; index++) {
-      ContainerWithPipeline container = containerOperationClient.createContainer(
-          xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE,
-          containerOwner);
-      containers.add(container.getContainerInfo());
-    }
-
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    ByteArrayOutputStream err = new ByteArrayOutputStream();
-
-    // Test without -start, -prefix and -count
-    String[] args = new String[] {"-container", "-list"};
-    int exitCode = runCommandAndGetOutput(args, out, err);
-    assertEquals(EXECUTION_ERROR, exitCode);
-    assertTrue(err.toString()
-        .contains("Expecting container count"));
-
-    out.reset();
-    err.reset();
-
-    long startContainerID = containers.get(0).getContainerID();
-    String startContainerIDStr = Long.toString(startContainerID);
-    // Test with -start and -count, the value of -count is negative.
-    args = new String[] {"-container", "-list",
-        "-start", startContainerIDStr, "-count", "-1"};
-    exitCode = runCommandAndGetOutput(args, out, err);
-    assertEquals(EXECUTION_ERROR, exitCode);
-    assertTrue(err.toString()
-        .contains("-count should not be negative"));
-
-    out.reset();
-    err.reset();
-
-    // Test with -start and -count.
-    args = new String[] {"-container", "-list", "-start",
-        startContainerIDStr, "-count", "10"};
-    exitCode = runCommandAndGetOutput(args, out, err);
-    assertEquals(ResultCode.SUCCESS, exitCode);
-    for (int index = 1; index < 10; index++) {
-      String containerID = Long.toString(
-          containers.get(index).getContainerID());
-      assertTrue(out.toString().contains(containerID));
-    }
-
-    out.reset();
-    err.reset();
-
-    // Test with -start, while -count doesn't exist.
-    args = new String[] {"-container", "-list", "-start",
-        startContainerIDStr};
-    exitCode = runCommandAndGetOutput(args, out, err);
-    assertEquals(ResultCode.EXECUTION_ERROR, exitCode);
-    assertTrue(err.toString().contains(
-        "java.io.IOException: Expecting container count"));
-  }
-
-  @Test
-  public void testCloseContainer() throws Exception {
-    long containerID = containerOperationClient.createContainer(
-        xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE,
-        containerOwner).getContainerInfo().getContainerID();
-    ContainerInfo container = scm.getClientProtocolServer()
-        .getContainerWithPipeline(containerID).getContainerInfo();
-    assertNotNull(container);
-    assertEquals(containerID, container.getContainerID());
-
-    ContainerInfo containerInfo = scm.getContainerInfo(containerID);
-    assertEquals(OPEN, containerInfo.getState());
-
-    String[] args1 = {"-container", "-close", "-c",
-        Long.toString(containerID)};
-    assertEquals(ResultCode.SUCCESS, cli.run(args1));
-
-    containerInfo = scm.getContainerInfo(containerID);
-    assertEquals(CLOSED, containerInfo.getState());
-
-    // closing this container again will trigger an error.
-    assertEquals(EXECUTION_ERROR, cli.run(args1));
-  }
-
-  @Test
-  public void testHelp() throws Exception {
-    // TODO : this test assertion may break for every new help entry added
-    // may want to disable this test some time later. For now, mainly to show
-    // case the format of help output.
-    PrintStream init = System.out;
-    ByteArrayOutputStream testContent = new ByteArrayOutputStream();
-    PrintStream testPrintOut = new PrintStream(testContent);
-    System.setOut(testPrintOut);
-    String[] args = {"-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args));
-    String expected =
-        "usage: hdfs scmcli <commands> [<options>]\n" +
-        "where <commands> can be one of the following\n" +
-        " -container   Container related options\n";
-    assertEquals(expected, testContent.toString());
-    testContent.reset();
-
-    String[] args1 = {"-container", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args1));
-    String expected1 =
-        "usage: hdfs scm -container <commands> <options>\n" +
-        "where <commands> can be one of the following\n" +
-        " -close    Close container\n" +
-        " -create   Create container\n" +
-        " -delete   Delete container\n" +
-        " -info     Info container\n" +
-        " -list     List container\n";
-
-    assertEquals(expected1, testContent.toString());
-    testContent.reset();
-
-    String[] args2 = {"-container", "-create", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args2));
-    String expected2 =
-        "usage: hdfs scm -container -create\n\n";
-    assertEquals(expected2, testContent.toString());
-    testContent.reset();
-
-    String[] args3 = {"-container", "-delete", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args3));
-    String expected3 =
-        "usage: hdfs scm -container -delete <option>\n" +
-        "where <option> is\n" +
-        " -c <arg>   Specify container id\n" +
-        " -f         forcibly delete a container\n";
-    assertEquals(expected3, testContent.toString());
-    testContent.reset();
-
-    String[] args4 = {"-container", "-info", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args4));
-    String expected4 =
-        "usage: hdfs scm -container -info <option>\n" +
-        "where <option> is\n" +
-        " -c <arg>   Specify container id\n";
-    assertEquals(expected4, testContent.toString());
-    testContent.reset();
-
-    String[] args5 = {"-container", "-list", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args5));
-    String expected5 = "usage: hdfs scm -container -list <option>\n"
-        + "where <option> can be the following\n"
-        + " -count <arg>   Specify count number, required\n"
-        + " -start <arg>   Specify start container id\n";
-    assertEquals(expected5, testContent.toString());
-    testContent.reset();
-
-    System.setOut(init);
-  }
-}