You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2018/09/07 20:03:06 UTC

[1/3] hadoop git commit: HDDS-351. Add chill mode state to SCM. Contributed by Ajay Kumar.

Repository: hadoop
Updated Branches:
  refs/heads/ozone-0.2 be1ec005f -> fd792ce54


HDDS-351. Add chill mode state to SCM.
Contributed by Ajay Kumar.

(cherry picked from commit ff64d3571660ace3fb266ee47bea181cebfee8d9)


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

Branch: refs/heads/ozone-0.2
Commit: 48bcebc0805df7428549b067f26620bad7940d9f
Parents: be1ec00
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Sep 7 10:35:45 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Sep 7 11:00:31 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/HddsConfigKeys.java  |   6 +
 .../common/src/main/resources/ozone-default.xml |   9 +
 .../hdds/scm/container/ContainerMapping.java    |   5 +
 .../hadoop/hdds/scm/events/SCMEvents.java       |  10 +
 .../hdds/scm/server/SCMChillModeManager.java    | 198 +++++++++++++++++++
 .../scm/server/SCMDatanodeProtocolServer.java   |  19 ++
 .../scm/server/StorageContainerManager.java     |  17 +-
 .../apache/hadoop/hdds/scm/HddsTestUtils.java   |  85 ++++++++
 .../scm/server/TestSCMChillModeManager.java     | 115 +++++++++++
 .../apache/hadoop/ozone/MiniOzoneCluster.java   |  23 ++-
 .../hadoop/ozone/MiniOzoneClusterImpl.java      |  93 ++++++---
 .../ozone/TestStorageContainerManager.java      | 170 ++++++++++++----
 12 files changed, 683 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index 4dc7e0a..98efbf8 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
@@ -75,4 +75,10 @@ public final class HddsConfigKeys {
       "hdds.container.close.threshold";
   public static final float HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT = 0.9f;
 
+  // % of containers which should have at least one reported replica
+  // before SCM comes out of chill mode.
+  public static final String HDDS_SCM_CHILLMODE_THRESHOLD_PCT =
+      "hdds.scm.chillmode.threshold.pct";
+  public static final double HDDS_SCM_CHILLMODE_THRESHOLD_PCT_DEFAULT = 0.99;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 778d641..be19e90 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1113,6 +1113,15 @@
   </property>
 
   <property>
+    <name>hdds.scm.chillmode.threshold.pct</name>
+    <value>0.99</value>
+    <tag>HDDS,SCM,OPERATION</tag>
+    <description> % of containers which should have at least one
+      reported replica before SCM comes out of chill mode.
+    </description>
+  </property>
+
+  <property>
     <name>hdds.container.action.max.limit</name>
     <value>20</value>
     <tag>DATANODE</tag>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index 3554339..5678205 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -210,6 +211,10 @@ public class ContainerMapping implements Mapping {
         // For close containers create pipeline from datanodes with replicas
         Set<DatanodeDetails> dnWithReplicas = containerStateManager
             .getContainerReplicas(contInfo.containerID());
+        if (dnWithReplicas.size() == 0) {
+          throw new SCMException("Can't create a pipeline for container with "
+              + "no replica.", ResultCodes.NO_REPLICA_FOUND);
+        }
         pipeline =
             new Pipeline(dnWithReplicas.iterator().next().getUuidString(),
                 contInfo.getState(), ReplicationType.STAND_ALONE,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
index 03df8eb..6985834 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager
     .ReplicationCompleted;
 import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
 
+import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport;
 import org.apache.hadoop.hdds.server.events.Event;
 import org.apache.hadoop.hdds.server.events.TypedEvent;
 import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
@@ -60,6 +61,15 @@ public final class SCMEvents {
    */
   public static final TypedEvent<NodeReportFromDatanode> NODE_REPORT =
       new TypedEvent<>(NodeReportFromDatanode.class, "Node_Report");
+
+  /**
+   * Event generated on DataNode registration.
+   */
+  public static final TypedEvent<NodeRegistrationContainerReport>
+      NODE_REGISTRATION_CONT_REPORT = new TypedEvent<>(
+      NodeRegistrationContainerReport.class,
+      "Node_Registration_Container_Report");
+
   /**
    * ContainerReports are send out by Datanodes. This report is received by
    * SCMDatanodeHeartbeatDispatcher and Container_Report Event

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMChillModeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMChillModeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMChillModeManager.java
new file mode 100644
index 0000000..d278637
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMChillModeManager.java
@@ -0,0 +1,198 @@
+/*
+ * 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.server;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
+    .NodeRegistrationContainerReport;
+import org.apache.hadoop.hdds.server.events.EventHandler;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * StorageContainerManager enters chill mode on startup to allow system to
+ * reach a stable state before becoming fully functional. SCM will wait
+ * for certain resources to be reported before coming out of chill mode.
+ *
+ * ChillModeExitRule defines format to define new rules which must be satisfied
+ * to exit Chill mode.
+ * ContainerChillModeRule defines the only exit criteria right now.
+ * On every new datanode registration event this class adds replicas
+ * for reported containers and validates if cutoff threshold for
+ * containers is meet.
+ */
+public class SCMChillModeManager implements
+    EventHandler<NodeRegistrationContainerReport> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SCMChillModeManager.class);
+  private AtomicBoolean inChillMode = new AtomicBoolean(true);
+  private AtomicLong containerWithMinReplicas = new AtomicLong(0);
+  private Map<String, ChillModeExitRule> exitRules = new HashMap(1);
+  private Configuration config;
+  private static final String CONT_EXIT_RULE = "ContainerChillModeRule";
+
+  SCMChillModeManager(Configuration conf, List<ContainerInfo> allContainers) {
+    this.config = conf;
+    exitRules
+        .put(CONT_EXIT_RULE, new ContainerChillModeRule(config, allContainers));
+  }
+
+  private void validateChillModeExitRules(EventPublisher eventQueue) {
+    for (ChillModeExitRule exitRule : exitRules.values()) {
+      if (!exitRule.validate()) {
+        return;
+      }
+    }
+    exitChillMode(eventQueue);
+  }
+
+  private void exitChillMode(EventPublisher eventQueue) {
+    LOG.info("SCM exiting chill mode.");
+    setInChillMode(false);
+    // Emit event to ReplicationManager to start replication.
+    eventQueue.fireEvent(SCMEvents.START_REPLICATION, true);
+
+    // TODO: Remove handler registration as there is no need to listen to
+    // register events anymore.
+
+    for (ChillModeExitRule e : exitRules.values()) {
+      e.cleanup();
+    }
+  }
+
+  @Override
+  public void onMessage(
+      NodeRegistrationContainerReport nodeRegistrationContainerReport,
+      EventPublisher publisher) {
+    if (getInChillMode()) {
+      exitRules.get(CONT_EXIT_RULE).process(nodeRegistrationContainerReport);
+      validateChillModeExitRules(publisher);
+    }
+  }
+
+  public boolean getInChillMode() {
+    return inChillMode.get();
+  }
+
+  public void setInChillMode(boolean inChillMode) {
+    this.inChillMode.set(inChillMode);
+  }
+
+  /**
+   * Interface for defining chill mode exit rules.
+   *
+   * @param <T>
+   */
+  public interface ChillModeExitRule<T> {
+
+    boolean validate();
+
+    void process(T report);
+
+    void cleanup();
+  }
+
+  /**
+   * Class defining Chill mode exit criteria for Containers.
+   */
+  public class ContainerChillModeRule implements
+      ChillModeExitRule<NodeRegistrationContainerReport> {
+
+    // Required cutoff % for containers with at least 1 reported replica.
+    private double chillModeCutoff;
+    // Containers read from scm db.
+    private Map<Long, ContainerInfo> containerMap;
+    private double maxContainer;
+
+    public ContainerChillModeRule(Configuration conf,
+        List<ContainerInfo> containers) {
+      chillModeCutoff = conf
+          .getDouble(HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT,
+              HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT_DEFAULT);
+      containerMap = new ConcurrentHashMap<>();
+      if(containers != null) {
+        containers.forEach(c -> {
+          if (c != null) {
+            containerMap.put(c.getContainerID(), c);
+          }
+        });
+        maxContainer = containers.size();
+      }
+    }
+
+    @Override
+    public boolean validate() {
+      if (maxContainer == 0) {
+        return true;
+      }
+      return getCurrentContainerThreshold() >= chillModeCutoff;
+    }
+
+    @VisibleForTesting
+    public double getCurrentContainerThreshold() {
+      return (containerWithMinReplicas.doubleValue() / maxContainer);
+    }
+
+    @Override
+    public void process(NodeRegistrationContainerReport reportsProto) {
+      if (maxContainer == 0) {
+        // No container to check.
+        return;
+      }
+
+      reportsProto.getReport().getReportsList().forEach(c -> {
+        if (containerMap.containsKey(c.getContainerID())) {
+          if(containerMap.remove(c.getContainerID()) != null) {
+            containerWithMinReplicas.getAndAdd(1);
+          }
+        }
+      });
+
+      LOG.info("SCM in chill mode. {} % containers have at least one reported "
+          + "replica.", (containerWithMinReplicas.get() / maxContainer) * 100);
+    }
+
+    @Override
+    public void cleanup() {
+      containerMap.clear();
+    }
+  }
+
+  @VisibleForTesting
+  public static Logger getLogger() {
+    return LOG;
+  }
+
+  @VisibleForTesting
+  public double getCurrentContainerThreshold() {
+    return ((ContainerChillModeRule) exitRules.get(CONT_EXIT_RULE))
+        .getCurrentContainerThreshold();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
index 9215803..8a09dc8 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
@@ -73,6 +73,8 @@ import static org.apache.hadoop.hdds.protocol.proto
 
 
 import org.apache.hadoop.hdds.scm.HddsServerUtil;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ReportFromDatanode;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -120,6 +122,7 @@ public class SCMDatanodeProtocolServer implements
   private final StorageContainerManager scm;
   private final InetSocketAddress datanodeRpcAddress;
   private final SCMDatanodeHeartbeatDispatcher heartbeatDispatcher;
+  private final EventPublisher eventPublisher;
 
   public SCMDatanodeProtocolServer(final OzoneConfiguration conf,
       StorageContainerManager scm, EventPublisher eventPublisher)
@@ -129,6 +132,7 @@ public class SCMDatanodeProtocolServer implements
     Preconditions.checkNotNull(eventPublisher, "EventPublisher cannot be null");
 
     this.scm = scm;
+    this.eventPublisher = eventPublisher;
     final int handlerCount =
         conf.getInt(OZONE_SCM_HANDLER_COUNT_KEY,
             OZONE_SCM_HANDLER_COUNT_DEFAULT);
@@ -197,6 +201,9 @@ public class SCMDatanodeProtocolServer implements
         == SCMRegisteredResponseProto.ErrorCode.success) {
       scm.getScmContainerManager().processContainerReports(datanodeDetails,
           containerReportsProto, true);
+      eventPublisher.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
+          new NodeRegistrationContainerReport(datanodeDetails,
+              containerReportsProto));
     }
     return getRegisteredResponse(registeredCommand);
   }
@@ -305,4 +312,16 @@ public class SCMDatanodeProtocolServer implements
     IOUtils.cleanupWithLogger(LOG, scm.getScmNodeManager());
   }
 
+  /**
+   * Wrapper class for events with the datanode origin.
+   */
+  public static class NodeRegistrationContainerReport extends
+      ReportFromDatanode<ContainerReportsProto> {
+
+    public NodeRegistrationContainerReport(DatanodeDetails datanodeDetails,
+        ContainerReportsProto report) {
+      super(datanodeDetails, report);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index b84f399..f505430 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -176,6 +176,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
   private final LeaseManager<Long> commandWatcherLeaseManager;
 
   private final ReplicationActivityStatus replicationStatus;
+  private final SCMChillModeManager scmChillModeManager;
 
   /**
    * Creates a new StorageContainerManager. Configuration will be updated
@@ -231,7 +232,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
     ContainerReportHandler containerReportHandler =
         new ContainerReportHandler(scmContainerManager, node2ContainerMap,
             replicationStatus);
-
+    scmChillModeManager = new SCMChillModeManager(conf,
+        getScmContainerManager().getStateManager().getAllContainers());
     PipelineActionEventHandler pipelineActionEventHandler =
         new PipelineActionEventHandler();
 
@@ -253,6 +255,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
     eventQueue.addHandler(SCMEvents.PIPELINE_ACTIONS,
         pipelineActionEventHandler);
     eventQueue.addHandler(SCMEvents.PIPELINE_CLOSE, pipelineCloseHandler);
+    eventQueue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
+        scmChillModeManager);
 
     long watcherTimeout =
         conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
@@ -619,9 +623,9 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
         "server", getDatanodeProtocolServer().getDatanodeRpcAddress()));
     getDatanodeProtocolServer().start();
 
-    replicationStatus.start();
     httpServer.start();
     scmBlockManager.start();
+    replicationStatus.start();
     replicationManager.start();
     setStartTime();
   }
@@ -809,6 +813,15 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
     return id2StatMap;
   }
 
+  public boolean isInChillMode() {
+    return scmChillModeManager.getInChillMode();
+  }
+
+  @VisibleForTesting
+  public double getCurrentContainerThreshold() {
+    return scmChillModeManager.getCurrentContainerThreshold();
+  }
+
   /**
    * Startup options.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
new file mode 100644
index 0000000..50d1eed
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/HddsTestUtils.java
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
+    .NodeRegistrationContainerReport;
+
+/**
+ * Stateless helper functions for Hdds tests.
+ */
+public final class HddsTestUtils {
+
+  private HddsTestUtils() {
+  }
+
+  /**
+   * Create Command Status report object.
+   *
+   * @param numOfContainers number of containers to be included in report.
+   * @return CommandStatusReportsProto
+   */
+  public static NodeRegistrationContainerReport
+      createNodeRegistrationContainerReport(int numOfContainers) {
+    return new NodeRegistrationContainerReport(
+        TestUtils.randomDatanodeDetails(),
+        TestUtils.getRandomContainerReports(numOfContainers));
+  }
+
+  /**
+   * Create NodeRegistrationContainerReport object.
+   *
+   * @param dnContainers List of containers to be included in report
+   * @return NodeRegistrationContainerReport
+   */
+  public static NodeRegistrationContainerReport
+      createNodeRegistrationContainerReport(List<ContainerInfo> dnContainers) {
+    List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
+        containers = new ArrayList<>();
+    dnContainers.forEach(c -> {
+      containers.add(TestUtils.getRandomContainerInfo(c.getContainerID()));
+    });
+    return new NodeRegistrationContainerReport(
+        TestUtils.randomDatanodeDetails(),
+        TestUtils.getContainerReports(containers));
+  }
+
+  /**
+   * Creates list of ContainerInfo.
+   *
+   * @param numContainers number of ContainerInfo to be included in list.
+   * @return List<ContainerInfo>
+   */
+  public static List<ContainerInfo> getContainerInfo(int numContainers) {
+    List<ContainerInfo> containerInfoList = new ArrayList<>();
+    for (int i = 0; i < numContainers; i++) {
+      ContainerInfo.Builder builder = new ContainerInfo.Builder();
+      containerInfoList.add(builder
+          .setContainerID(RandomUtils.nextLong())
+          .build());
+    }
+    return containerInfoList;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMChillModeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMChillModeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMChillModeManager.java
new file mode 100644
index 0000000..e98a9ae
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMChillModeManager.java
@@ -0,0 +1,115 @@
+/*
+ * 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.server;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.HddsTestUtils;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.server.events.EventQueue;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+/** Test class for SCMChillModeManager.
+ */
+public class TestSCMChillModeManager {
+
+  private static EventQueue queue;
+  private SCMChillModeManager scmChillModeManager;
+  private static Configuration config;
+  private List<ContainerInfo> containers;
+
+  @Rule
+  public Timeout timeout = new Timeout(1000 * 20);
+
+  @BeforeClass
+  public static void setUp() {
+    queue = new EventQueue();
+    config = new OzoneConfiguration();
+  }
+
+  @Test
+  public void testChillModeState() throws Exception {
+    // Test 1: test for 0 containers
+    testChillMode(0);
+
+    // Test 2: test for 20 containers
+    testChillMode(20);
+  }
+
+  @Test
+  public void testChillModeStateWithNullContainers() {
+    new SCMChillModeManager(config, null);
+  }
+
+  private void testChillMode(int numContainers) throws Exception {
+    containers = new ArrayList<>();
+    containers.addAll(HddsTestUtils.getContainerInfo(numContainers));
+    scmChillModeManager = new SCMChillModeManager(config, containers);
+    queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
+        scmChillModeManager);
+    assertTrue(scmChillModeManager.getInChillMode());
+    queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
+        HddsTestUtils.createNodeRegistrationContainerReport(containers));
+    GenericTestUtils.waitFor(() -> {
+      return !scmChillModeManager.getInChillMode();
+    }, 100, 1000 * 5);
+  }
+
+  @Test
+  public void testChillModeExitRule() throws Exception {
+    containers = new ArrayList<>();
+    containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
+    scmChillModeManager = new SCMChillModeManager(config, containers);
+    queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
+        scmChillModeManager);
+    assertTrue(scmChillModeManager.getInChillMode());
+
+    testContainerThreshold(containers.subList(0, 25), 0.25);
+    assertTrue(scmChillModeManager.getInChillMode());
+    testContainerThreshold(containers.subList(25, 50), 0.50);
+    assertTrue(scmChillModeManager.getInChillMode());
+    testContainerThreshold(containers.subList(50, 75), 0.75);
+    assertTrue(scmChillModeManager.getInChillMode());
+    testContainerThreshold(containers.subList(75, 100), 1.0);
+
+    GenericTestUtils.waitFor(() -> {
+      return !scmChillModeManager.getInChillMode();
+    }, 100, 1000 * 5);
+  }
+
+  private void testContainerThreshold(List<ContainerInfo> dnContainers,
+      double expectedThreshold)
+      throws Exception {
+    queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
+        HddsTestUtils.createNodeRegistrationContainerReport(dnContainers));
+    GenericTestUtils.waitFor(() -> {
+      double threshold = scmChillModeManager.getCurrentContainerThreshold();
+      return threshold == expectedThreshold;
+    }, 100, 2000 * 9);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
index e11cf9b..3cba839 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
@@ -178,11 +178,26 @@ public interface MiniOzoneCluster {
   void shutdownHddsDatanode(DatanodeDetails dn) throws IOException;
 
   /**
-   * Shutdown the MiniOzoneCluster.
+   * Shutdown the MiniOzoneCluster and delete the storage dirs.
    */
   void shutdown();
 
   /**
+   * Stop the MiniOzoneCluster without any cleanup.
+   */
+  void stop();
+
+  /**
+   * Start Scm.
+   */
+  void startScm() throws IOException;
+
+  /**
+   * Start DataNodes.
+   */
+  void startHddsDatanodes();
+
+  /**
    * Builder class for MiniOzoneCluster.
    */
   @SuppressWarnings("CheckStyle")
@@ -209,6 +224,7 @@ public interface MiniOzoneCluster {
     protected int numOfOmHandlers = 20;
     protected int numOfScmHandlers = 20;
     protected int numOfDatanodes = 1;
+    protected boolean  startDataNodes = true;
 
     protected Builder(OzoneConfiguration conf) {
       this.conf = conf;
@@ -229,6 +245,11 @@ public interface MiniOzoneCluster {
       return this;
     }
 
+    public Builder setStartDataNodes(boolean startDataNodes) {
+      this.startDataNodes = startDataNodes;
+      return this;
+    }
+
     /**
      * Sets the SCM id.
      *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index 7b9bb0e..c2169a3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.common.Storage.StorageState;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.hdds.scm.server.SCMStorage;
@@ -276,33 +277,55 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
       File baseDir = new File(GenericTestUtils.getTempPath(
           MiniOzoneClusterImpl.class.getSimpleName() + "-" +
               scm.getClientProtocolServer().getScmInfo().getClusterId()));
+      stop();
       FileUtils.deleteDirectory(baseDir);
+    } catch (IOException e) {
+      LOG.error("Exception while shutting down the cluster.", e);
+    }
+  }
 
-      if (ozoneManager != null) {
-        LOG.info("Shutting down the OzoneManager");
-        ozoneManager.stop();
-        ozoneManager.join();
-      }
+  @Override
+  public void stop() {
+    LOG.info("Stopping the Mini Ozone Cluster");
+    if (ozoneManager != null) {
+      LOG.info("Stopping the OzoneManager");
+      ozoneManager.stop();
+      ozoneManager.join();
+    }
 
-      if (scm != null) {
-        LOG.info("Shutting down the StorageContainerManager");
-        scm.stop();
-        scm.join();
-      }
+    if (scm != null) {
+      LOG.info("Stopping the StorageContainerManager");
+      scm.stop();
+      scm.join();
+    }
 
-      if (!hddsDatanodes.isEmpty()) {
-        LOG.info("Shutting down the HddsDatanodes");
-        for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
-          hddsDatanode.stop();
-          hddsDatanode.join();
-        }
+    if (!hddsDatanodes.isEmpty()) {
+      LOG.info("Shutting the HddsDatanodes");
+      for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+        hddsDatanode.stop();
+        hddsDatanode.join();
       }
-    } catch (IOException e) {
-      LOG.error("Exception while shutting down the cluster.", e);
     }
   }
 
   /**
+   * Start Scm.
+   */
+  @Override
+  public void startScm() throws IOException {
+    scm.start();
+  }
+
+  /**
+   * Start DataNodes.
+   */
+  @Override
+  public void startHddsDatanodes() {
+    hddsDatanodes.forEach((datanode) -> datanode.start(null));
+  }
+
+
+  /**
    * Builder for configuring the MiniOzoneCluster to run.
    */
   public static class Builder extends MiniOzoneCluster.Builder {
@@ -324,9 +347,13 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
       scm.start();
       OzoneManager om = createOM();
       om.start();
-      List<HddsDatanodeService> hddsDatanodes = createHddsDatanodes(scm);
-      hddsDatanodes.forEach((datanode) -> datanode.start(null));
-      return new MiniOzoneClusterImpl(conf, om, scm, hddsDatanodes);
+      final List<HddsDatanodeService> hddsDatanodes = createHddsDatanodes(scm);
+      MiniOzoneClusterImpl cluster = new MiniOzoneClusterImpl(conf, om, scm,
+          hddsDatanodes);
+      if (startDataNodes) {
+        cluster.startHddsDatanodes();
+      }
+      return cluster;
     }
 
     /**
@@ -352,13 +379,30 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
     private StorageContainerManager createSCM() throws IOException {
       configureSCM();
       SCMStorage scmStore = new SCMStorage(conf);
+      initializeScmStorage(scmStore);
+      return StorageContainerManager.createSCM(null, conf);
+    }
+
+    private void initializeScmStorage(SCMStorage scmStore) throws IOException {
+      if (scmStore.getState() == StorageState.INITIALIZED) {
+        return;
+      }
       scmStore.setClusterId(clusterId);
       if (!scmId.isPresent()) {
         scmId = Optional.of(UUID.randomUUID().toString());
       }
       scmStore.setScmId(scmId.get());
       scmStore.initialize();
-      return StorageContainerManager.createSCM(null, conf);
+    }
+
+    private void initializeOmStorage(OMStorage omStorage) throws IOException{
+      if (omStorage.getState() == StorageState.INITIALIZED) {
+        return;
+      }
+      omStorage.setClusterId(clusterId);
+      omStorage.setScmId(scmId.get());
+      omStorage.setOmId(omId.orElse(UUID.randomUUID().toString()));
+      omStorage.initialize();
     }
 
     /**
@@ -371,10 +415,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
     private OzoneManager createOM() throws IOException {
       configureOM();
       OMStorage omStore = new OMStorage(conf);
-      omStore.setClusterId(clusterId);
-      omStore.setScmId(scmId.get());
-      omStore.setOmId(omId.orElse(UUID.randomUUID().toString()));
-      omStore.initialize();
+      initializeOmStorage(omStore);
       return OzoneManager.createOm(null, conf);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48bcebc0/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
index 8762c0e..1364d77 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
@@ -17,59 +17,63 @@
  */
 package org.apache.hadoop.ozone;
 
-import java.io.IOException;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.AtomicDouble;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.ScmInfo;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
+import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
+import org.apache.hadoop.hdds.scm.container.ContainerMapping;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.server.SCMChillModeManager;
 import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
 import org.apache.hadoop.hdds.scm.server.SCMStorage;
-import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager.StartupOption;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
-import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
-import org.apache.hadoop.hdds.scm.server.StorageContainerManager.StartupOption;
-import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
-import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
-import org.apache.hadoop.hdds.scm.exceptions.SCMException;
-import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.ScmInfo;
-import org.junit.Rule;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
-
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.List;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.Map;
-import java.util.Collections;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-
 import org.junit.rules.Timeout;
 import org.mockito.Mockito;
-import org.apache.hadoop.test.GenericTestUtils;
-
-import static org.apache.hadoop.hdds
-    .HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
-import static org.junit.Assert.fail;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Test class that exercises the StorageContainerManager.
@@ -78,6 +82,8 @@ public class TestStorageContainerManager {
   private static XceiverClientManager xceiverClientManager =
       new XceiverClientManager(
       new OzoneConfiguration());
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestStorageContainerManager.class);
   /**
    * Set the timeout for every test.
    */
@@ -457,4 +463,92 @@ public class TestStorageContainerManager {
     Assert.assertEquals(clusterId, scmInfo.getClusterId());
     Assert.assertEquals(scmId, scmInfo.getScmId());
   }
+
+  @Test
+  public void testSCMChillMode() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    MiniOzoneCluster.Builder builder = MiniOzoneCluster.newBuilder(conf)
+        .setHbInterval(1000)
+        .setNumDatanodes(3)
+        .setStartDataNodes(false)
+        .setHbProcessorInterval(500);
+    MiniOzoneClusterImpl cluster = (MiniOzoneClusterImpl) builder.build();
+    // Test1: Test chill mode  when there are no containers in system.
+    assertTrue(cluster.getStorageContainerManager().isInChillMode());
+    cluster.startHddsDatanodes();
+    cluster.waitForClusterToBeReady();
+    assertFalse(cluster.getStorageContainerManager().isInChillMode());
+
+    // Test2: Test chill mode  when containers are there in system.
+    // Create {numKeys} random names keys.
+    TestStorageContainerManagerHelper helper =
+        new TestStorageContainerManagerHelper(cluster, conf);
+    Map<String, OmKeyInfo> keyLocations = helper.createKeys(100*2, 4096);
+    final List<ContainerInfo> containers = cluster.getStorageContainerManager()
+        .getScmContainerManager().getStateManager().getAllContainers();
+    GenericTestUtils.waitFor(() -> {
+      return containers.size() > 10;
+    }, 100, 1000);
+
+    // Removing some container to keep them open.
+    containers.remove(0);
+    containers.remove(1);
+    containers.remove(2);
+    containers.remove(3);
+
+    // Close remaining containers
+    ContainerMapping mapping = (ContainerMapping) cluster
+        .getStorageContainerManager().getScmContainerManager();
+    containers.forEach(c -> {
+      try {
+        mapping.updateContainerState(c.getContainerID(),
+            HddsProtos.LifeCycleEvent.FINALIZE);
+        mapping.updateContainerState(c.getContainerID(),
+            LifeCycleEvent.CLOSE);
+      } catch (IOException e) {
+        LOG.info("Failed to change state of open containers.", e);
+      }
+    });
+    cluster.stop();
+
+    GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
+        .captureLogs(SCMChillModeManager.getLogger());
+    logCapturer.clearOutput();
+    AtomicReference<MiniOzoneCluster> miniCluster = new AtomicReference<>();
+    new Thread(() -> {
+      try {
+        miniCluster.set(builder.setStartDataNodes(false).build());
+      } catch (IOException e) {
+        fail("failed");
+      }
+    }).start();
+
+    StorageContainerManager scm;
+    GenericTestUtils.waitFor(() -> {
+      return miniCluster.get() != null;
+    }, 100, 1000 * 3);
+
+    scm = miniCluster.get().getStorageContainerManager();
+    assertTrue(scm.isInChillMode());
+    assertFalse(logCapturer.getOutput().contains("SCM exiting chill mode."));
+    assertTrue(scm.getCurrentContainerThreshold() == 0);
+    AtomicDouble curThreshold = new AtomicDouble();
+    AtomicDouble lastReportedThreshold = new AtomicDouble();
+    for(HddsDatanodeService dn:miniCluster.get().getHddsDatanodes()){
+      dn.start(null);
+      GenericTestUtils.waitFor(() -> {
+        curThreshold.set(scm.getCurrentContainerThreshold());
+        return curThreshold.get() > lastReportedThreshold.get();
+      }, 100, 1000 * 5);
+      lastReportedThreshold.set(curThreshold.get());
+    }
+    double chillModeCutoff = conf
+        .getDouble(HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT,
+            HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT_DEFAULT);
+    assertTrue(scm.getCurrentContainerThreshold() >= chillModeCutoff);
+    assertTrue(logCapturer.getOutput().contains("SCM exiting chill mode."));
+    assertFalse(scm.isInChillMode());
+    cluster.shutdown();
+  }
+
 }


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


[2/3] hadoop git commit: HDDS-190. Improve shell error message for unrecognized option. Contributed by Elek, Marton.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java
index c73307d..bea68f2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/PutKeyHandler.java
@@ -18,86 +18,80 @@
 
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.codec.digest.DigestUtils;
+import java.io.File;
+import java.io.FileInputStream;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.OzoneVolume;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
+import org.apache.commons.codec.digest.DigestUtils;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
 
 /**
  * Puts a file into an ozone bucket.
  */
+@Command(name = "-putKey",
+    description = "creates or overwrites an existing key")
 public class PutKeyHandler extends Handler {
-  private String volumeName;
-  private String bucketName;
-  private String keyName;
 
+  @Parameters(arity = "1..1", description = Shell.OZONE_KEY_URI_DESCRIPTION)
+  private String uri;
+
+  @Option(names = {"-f", "--file", "-file"},
+      description = "File to upload",
+      required = true)
+  private String fileName;
+
+  @Option(names = {"-r", "--replication", "-replicationFactor"},
+      description = "Replication factor of the new key. (use ONE or THREE) "
+          + "Default is specified in the cluster-wide config.")
+  private ReplicationFactor replicationFactor;
   /**
    * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.PUT_KEY)) {
-      throw new OzoneClientException("Incorrect call : putKey is missing");
-    }
+  public Void call() throws Exception {
 
-    if (!cmd.hasOption(Shell.FILE)) {
-      throw new OzoneClientException("put key needs a file to put");
-    }
-
-    String ozoneURIString = cmd.getOptionValue(Shell.PUT_KEY);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 3) {
       throw new OzoneClientException(
           "volume/bucket/key name required in putKey");
     }
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
-    keyName = path.getName(2).toString();
-
+    String volumeName = path.getName(0).toString();
+    String bucketName = path.getName(1).toString();
+    String keyName = path.getName(2).toString();
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Bucket Name : %s%n", bucketName);
       System.out.printf("Key Name : %s%n", keyName);
     }
 
-    String fileName = cmd.getOptionValue(Shell.FILE);
     File dataFile = new File(fileName);
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       FileInputStream stream = new FileInputStream(dataFile);
       String hash = DigestUtils.md5Hex(stream);
       System.out.printf("File Hash : %s%n", hash);
@@ -105,11 +99,7 @@ public class PutKeyHandler extends Handler {
     }
 
     Configuration conf = new OzoneConfiguration();
-    ReplicationFactor replicationFactor;
-    if (cmd.hasOption(Shell.REPLICATION_FACTOR)) {
-      replicationFactor = ReplicationFactor.valueOf(Integer.parseInt(cmd
-          .getOptionValue(Shell.REPLICATION_FACTOR)));
-    } else {
+    if (replicationFactor == null) {
       replicationFactor = ReplicationFactor.valueOf(
           conf.getInt(OZONE_REPLICATION, OZONE_REPLICATION_DEFAULT));
     }
@@ -126,6 +116,7 @@ public class PutKeyHandler extends Handler {
         conf.getInt(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT));
     outputStream.close();
     fileInputStream.close();
+    return null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
index 0057282..25e207a 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/CreateVolumeHandler.java
@@ -18,77 +18,72 @@
 
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
-import org.apache.commons.cli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
+
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
-import java.io.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 
 /**
  * Executes the create volume call for the shell.
  */
+@Command(name = "-createVolume",
+    description = "Creates a volume for the specified user")
 public class CreateVolumeHandler extends Handler {
 
-  private String rootName;
+  @Parameters(arity = "1..1", description = Shell.OZONE_VOLUME_URI_DESCRIPTION)
+  private String uri;
+
+  @Option(names = {"--user", "-user"},
+      description = "Owner of of the volume", required =
+      true)
   private String userName;
-  private String volumeName;
+
+  @Option(names = {"--quota", "-quota"},
+      description =
+          "Quota of the newly created volume (eg. 1G)")
   private String quota;
 
+  @Option(names = {"--root", "-root"},
+      description = "Development flag to execute the "
+          + "command as the admin (hdfs) user.")
+  private boolean root;
+
   /**
    * Executes the Create Volume.
-   *
-   * @param cmd - CommandLine
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.CREATE_VOLUME)) {
-      throw new OzoneClientException(
-          "Incorrect call : createVolume is missing");
-    }
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.CREATE_VOLUME);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
 
     // we need to skip the slash in the URI path
     // getPath returns /volumeName needs to remove the initial slash.
-    volumeName = ozoneURI.getPath().replaceAll("^/+", "");
+    String volumeName = ozoneURI.getPath().replaceAll("^/+", "");
     if (volumeName.isEmpty()) {
       throw new OzoneClientException(
           "Volume name is required to create a volume");
     }
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume name : %s%n", volumeName);
     }
-    if (cmd.hasOption(Shell.RUNAS)) {
+
+    String rootName;
+    if (root) {
       rootName = "hdfs";
     } else {
       rootName = System.getProperty("user.name");
     }
 
-    if (!cmd.hasOption(Shell.USER)) {
-      throw new OzoneClientException(
-          "User name is needed in createVolume call.");
-    }
-
-    if (cmd.hasOption(Shell.QUOTA)) {
-      quota = cmd.getOptionValue(Shell.QUOTA);
-    }
-
-    userName = cmd.getOptionValue(Shell.USER);
-
     VolumeArgs.Builder volumeArgsBuilder = VolumeArgs.newBuilder()
         .setAdmin(rootName)
         .setOwner(userName);
@@ -97,11 +92,13 @@ public class CreateVolumeHandler extends Handler {
     }
     client.getObjectStore().createVolume(volumeName, volumeArgsBuilder.build());
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
       System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
           JsonUtils.toJsonString(OzoneClientUtils.asVolumeInfo(vol))));
     }
+    return null;
   }
+
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
index 2df788a..e6444e7 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/DeleteVolumeHandler.java
@@ -18,55 +18,46 @@
 
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
-import org.apache.commons.cli.CommandLine;
+import java.net.URI;
+
 import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
 
 /**
  * Executes deleteVolume call for the shell.
  */
+@Command(name = "-deleteVolume",
+    description = "deletes a volume if it is empty")
 public class DeleteVolumeHandler extends Handler {
 
-  private String volumeName;
+  @Parameters(arity = "1..1", description = Shell.OZONE_VOLUME_URI_DESCRIPTION)
+  private String uri;
 
   /**
    * Executes the delete volume call.
-   *
-   * @param cmd - CommandLine
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-
-    if (!cmd.hasOption(Shell.DELETE_VOLUME)) {
-      throw new OzoneClientException(
-          "Incorrect call : deleteVolume call is missing");
-    }
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.DELETE_VOLUME);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     if (ozoneURI.getPath().isEmpty()) {
       throw new OzoneClientException(
           "Volume name is required to delete a volume");
     }
 
     // we need to skip the slash in the URI path
-    volumeName = ozoneURI.getPath().substring(1);
+    String volumeName = ozoneURI.getPath().substring(1);
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume name : %s%n", volumeName);
     }
 
     client.getObjectStore().deleteVolume(volumeName);
     System.out.printf("Volume %s is deleted%n", volumeName);
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
index b5be2c6..9ee3dce 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/InfoVolumeHandler.java
@@ -18,56 +18,47 @@
 
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
-import org.apache.commons.cli.CommandLine;
+import java.net.URI;
+
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
 
 /**
  * Executes volume Info calls.
  */
+@Command(name = "-infoVolume",
+    description = "returns information about a specific volume")
 public class InfoVolumeHandler extends Handler{
 
-  private String volumeName;
+  @Parameters(arity = "1..1", description = Shell.OZONE_VOLUME_URI_DESCRIPTION)
+  private String uri;
 
   /**
    * Executes volume Info.
-   *
-   * @param cmd - CommandLine
-   *
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
+  public Void call() throws Exception {
 
-    if (!cmd.hasOption(Shell.INFO_VOLUME)) {
-      throw new OzoneClientException(
-          "Incorrect call : infoVolume is missing");
-    }
-
-    String ozoneURIString = cmd.getOptionValue(Shell.INFO_VOLUME);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     if (ozoneURI.getPath().isEmpty()) {
       throw new OzoneClientException(
           "Volume name is required to get info of a volume");
     }
 
     // we need to skip the slash in the URI path
-    volumeName = ozoneURI.getPath().substring(1);
+    String volumeName = ozoneURI.getPath().substring(1);
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
         JsonUtils.toJsonString(OzoneClientUtils.asVolumeInfo(vol))));
+    return null;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
index 85b7b2b..058ef2e 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
@@ -19,20 +19,19 @@
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
 import com.google.common.base.Strings;
-import org.apache.commons.cli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
+
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
 import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 
-import java.io.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -40,49 +39,39 @@ import java.util.List;
 /**
  * Executes List Volume call.
  */
+@Command(name = "-listVolume",
+    description = "List the volumes of a given user")
 public class ListVolumeHandler extends Handler {
-  private String userName;
 
-  /**
-   * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
-   */
-  @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
+  @Parameters(arity = "1..1",
+      description = Shell.OZONE_VOLUME_URI_DESCRIPTION,
+      defaultValue = "/")
+  private String uri;
 
-    if (!cmd.hasOption(Shell.LIST_VOLUME)) {
-      throw new OzoneClientException(
-          "Incorrect call : listVolume is missing");
-    }
+  @Option(names = {"--length", "-length", "-l"},
+      description = "Limit of the max results",
+      defaultValue = "100")
+  private int maxVolumes;
 
-    int maxVolumes = Integer.MAX_VALUE;
-    if (cmd.hasOption(Shell.LIST_LENGTH)) {
-      String length = cmd.getOptionValue(Shell.LIST_LENGTH);
-      OzoneUtils.verifyMaxKeyLength(length);
+  @Option(names = {"--start", "-start", "-s"},
+      description = "The first volume to start the listing")
+  private String startVolume;
 
-      maxVolumes = Integer.parseInt(length);
-    }
+  @Option(names = {"--prefix", "-prefix", "-p"},
+      description = "Prefix to filter the volumes")
+  private String prefix;
 
-    String startVolume = null;
-    if (cmd.hasOption(Shell.START)) {
-      startVolume = cmd.getOptionValue(Shell.START);
-    }
+  @Option(names = {"--user", "-user", "-u"},
+      description = "Owner of the volumes to list.")
+  private String userName;
 
-    String prefix = null;
-    if (cmd.hasOption(Shell.PREFIX)) {
-      prefix = cmd.getOptionValue(Shell.PREFIX);
-    }
+  /**
+   * Executes the Client Calls.
+   */
+  @Override
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.LIST_VOLUME);
-    if (Strings.isNullOrEmpty(ozoneURIString)) {
-      ozoneURIString = "/";
-    }
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     if (!Strings.isNullOrEmpty(ozoneURI.getPath()) && !ozoneURI.getPath()
         .equals("/")) {
       throw new OzoneClientException(
@@ -90,12 +79,15 @@ public class ListVolumeHandler extends Handler {
               .getPath());
     }
 
-    if (cmd.hasOption(Shell.USER)) {
-      userName = cmd.getOptionValue(Shell.USER);
-    } else {
+    if (userName == null) {
       userName = System.getProperty("user.name");
     }
 
+    if (maxVolumes < 1) {
+      throw new IllegalArgumentException(
+          "the length should be a positive number");
+    }
+
     Iterator<OzoneVolume> volumeIterator;
     if(userName != null) {
       volumeIterator = client.getObjectStore()
@@ -112,12 +104,13 @@ public class ListVolumeHandler extends Handler {
       maxVolumes -= 1;
     }
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Found : %d volumes for user : %s ", volumeInfos.size(),
           userName);
     }
     System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
         JsonUtils.toJsonString(volumeInfos)));
+    return null;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
index 3738cb4..aa692bd 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/UpdateVolumeHandler.java
@@ -18,61 +18,53 @@
 
 package org.apache.hadoop.ozone.web.ozShell.volume;
 
-import org.apache.commons.cli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
+
 import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
-import java.io.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 
 /**
  * Executes update volume calls.
  */
+@Command(name = "-updateVolume",
+    description = "Updates parameter of the volumes")
 public class UpdateVolumeHandler extends Handler {
+
+  @Parameters(arity = "1..1", description = Shell.OZONE_VOLUME_URI_DESCRIPTION)
+  private String uri;
+
+  @Option(names = {"--user", "-user"},
+      description = "Owner of the volume to set")
   private String ownerName;
-  private String volumeName;
+
+  @Option(names = {"--quota", "-quota"},
+      description = "Quota of the volume to set"
+          + "(eg. 1G)")
   private String quota;
 
   /**
-   * Executes update volume calls.
-   *
-   * @param cmd - CommandLine
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
+   * Executes the Client Calls.
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.UPDATE_VOLUME)) {
-      throw new OzoneClientException(
-          "Incorrect call : updateVolume is missing");
-    }
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.UPDATE_VOLUME);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     if (ozoneURI.getPath().isEmpty()) {
       throw new OzoneClientException(
           "Volume name is required to update a volume");
     }
 
     // we need to skip the slash in the URI path
-    volumeName = ozoneURI.getPath().substring(1);
-
-    if (cmd.hasOption(Shell.QUOTA)) {
-      quota = cmd.getOptionValue(Shell.QUOTA);
-    }
-
-    if (cmd.hasOption(Shell.USER)) {
-      ownerName = cmd.getOptionValue(Shell.USER);
-    }
+    String volumeName = ozoneURI.getPath().substring(1);
 
     OzoneVolume volume = client.getObjectStore().getVolume(volumeName);
     if (quota != null && !quota.isEmpty()) {
@@ -85,5 +77,6 @@ public class UpdateVolumeHandler extends Handler {
 
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
         JsonUtils.toJsonString(OzoneClientUtils.asVolumeInfo(volume))));
+    return null;
   }
 }


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


[3/3] hadoop git commit: HDDS-190. Improve shell error message for unrecognized option. Contributed by Elek, Marton.

Posted by ae...@apache.org.
HDDS-190. Improve shell error message for unrecognized option.
Contributed by Elek, Marton.

(cherry picked from commit 73fcbdd296fb0f6e7cde17ef0bd6f3b981878077)


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

Branch: refs/heads/ozone-0.2
Commit: fd792ce54d53282669696fd90c60689d06ac44f6
Parents: 48bcebc
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Sep 7 12:53:46 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Sep 7 13:02:49 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/cli/GenericCli.java  |  17 +-
 .../hadoop/ozone/ozShell/TestOzoneShell.java    | 246 ++++++-----
 .../hadoop/ozone/web/ozShell/Handler.java       |  27 +-
 .../apache/hadoop/ozone/web/ozShell/Shell.java  | 407 +++----------------
 .../web/ozShell/bucket/CreateBucketHandler.java |  46 +--
 .../web/ozShell/bucket/DeleteBucketHandler.java |  45 +-
 .../web/ozShell/bucket/InfoBucketHandler.java   |  42 +-
 .../web/ozShell/bucket/ListBucketHandler.java   |  88 ++--
 .../web/ozShell/bucket/UpdateBucketHandler.java |  66 +--
 .../web/ozShell/keys/DeleteKeyHandler.java      |  48 +--
 .../ozone/web/ozShell/keys/GetKeyHandler.java   |  78 ++--
 .../ozone/web/ozShell/keys/InfoKeyHandler.java  |  45 +-
 .../ozone/web/ozShell/keys/ListKeyHandler.java  |  91 ++---
 .../ozone/web/ozShell/keys/PutKeyHandler.java   |  81 ++--
 .../web/ozShell/volume/CreateVolumeHandler.java |  65 ++-
 .../web/ozShell/volume/DeleteVolumeHandler.java |  35 +-
 .../web/ozShell/volume/InfoVolumeHandler.java   |  37 +-
 .../web/ozShell/volume/ListVolumeHandler.java   |  81 ++--
 .../web/ozShell/volume/UpdateVolumeHandler.java |  49 +--
 19 files changed, 617 insertions(+), 977 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
index 2b3e6c0..f829d82 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
@@ -23,6 +23,7 @@ import java.util.concurrent.Callable;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 
+import com.google.common.annotations.VisibleForTesting;
 import picocli.CommandLine;
 import picocli.CommandLine.ExecutionException;
 import picocli.CommandLine.Option;
@@ -49,13 +50,18 @@ public class GenericCli implements Callable<Void> {
 
   public void run(String[] argv) {
     try {
-      cmd.parseWithHandler(new RunLast(), argv);
+      execute(argv);
     } catch (ExecutionException ex) {
       printError(ex.getCause());
       System.exit(-1);
     }
   }
 
+  @VisibleForTesting
+  public void execute(String[] argv) {
+    cmd.parseWithHandler(new RunLast(), argv);
+  }
+
   private void printError(Throwable error) {
     if (verbose) {
       error.printStackTrace(System.err);
@@ -79,4 +85,13 @@ public class GenericCli implements Callable<Void> {
     }
     return ozoneConf;
   }
+
+  public boolean isVerbose() {
+    return verbose;
+  }
+
+  @VisibleForTesting
+  public picocli.CommandLine getCmd() {
+    return cmd;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
index 386b1d2..f50de4b 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
@@ -17,12 +17,6 @@
  */
 package org.apache.hadoop.ozone.ozShell;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
@@ -38,17 +32,15 @@ import java.util.Random;
 import java.util.UUID;
 import java.util.stream.Collectors;
 
-import com.google.common.base.Strings;
-import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneAcl.OzoneACLRights;
 import org.apache.hadoop.ozone.OzoneAcl.OzoneACLType;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKey;
@@ -67,10 +59,17 @@ import org.apache.hadoop.ozone.web.response.KeyInfo;
 import org.apache.hadoop.ozone.web.response.VolumeInfo;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.ToolRunner;
+
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.RandomStringUtils;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -80,6 +79,12 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+import picocli.CommandLine.ExecutionException;
+import picocli.CommandLine.IExceptionHandler2;
+import picocli.CommandLine.ParameterException;
+import picocli.CommandLine.ParseResult;
+import picocli.CommandLine.RunLast;
 
 /**
  * This test class specified for testing Ozone shell command.
@@ -134,7 +139,6 @@ public class TestOzoneShell {
     baseDir.mkdirs();
 
     shell = new Shell();
-    shell.setConf(conf);
 
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
@@ -206,7 +210,7 @@ public class TestOzoneShell {
     testCreateVolume("/////", "Volume name is required " +
         "to create a volume");
     testCreateVolume("/////vol/123",
-        "Illegal argument: Bucket or Volume name has " +
+        "Bucket or Volume name has " +
             "an unsupported character : /");
   }
 
@@ -218,10 +222,10 @@ public class TestOzoneShell {
         "-user", userName, "-root"};
 
     if (Strings.isNullOrEmpty(errorMsg)) {
-      assertEquals(0, ToolRunner.run(shell, args));
+      execute(shell, args);
+
     } else {
-      assertEquals(1, ToolRunner.run(shell, args));
-      assertTrue(err.toString().contains(errorMsg));
+      executeWithError(shell, args, errorMsg);
       return;
     }
 
@@ -232,6 +236,29 @@ public class TestOzoneShell {
     assertEquals(userName, volumeInfo.getOwner());
   }
 
+  private void execute(Shell shell, String[] args) {
+    List<String> arguments = new ArrayList(Arrays.asList(args));
+    LOG.info("Executing shell command with args {}", arguments);
+    CommandLine cmd = shell.getCmd();
+
+    IExceptionHandler2<List<Object>> exceptionHandler =
+        new IExceptionHandler2<List<Object>>() {
+          @Override
+          public List<Object> handleParseException(ParameterException ex,
+              String[] args) {
+            throw ex;
+          }
+
+          @Override
+          public List<Object> handleExecutionException(ExecutionException ex,
+              ParseResult parseResult) {
+            throw ex;
+          }
+        };
+    cmd.parseWithHandlers(new RunLast(),
+        exceptionHandler, args);
+  }
+
   @Test
   public void testDeleteVolume() throws Exception {
     LOG.info("Running testDeleteVolume");
@@ -244,9 +271,8 @@ public class TestOzoneShell {
     OzoneVolume volume = client.getVolumeDetails(volumeName);
     assertNotNull(volume);
 
-    String[] args = new String[] {"-deleteVolume", url + "/" + volumeName,
-        "-root"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    String[] args = new String[] {"-deleteVolume", url + "/" + volumeName};
+    execute(shell, args);
     String output = out.toString();
     assertTrue(output.contains("Volume " + volumeName + " is deleted"));
 
@@ -270,9 +296,8 @@ public class TestOzoneShell {
         .build();
     client.createVolume(volumeName, volumeArgs);
 
-    String[] args = new String[] {"-infoVolume", url + "/" + volumeName,
-        "-root"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    String[] args = new String[] {"-infoVolume", url + "/" + volumeName};
+    execute(shell, args);
 
     String output = out.toString();
     assertTrue(output.contains(volumeName));
@@ -280,10 +305,8 @@ public class TestOzoneShell {
         && output.contains(OzoneConsts.OZONE_TIME_ZONE));
 
     // get info for non-exist volume
-    args = new String[] {"-infoVolume", url + "/invalid-volume", "-root"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Volume failed, error:VOLUME_NOT_FOUND"));
+    args = new String[] {"-infoVolume", url + "/invalid-volume"};
+    executeWithError(shell, args, "VOLUME_NOT_FOUND");
   }
 
   @Test
@@ -301,32 +324,58 @@ public class TestOzoneShell {
     assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(), vol.getQuota());
 
     String[] args = new String[] {"-updateVolume", url + "/" + volumeName,
-        "-quota", "500MB", "-root"};
-    assertEquals(0, ToolRunner.run(shell, args));
+        "-quota", "500MB"};
+    execute(shell, args);
     vol = client.getVolumeDetails(volumeName);
     assertEquals(userName, vol.getOwner());
     assertEquals(OzoneQuota.parseQuota("500MB").sizeInBytes(), vol.getQuota());
 
     String newUser = "new-user";
     args = new String[] {"-updateVolume", url + "/" + volumeName,
-        "-user", newUser, "-root"};
-    assertEquals(0, ToolRunner.run(shell, args));
+        "-user", newUser};
+    execute(shell, args);
     vol = client.getVolumeDetails(volumeName);
     assertEquals(newUser, vol.getOwner());
 
     // test error conditions
     args = new String[] {"-updateVolume", url + "/invalid-volume",
-        "-user", newUser, "-root"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Volume failed, error:VOLUME_NOT_FOUND"));
+        "-user", newUser};
+    executeWithError(shell, args, "Info Volume failed, error:VOLUME_NOT_FOUND");
 
     err.reset();
     args = new String[] {"-updateVolume", url + "/invalid-volume",
-        "-quota", "500MB", "-root"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Volume failed, error:VOLUME_NOT_FOUND"));
+        "-quota", "500MB"};
+    executeWithError(shell, args, "Info Volume failed, error:VOLUME_NOT_FOUND");
+  }
+
+  /**
+   * Execute command, assert exeception message and returns true if error
+   * was thrown.
+   */
+  private void executeWithError(Shell shell, String[] args,
+      String expectedError) {
+    if (Strings.isNullOrEmpty(expectedError)) {
+      execute(shell, args);
+    } else {
+      try {
+        execute(shell, args);
+        fail("Exception is expected from command execution " + Arrays
+            .asList(args));
+      } catch (Exception ex) {
+        if (!Strings.isNullOrEmpty(expectedError)) {
+          Throwable exceptionToCheck = ex;
+          if (exceptionToCheck.getCause() != null) {
+            exceptionToCheck = exceptionToCheck.getCause();
+          }
+          Assert.assertTrue(
+              String.format(
+                  "Error of shell code doesn't contain the " +
+                      "exception [%s] in [%s]",
+                  expectedError, exceptionToCheck.getMessage()),
+              exceptionToCheck.getMessage().contains(expectedError));
+        }
+      }
+    }
   }
 
   @Test
@@ -364,15 +413,13 @@ public class TestOzoneShell {
 
     String[] args = new String[] {"-listVolume", url + "/abcde", "-user",
         user1, "-length", "100"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    commandError = err.toString();
-    Assert.assertTrue(commandError.contains("Invalid URI:"));
+    executeWithError(shell, args, "Invalid URI");
 
     err.reset();
     // test -length option
     args = new String[] {"-listVolume", url + "/", "-user",
         user1, "-length", "100"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     volumes = (List<VolumeInfo>) JsonUtils
         .toJsonList(commandOutput, VolumeInfo.class);
@@ -386,7 +433,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listVolume", url + "/", "-user",
         user1, "-length", "2"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     volumes = (List<VolumeInfo>) JsonUtils
         .toJsonList(commandOutput, VolumeInfo.class);
@@ -397,7 +444,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listVolume", url + "/", "-user", user1, "-length",
         "100", "-prefix", "test-vol-" + protocol + "1" };
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     volumes = (List<VolumeInfo>) JsonUtils
         .toJsonList(commandOutput, VolumeInfo.class);
@@ -414,7 +461,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listVolume", url + "/", "-user", user2, "-length",
         "100", "-start", "test-vol-" + protocol + "15" };
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     volumes = (List<VolumeInfo>) JsonUtils
         .toJsonList(commandOutput, VolumeInfo.class);
@@ -430,16 +477,12 @@ public class TestOzoneShell {
     err.reset();
     args  = new String[] {"-listVolume", url + "/", "-user",
         user2, "-length", "-1"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "the vaule should be a positive number"));
+    executeWithError(shell, args, "the length should be a positive number");
 
     err.reset();
     args  = new String[] {"-listVolume", url + "/", "-user",
         user2, "-length", "invalid-length"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "the vaule should be digital"));
+    executeWithError(shell, args, "For input string: \"invalid-length\"");
   }
 
   @Test
@@ -450,7 +493,7 @@ public class TestOzoneShell {
     String[] args = new String[] {"-createBucket",
         url + "/" + vol.getName() + "/" + bucketName};
 
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     OzoneBucket bucketInfo = vol.getBucket(bucketName);
     assertEquals(vol.getName(),
         bucketInfo.getVolumeName());
@@ -460,9 +503,7 @@ public class TestOzoneShell {
     args = new String[] {"-createBucket",
         url + "/invalid-volume/" + bucketName};
 
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Volume failed, error:VOLUME_NOT_FOUND"));
+    executeWithError(shell, args, "Info Volume failed, error:VOLUME_NOT_FOUND");
   }
 
   @Test
@@ -476,7 +517,7 @@ public class TestOzoneShell {
 
     String[] args = new String[] {"-deleteBucket",
         url + "/" + vol.getName() + "/" + bucketName};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
 
     // verify if bucket has been deleted in volume
     try {
@@ -490,17 +531,14 @@ public class TestOzoneShell {
     // test delete bucket in a non-exist volume
     args = new String[] {"-deleteBucket",
         url + "/invalid-volume" + "/" + bucketName};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Volume failed, error:VOLUME_NOT_FOUND"));
+    executeWithError(shell, args, "Info Volume failed, error:VOLUME_NOT_FOUND");
 
     err.reset();
     // test delete non-exist bucket
     args = new String[] {"-deleteBucket",
         url + "/" + vol.getName() + "/invalid-bucket"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Delete Bucket failed, error:BUCKET_NOT_FOUND"));
+    executeWithError(shell, args,
+        "Delete Bucket failed, error:BUCKET_NOT_FOUND");
   }
 
   @Test
@@ -512,7 +550,7 @@ public class TestOzoneShell {
 
     String[] args = new String[] {"-infoBucket",
         url + "/" + vol.getName() + "/" + bucketName};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
 
     String output = out.toString();
     assertTrue(output.contains(bucketName));
@@ -522,9 +560,8 @@ public class TestOzoneShell {
     // test get info from a non-exist bucket
     args = new String[] {"-infoBucket",
         url + "/" + vol.getName() + "/invalid-bucket" + bucketName};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Bucket failed, error: BUCKET_NOT_FOUND"));
+    executeWithError(shell, args,
+        "Info Bucket failed, error: BUCKET_NOT_FOUND");
   }
 
   @Test
@@ -539,7 +576,7 @@ public class TestOzoneShell {
     String[] args = new String[] {"-updateBucket",
         url + "/" + vol.getName() + "/" + bucketName, "-addAcl",
         "user:frodo:rw,group:samwise:r"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     String output = out.toString();
     assertTrue(output.contains("createdOn")
         && output.contains(OzoneConsts.OZONE_TIME_ZONE));
@@ -555,7 +592,7 @@ public class TestOzoneShell {
     args = new String[] {"-updateBucket",
         url + "/" + vol.getName() + "/" + bucketName, "-removeAcl",
         "user:frodo:rw"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
 
     bucket = vol.getBucket(bucketName);
     acl = bucket.getAcls().get(aclSize);
@@ -568,9 +605,8 @@ public class TestOzoneShell {
     args = new String[] {"-updateBucket",
         url + "/" + vol.getName() + "/invalid-bucket", "-addAcl",
         "user:frodo:rw"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Bucket failed, error: BUCKET_NOT_FOUND"));
+    executeWithError(shell, args,
+        "Info Bucket failed, error: BUCKET_NOT_FOUND");
   }
 
   @Test
@@ -594,7 +630,7 @@ public class TestOzoneShell {
     // test -length option
     String[] args = new String[] {"-listBucket",
         url + "/" + vol.getName(), "-length", "100"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     buckets = (List<BucketInfo>) JsonUtils.toJsonList(commandOutput,
         BucketInfo.class);
@@ -614,7 +650,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listBucket", url + "/" + vol.getName(),
         "-length", "3"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     buckets = (List<BucketInfo>) JsonUtils.toJsonList(commandOutput,
         BucketInfo.class);
@@ -630,7 +666,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listBucket", url + "/" + vol.getName(),
         "-length", "100", "-prefix", "test-bucket1"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     buckets = (List<BucketInfo>) JsonUtils.toJsonList(commandOutput,
         BucketInfo.class);
@@ -644,7 +680,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listBucket", url + "/" + vol.getName(),
         "-length", "100", "-start", "test-bucket7"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     buckets = (List<BucketInfo>) JsonUtils.toJsonList(commandOutput,
         BucketInfo.class);
@@ -657,9 +693,7 @@ public class TestOzoneShell {
     err.reset();
     args = new String[] {"-listBucket", url + "/" + vol.getName(),
         "-length", "-1"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "the vaule should be a positive number"));
+    executeWithError(shell, args, "the length should be a positive number");
   }
 
   @Test
@@ -673,7 +707,7 @@ public class TestOzoneShell {
     String[] args = new String[] {"-putKey",
         url + "/" + volumeName + "/" + bucketName + "/" + keyName, "-file",
         createTmpFile()};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
 
     OzoneKey keyInfo = bucket.getKey(keyName);
     assertEquals(keyName, keyInfo.getName());
@@ -682,9 +716,8 @@ public class TestOzoneShell {
     args = new String[] {"-putKey",
         url + "/" + volumeName + "/invalid-bucket/" + keyName, "-file",
         createTmpFile()};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Bucket failed, error: BUCKET_NOT_FOUND"));
+    executeWithError(shell, args,
+        "Info Bucket failed, error: BUCKET_NOT_FOUND");
   }
 
   @Test
@@ -706,7 +739,7 @@ public class TestOzoneShell {
     String[] args = new String[] {"-getKey",
         url + "/" + volumeName + "/" + bucketName + "/" + keyName, "-file",
         tmpPath};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
 
     byte[] dataBytes = new byte[dataStr.length()];
     try (FileInputStream randFile = new FileInputStream(new File(tmpPath))) {
@@ -718,7 +751,7 @@ public class TestOzoneShell {
     args = new String[] {"-getKey",
         url + "/" + volumeName + "/" + bucketName + "/" + keyName, "-file",
         baseDir.getAbsolutePath()};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
 
     dataBytes = new byte[dataStr.length()];
     try (FileInputStream randFile = new FileInputStream(new File(tmpPath))) {
@@ -745,7 +778,7 @@ public class TestOzoneShell {
 
     String[] args = new String[] {"-deleteKey",
         url + "/" + volumeName + "/" + bucketName + "/" + keyName};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
 
     // verify if key has been deleted in the bucket
     try {
@@ -759,17 +792,14 @@ public class TestOzoneShell {
     // test delete key in a non-exist bucket
     args = new String[] {"-deleteKey",
         url + "/" + volumeName + "/invalid-bucket/" + keyName};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Info Bucket failed, error: BUCKET_NOT_FOUND"));
+    executeWithError(shell, args,
+        "Info Bucket failed, error: BUCKET_NOT_FOUND");
 
     err.reset();
     // test delete a non-exist key in bucket
     args = new String[] {"-deleteKey",
         url + "/" + volumeName + "/" + bucketName + "/invalid-key"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Delete key failed, error:KEY_NOT_FOUND"));
+    executeWithError(shell, args, "Delete key failed, error:KEY_NOT_FOUND");
   }
 
   @Test
@@ -789,9 +819,8 @@ public class TestOzoneShell {
         url + "/" + volumeName + "/" + bucketName + "/" + keyName};
 
     // verify the response output
-    int a = ToolRunner.run(shell, args);
+    execute(shell, args);
     String output = out.toString();
-    assertEquals(0, a);
 
     assertTrue(output.contains(keyName));
     assertTrue(
@@ -810,9 +839,7 @@ public class TestOzoneShell {
 
     // verify the response output
     // get the non-exist key info should be failed
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "Lookup key failed, error:KEY_NOT_FOUND"));
+    executeWithError(shell, args, "Lookup key failed, error:KEY_NOT_FOUND");
   }
 
   @Test
@@ -831,21 +858,15 @@ public class TestOzoneShell {
     String[] args = new String[] {"-infoKey",
         url + "/" + volumeName + "/" + bucketName + "/" + dirKeyName};
     // verify the response output
-    int a = ToolRunner.run(shell, args);
+    execute(shell, args);
     String output = out.toString();
-    assertEquals(0, a);
     assertTrue(output.contains(dirKeyName));
     assertTrue(output.contains("createdOn") &&
                 output.contains("modifiedOn") &&
                 output.contains(OzoneConsts.OZONE_TIME_ZONE));
     args = new String[] {"-infoKey",
         url + "/" + volumeName + "/" + bucketName + "/" + keyNameOnly};
-    a = ToolRunner.run(shell, args);
-    output = out.toString();
-    assertEquals(1, a);
-    assertTrue(err.toString().contains(
-        "Lookup key failed, error:KEY_NOT_FOUND"));
-    // reset stream
+    executeWithError(shell, args, "Lookup key failed, error:KEY_NOT_FOUND");
     out.reset();
     err.reset();
   }
@@ -875,7 +896,7 @@ public class TestOzoneShell {
     // test -length option
     String[] args = new String[] {"-listKey",
         url + "/" + volumeName + "/" + bucketName, "-length", "100"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     keys = (List<KeyInfo>) JsonUtils.toJsonList(commandOutput,
         KeyInfo.class);
@@ -897,7 +918,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listKey", url + "/" + volumeName + "/" + bucketName,
         "-length", "3"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     keys = (List<KeyInfo>) JsonUtils.toJsonList(commandOutput,
         KeyInfo.class);
@@ -912,7 +933,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listKey", url + "/" + volumeName + "/" + bucketName,
         "-length", "100", "-prefix", "test-key1"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     keys = (List<KeyInfo>) JsonUtils.toJsonList(commandOutput,
         KeyInfo.class);
@@ -926,7 +947,7 @@ public class TestOzoneShell {
     out.reset();
     args = new String[] {"-listKey", url + "/" + volumeName + "/" + bucketName,
         "-length", "100", "-start", "test-key7"};
-    assertEquals(0, ToolRunner.run(shell, args));
+    execute(shell, args);
     commandOutput = out.toString();
     keys = (List<KeyInfo>) JsonUtils.toJsonList(commandOutput,
         KeyInfo.class);
@@ -938,9 +959,7 @@ public class TestOzoneShell {
     err.reset();
     args = new String[] {"-listKey", url + "/" + volumeName + "/" + bucketName,
         "-length", "-1"};
-    assertEquals(1, ToolRunner.run(shell, args));
-    assertTrue(err.toString().contains(
-        "the vaule should be a positive number"));
+    executeWithError(shell, args, "the length should be a positive number");
   }
 
   private OzoneVolume creatVolume() throws OzoneException, IOException {
@@ -949,7 +968,12 @@ public class TestOzoneShell {
         .setOwner("bilbo")
         .setQuota("100TB")
         .build();
-    client.createVolume(volumeName, volumeArgs);
+    try {
+      client.createVolume(volumeName, volumeArgs);
+    } catch (Exception ex) {
+      Assert.assertEquals("PartialGroupNameException",
+          ex.getCause().getClass().getSimpleName());
+    }
     OzoneVolume volume = client.getVolumeDetails(volumeName);
 
     return volume;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
index a66e227..45413f8 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Handler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.web.ozShell;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
@@ -32,18 +33,27 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.concurrent.Callable;
 
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_HTTP_SCHEME;
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_SCHEME;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.ParentCommand;
 
 /**
  * Common interface for command handling.
  */
-public abstract class Handler {
+@Command(mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public abstract class Handler implements Callable<Void> {
 
   protected static final Logger LOG = LoggerFactory.getLogger(Handler.class);
+
   protected OzoneClient client;
 
+  @ParentCommand
+  private Shell parent;
+
   /**
    * Executes the Client command.
    *
@@ -52,8 +62,15 @@ public abstract class Handler {
    * @throws OzoneException
    * @throws URISyntaxException
    */
-  protected abstract void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException;
+  protected void execute(CommandLine cmd)
+      throws IOException, OzoneException, URISyntaxException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Void call() throws Exception {
+    throw new UnsupportedOperationException();
+  }
 
   /**
    * verifies user provided URI.
@@ -148,4 +165,8 @@ public abstract class Handler {
       throw new IllegalArgumentException(e);
     }
   }
+
+  public boolean isVerbose() {
+    return parent.isVerbose();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
index 41eef1a..27cf732 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
@@ -18,16 +18,12 @@
 
 package org.apache.hadoop.ozone.web.ozShell;
 
-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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.hdds.cli.GenericCli;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.ozone.web.ozShell.bucket.CreateBucketHandler;
+import org.apache.hadoop.ozone.web.ozShell.bucket.DeleteBucketHandler;
+import org.apache.hadoop.ozone.web.ozShell.bucket.InfoBucketHandler;
+import org.apache.hadoop.ozone.web.ozShell.bucket.ListBucketHandler;
 import org.apache.hadoop.ozone.web.ozShell.bucket.UpdateBucketHandler;
 import org.apache.hadoop.ozone.web.ozShell.keys.DeleteKeyHandler;
 import org.apache.hadoop.ozone.web.ozShell.keys.GetKeyHandler;
@@ -39,17 +35,10 @@ import org.apache.hadoop.ozone.web.ozShell.volume.DeleteVolumeHandler;
 import org.apache.hadoop.ozone.web.ozShell.volume.InfoVolumeHandler;
 import org.apache.hadoop.ozone.web.ozShell.volume.ListVolumeHandler;
 import org.apache.hadoop.ozone.web.ozShell.volume.UpdateVolumeHandler;
-import org.apache.hadoop.ozone.web.ozShell.bucket.CreateBucketHandler;
-import org.apache.hadoop.ozone.web.ozShell.bucket.DeleteBucketHandler;
-import org.apache.hadoop.ozone.web.ozShell.bucket.InfoBucketHandler;
-import org.apache.hadoop.ozone.web.ozShell.bucket.ListBucketHandler;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URISyntaxException;
+import picocli.CommandLine.Command;
 
 /**
  * Ozone user interface commands.
@@ -57,47 +46,50 @@ import java.net.URISyntaxException;
  * This class uses dispatch method to make calls
  * to appropriate handlers that execute the ozone functions.
  */
-public class Shell extends Configured implements Tool {
+@Command(name = "ozone oz",
+    description = "Client for the Ozone object store",
+    subcommands = {
+        InfoVolumeHandler.class,
+        ListVolumeHandler.class,
+        CreateVolumeHandler.class,
+        UpdateVolumeHandler.class,
+        DeleteVolumeHandler.class,
+        InfoBucketHandler.class,
+        ListBucketHandler.class,
+        CreateBucketHandler.class,
+        UpdateBucketHandler.class,
+        DeleteBucketHandler.class,
+        InfoKeyHandler.class,
+        ListKeyHandler.class,
+        PutKeyHandler.class,
+        GetKeyHandler.class,
+        DeleteKeyHandler.class
+    },
+    versionProvider = HddsVersionProvider.class,
+    mixinStandardHelpOptions = true)
+public class Shell extends GenericCli {
+
+
   private static final Logger LOG = LoggerFactory.getLogger(Shell.class);
 
-  // General options
-  public static final int DEFAULT_OZONE_PORT = 50070;
-  public static final String VERBOSE = "v";
+  public static final String OZONE_URI_DESCRIPTION = "Ozone URI could start "
+      + "with o3:// or http(s):// or without prefix. REST protocol will "
+      + "be used for http(s), RPC otherwise. URI may contain the host and port "
+      + "of the SCM server. Both are optional. "
+      + "If they are not specified it will be identified from "
+      + "the config files.";
 
-  // volume related command line arguments
-  public static final String RUNAS = "root";
-  public static final String USER = "user";
-  public static final String OWNER = "owner";
-  public static final String QUOTA = "quota";
-  public static final String CREATE_VOLUME = "createVolume";
-  public static final String UPDATE_VOLUME = "updateVolume";
-  public static final String DELETE_VOLUME = "deleteVolume";
-  public static final String LIST_VOLUME = "listVolume";
-  public static final String INFO_VOLUME = "infoVolume";
+  public static final String OZONE_VOLUME_URI_DESCRIPTION =
+      "URI of the volume.\n" + OZONE_URI_DESCRIPTION;
 
-  // bucket related command line arguments
-  public static final String CREATE_BUCKET = "createBucket";
-  public static final String UPDATE_BUCKET = "updateBucket";
-  public static final String DELETE_BUCKET = "deleteBucket";
-  public static final String LIST_BUCKET = "listBucket";
-  public static final String INFO_BUCKET = "infoBucket";
-  public static final String ADD_ACLS = "addAcl";
-  public static final String REMOVE_ACLS = "removeAcl";
-  // TODO : Support versioning and StorageType for buckets
+  public static final String OZONE_BUCKET_URI_DESCRIPTION =
+      "URI of the volume/bucket.\n" + OZONE_URI_DESCRIPTION;
 
-  //Object related command line arguments
-  public static final String PUT_KEY = "putKey";
-  public static final String GET_KEY = "getKey";
-  public static final String INFO_KEY = "infoKey";
-  public static final String DELETE_KEY = "deleteKey";
-  public static final String LIST_KEY = "listKey";
-  public static final String FILE = "file";
-  public static final String REPLICATION_FACTOR = "replicationFactor";
+  public static final String OZONE_KEY_URI_DESCRIPTION =
+      "URI of the volume/bucket/key.\n" + OZONE_URI_DESCRIPTION;
 
-  // Listing related command line arguments
-  public static final String LIST_LENGTH = "length";
-  public static final String START = "start";
-  public static final String PREFIX = "prefix";
+  // General options
+  public static final int DEFAULT_OZONE_PORT = 50070;
 
   /**
    * Main for the ozShell Command handling.
@@ -106,315 +98,10 @@ public class Shell extends Configured implements Tool {
    * @throws Exception
    */
   public static void main(String[] argv) throws Exception {
-    Shell shell = new Shell();
-    Configuration conf = new OzoneConfiguration();
-    conf.setQuietMode(false);
-    shell.setConf(conf);
-    int res = 0;
-    try {
-      res = ToolRunner.run(shell, argv);
-    } catch (Exception ex) {
-      System.err.println("ERROR: " + ex.getMessage());
-      System.exit(1);
-    }
-    System.exit(res);
-  }
-
-  /**
-   * Execute the command with the given arguments.
-   *
-   * @param args command specific arguments.
-   *
-   * @return exit code.
-   *
-   * @throws Exception
-   */
-  @Override
-  public int run(String[] args) throws Exception {
-    Options opts = getOpts();
-    CommandLine cmd = parseArgs(args, opts);
-    return dispatch(cmd, opts);
-  }
-
-  /**
-   * returns the Command Line Options.
-   *
-   * @return Options
-   */
-  private Options getOpts() {
-    Options opts = new Options();
-    addVolumeCommands(opts);
-    addBucketCommands(opts);
-    addKeyCommands(opts);
-    addListingCommands(opts);
-    return opts;
-  }
-
-  /**
-   * This function parses all command line arguments
-   * and returns the appropriate values.
-   *
-   * @param argv - Argv from main
-   *
-   * @return CommandLine
-   */
-  private CommandLine parseArgs(String[] argv, Options opts)
-      throws org.apache.commons.cli.ParseException {
-    try {
-      BasicParser parser = new BasicParser();
-      return parser.parse(opts, argv);
-    } catch (ParseException ex) {
-      System.out.printf(ex.getMessage());
-    }
-
-    return null;
-  }
-
-
-  /**
-   * All volume related commands are added in this function for the command
-   * parser.
-   *
-   * @param options - Command Options class.
-   */
-  private void addVolumeCommands(Options options) {
-    Option verbose = new Option(VERBOSE, false, "verbose information output.");
-    options.addOption(verbose);
-
-    Option runas = new Option(RUNAS, false, "Run the command as \"hdfs\" user");
-    options.addOption(runas);
-
-    Option userName = new Option(USER, true,
-                                 "Name of the user in volume management " +
-                                     "functions");
-    options.addOption(userName);
-
-    Option quota = new Option(QUOTA, true, "Quota for the volume. E.g. 10TB");
-    options.addOption(quota);
-
-
-    Option createVolume = new Option(CREATE_VOLUME, true, "creates a volume" +
-        "for the specified user.\n \t For example : hdfs o3  -createVolume " +
-        "<volumeURI> -root -user <userName>\n");
-    options.addOption(createVolume);
-
-    Option deleteVolume = new Option(DELETE_VOLUME, true, "deletes a volume" +
-        "if it is empty.\n \t For example : ozone oz -deleteVolume <volumeURI>" +
-        " -root \n");
-    options.addOption(deleteVolume);
-
-    Option listVolume =
-        new Option(LIST_VOLUME, true, "List the volumes of a given user.\n" +
-            "For example : ozone oz -listVolume <ozoneURI>" +
-            "-user <username> -root or ozone oz " +
-            "-listVolume");
-    listVolume.setOptionalArg(true);
-    options.addOption(listVolume);
-
-    Option updateVolume =
-        new Option(UPDATE_VOLUME, true, "updates an existing volume.\n" +
-            "\t For example : ozone oz " +
-            "-updateVolume <volumeURI> -quota " +
-            "100TB\n");
-    options.addOption(updateVolume);
-
-    Option infoVolume = new Option(INFO_VOLUME, true,
-                                   "returns information about a specific " +
-                                       "volume.");
-    options.addOption(infoVolume);
-  }
-
-  /**
-   * All bucket related commands for ozone.
-   *
-   * @param opts - Options
-   */
-  private void addBucketCommands(Options opts) {
-    Option createBucket = new Option(CREATE_BUCKET, true,
-        "creates a bucket in a given volume." +
-        "For example: ozone oz -createBucket <bucketURI>");
-    opts.addOption(createBucket);
-
-    Option infoBucket =
-        new Option(INFO_BUCKET, true, "returns information about a bucket.");
-    opts.addOption(infoBucket);
-
-    Option deleteBucket =
-        new Option(DELETE_BUCKET, true, "deletes an empty bucket.");
-    opts.addOption(deleteBucket);
-
-    Option listBucket =
-        new Option(LIST_BUCKET, true, "lists the buckets in a volume.");
-    opts.addOption(listBucket);
-
-    Option updateBucket =
-        new Option(UPDATE_BUCKET, true, "allows changing bucket attributes.\n" +
-            " For example: ozone oz -updateBucket <bucketURI> " +
-            "-addAcl user:frodo:rw");
-    opts.addOption(updateBucket);
-
-    Option addAcl =
-        new Option(ADD_ACLS, true, "allows user to add acls to a bucket.");
-    opts.addOption(addAcl);
-
-    Option removeAcl =
-        new Option(REMOVE_ACLS, true, "allows user to remove acls from a " +
-            "bucket.");
-    opts.addOption(removeAcl);
-  }
-
-  /**
-   * All key commands.
-   *
-   * @param opts - options
-   */
-  private void addKeyCommands(Options opts) {
-    Option putKey =
-        new Option(PUT_KEY, true, "creates or overwrites an existing key");
-    opts.addOption(putKey);
-
-    Option deleteKey =
-        new Option(DELETE_KEY, true, "deletes an existing key");
-    opts.addOption(deleteKey);
-
-    Option infoKey =
-        new Option(INFO_KEY, true, "returns information about an existing key");
-    opts.addOption(infoKey);
-
-    Option listKey =
-        new Option(LIST_KEY, true, "list all keys in a given bucket");
-    opts.addOption(listKey);
-
-    Option getKey =
-        new Option(GET_KEY, true, "Gets a specific key from ozone server.");
-    opts.addOption(getKey);
-
-    Option fileArgument =
-        new Option(FILE, true, "Data file path");
-    opts.addOption(fileArgument);
-
-    Option repFactor =
-        new Option(REPLICATION_FACTOR, true, "Replication factor (1 or 3)");
-    opts.addOption(repFactor);
-  }
-
-  /**
-   * Sub commands for list command.
-   * @param opts
-   */
-  private void addListingCommands(Options opts) {
-    Option maxKeys = new Option(LIST_LENGTH, true,
-        "Specify the max length of listing result.");
-    opts.addOption(maxKeys);
-
-    Option prevKey = new Option(START, true,
-        "Specify the start key where to start listing from.");
-    opts.addOption(prevKey);
-
-    Option prefix = new Option(PREFIX, true,
-        "Specify the prefix to filter the listing result.");
-    opts.addOption(prefix);
+    new Shell().run(argv);
   }
 
-  /**
-   * Dispatches calls to the right command Handler classes.
-   *
-   * @param cmd - CommandLine
-   *
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
-   */
-  private int dispatch(CommandLine cmd, Options opts)
-      throws IOException, OzoneException, URISyntaxException {
-    Handler handler = null;
-    final int eightyColumn = 80;
-
-    try {
-
-      // volume functions
-      if (cmd.hasOption(Shell.CREATE_VOLUME)) {
-        handler = new CreateVolumeHandler();
-      }
-
-      if (cmd.hasOption(Shell.DELETE_VOLUME)) {
-        handler = new DeleteVolumeHandler();
-      }
-
-      if (cmd.hasOption(Shell.LIST_VOLUME)) {
-        handler = new ListVolumeHandler();
-      }
-
-      if (cmd.hasOption(Shell.UPDATE_VOLUME)) {
-        handler = new UpdateVolumeHandler();
-      }
 
-      if (cmd.hasOption(Shell.INFO_VOLUME)) {
-        handler = new InfoVolumeHandler();
-      }
 
-      // bucket functions
-      if (cmd.hasOption(Shell.CREATE_BUCKET)) {
-        handler = new CreateBucketHandler();
-      }
-
-      if (cmd.hasOption(Shell.DELETE_BUCKET)) {
-        handler = new DeleteBucketHandler();
-      }
-
-      if (cmd.hasOption(Shell.INFO_BUCKET)) {
-        handler = new InfoBucketHandler();
-      }
-
-      if (cmd.hasOption(Shell.LIST_BUCKET)) {
-        handler = new ListBucketHandler();
-      }
-
-      if(cmd.hasOption(Shell.UPDATE_BUCKET)){
-        handler = new UpdateBucketHandler();
-      }
-
-      //Key Functions
-
-      if(cmd.hasOption(Shell.PUT_KEY)) {
-        handler = new PutKeyHandler();
-      }
-
-      if(cmd.hasOption(Shell.DELETE_KEY)) {
-        handler = new DeleteKeyHandler();
-      }
-
-      if(cmd.hasOption(Shell.INFO_KEY)) {
-        handler = new InfoKeyHandler();
-      }
-
-      if(cmd.hasOption(Shell.LIST_KEY)) {
-        handler = new ListKeyHandler();
-      }
-
-      if(cmd.hasOption(Shell.GET_KEY)) {
-        handler = new GetKeyHandler();
-      }
-
-      if (handler != null) {
-        handler.execute(cmd);
-        return 0;
-      } else {
-        HelpFormatter helpFormatter = new HelpFormatter();
-        helpFormatter.printHelp(eightyColumn, "ozone oz -command uri [args]",
-            "Ozone Commands",
-            opts, "Please correct your command and try again.");
-        return 1;
-      }
-    } catch (IOException | URISyntaxException ex) {
-      System.err.printf("Command Failed : %s%n", ex.getMessage());
-    } catch (OzoneException ex) {
-      System.err.printf("Command Failed : %s%n", ex.toJsonString());
-      LOG.debug("Command Failed.", ex);
-    } catch (IllegalArgumentException ex) {
-      System.err.printf("Illegal argument: %s%n", ex.getMessage());
-    }
-    return 1;
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java
index 0788f9e..5fe5c7b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/CreateBucketHandler.java
@@ -17,59 +17,48 @@
  */
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import org.apache.commons.cli.CommandLine;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
 import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
 
 /**
  * create bucket handler.
  */
+@Command(name = "-createBucket",
+    description = "creates a bucket in a given volume")
 public class CreateBucketHandler extends Handler {
 
-  private String volumeName;
-  private String bucketName;
+  @Parameters(arity = "1..1", description = Shell.OZONE_BUCKET_URI_DESCRIPTION)
+  private String uri;
 
   /**
    * Executes create bucket.
-   *
-   * @param cmd - CommandLine
-   *
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.CREATE_BUCKET)) {
-      throw new OzoneClientException(
-          "Incorrect call : createBucket is missing");
-    }
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.CREATE_BUCKET);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 2) {
       throw new OzoneClientException(
           "volume and bucket name required in createBucket");
     }
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
+    String volumeName = path.getName(0).toString();
+    String bucketName = path.getName(1).toString();
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Bucket Name : %s%n", bucketName);
     }
@@ -77,10 +66,11 @@ public class CreateBucketHandler extends Handler {
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     vol.createBucket(bucketName);
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       OzoneBucket bucket = vol.getBucket(bucketName);
       System.out.printf(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
           JsonUtils.toJsonString(OzoneClientUtils.asBucketInfo(bucket))));
     }
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
index 5fc443e..5a9b5c5 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/DeleteBucketHandler.java
@@ -18,60 +18,51 @@
 
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.client.OzoneVolume;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
 import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
 
 /**
  * Delete bucket Handler.
  */
+@Command(name = "-deleteBucket",
+    description = "deletes an empty bucket")
 public class DeleteBucketHandler extends Handler {
-  private String volumeName;
-  private String bucketName;
+
+  @Parameters(arity = "1..1", description = Shell.OZONE_BUCKET_URI_DESCRIPTION)
+  private String uri;
 
   /**
    * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   *
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.DELETE_BUCKET)) {
-      throw new OzoneClientException(
-          "Incorrect call : deleteBucket is missing");
-    }
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.DELETE_BUCKET);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 2) {
       throw new OzoneClientException(
           "volume and bucket name required in delete Bucket");
     }
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
+    String volumeName = path.getName(0).toString();
+    String bucketName = path.getName(1).toString();
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Bucket Name : %s%n", bucketName);
     }
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     vol.deleteBucket(bucketName);
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
index b3ca4e5..aa1ce87 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/InfoBucketHandler.java
@@ -17,49 +17,38 @@
  */
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 
-import org.apache.commons.cli.CommandLine;
 import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
 
 /**
  * Executes Info bucket.
  */
+@Command(name = "-infoBucket",
+    description = "returns information about a bucket")
 public class InfoBucketHandler extends Handler {
-  private String volumeName;
-  private String bucketName;
+
+  @Parameters(arity = "1..1", description = Shell.OZONE_BUCKET_URI_DESCRIPTION)
+  private String uri;
 
   /**
    * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   *
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.INFO_BUCKET)) {
-      throw new OzoneClientException(
-          "Incorrect call : infoBucket is missing");
-    }
-
-    String ozoneURIString = cmd.getOptionValue(Shell.INFO_BUCKET);
-    URI ozoneURI = verifyURI(ozoneURIString);
+  public Void call() throws Exception {
+    String volumeName, bucketName;
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
 
     if (path.getNameCount() < 2) {
@@ -70,7 +59,7 @@ public class InfoBucketHandler extends Handler {
     volumeName = path.getName(0).toString();
     bucketName = path.getName(1).toString();
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Bucket Name : %s%n", bucketName);
     }
@@ -80,6 +69,7 @@ public class InfoBucketHandler extends Handler {
 
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
         JsonUtils.toJsonString(OzoneClientUtils.asBucketInfo(bucket))));
+    return null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
index 655022a..3d429cf 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
@@ -18,95 +18,91 @@
 
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import org.apache.commons.cli.CommandLine;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
 import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Help.Visibility;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
 
 /**
  * Executes List Bucket.
  */
+@Command(name = "-listBucket",
+    description = "lists the buckets in a volume.")
 public class ListBucketHandler extends Handler {
-  private String volumeName;
 
+  @Parameters(arity = "1..1", description = Shell.OZONE_VOLUME_URI_DESCRIPTION)
+  private String uri;
+
+  @Option(names = {"--length", "-length", "-l"},
+      description = "Limit of the max results",
+      defaultValue = "100",
+      showDefaultValue = Visibility.ALWAYS)
+  private int maxBuckets;
+
+  @Option(names = {"--start", "-start", "-s"},
+      description = "The first bucket to start the listing")
+  private String startBucket;
+
+  @Option(names = {"--prefix", "-prefix", "-p"},
+      description = "Prefix to filter the buckets")
+  private String prefix;
   /**
    * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   *
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.LIST_BUCKET)) {
-      throw new OzoneClientException(
-          "Incorrect call : listBucket is missing");
-    }
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.LIST_BUCKET);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 1) {
       throw new OzoneClientException("volume is required in listBucket");
     }
 
-    volumeName = path.getName(0).toString();
-
-    if (cmd.hasOption(Shell.VERBOSE)) {
-      System.out.printf("Volume Name : %s%n", volumeName);
+    if (maxBuckets < 1) {
+      throw new IllegalArgumentException(
+          "the length should be a positive number");
     }
 
-    int maxBuckets = Integer.MAX_VALUE;
-    if (cmd.hasOption(Shell.LIST_LENGTH)) {
-      String length = cmd.getOptionValue(Shell.LIST_LENGTH);
-      OzoneUtils.verifyMaxKeyLength(length);
-      maxBuckets = Integer.parseInt(length);
-    }
+    String volumeName = path.getName(0).toString();
 
-    String startBucket = null;
-    if (cmd.hasOption(Shell.START)) {
-      startBucket = cmd.getOptionValue(Shell.START);
+    if (isVerbose()) {
+      System.out.printf("Volume Name : %s%n", volumeName);
     }
 
-    String prefix = null;
-    if (cmd.hasOption(Shell.PREFIX)) {
-      prefix = cmd.getOptionValue(Shell.PREFIX);
-    }
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     Iterator<OzoneBucket> bucketIterator = vol.listBuckets(prefix, startBucket);
     List<BucketInfo> bucketList = new ArrayList<>();
     while (maxBuckets > 0 && bucketIterator.hasNext()) {
-      BucketInfo bucketInfo = OzoneClientUtils.asBucketInfo(bucketIterator.next());
+      BucketInfo bucketInfo =
+          OzoneClientUtils.asBucketInfo(bucketIterator.next());
       bucketList.add(bucketInfo);
       maxBuckets -= 1;
     }
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Found : %d buckets for volume : %s ",
           bucketList.size(), volumeName);
     }
     System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
         JsonUtils.toJsonString(bucketList)));
+    return null;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java
index aff0e19..3728740 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/UpdateBucketHandler.java
@@ -17,43 +17,50 @@
  */
 package org.apache.hadoop.ozone.web.ozShell.bucket;
 
-import org.apache.commons.cli.CommandLine;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.OzoneVolume;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
 
 /**
  * Allows users to add and remove acls and from a bucket.
  */
+@Command(name = "-updateBucket",
+    description = "allows changing bucket attributes")
 public class UpdateBucketHandler extends Handler {
-  private String volumeName;
-  private String bucketName;
+
+  @Parameters(arity = "1..1", description = Shell.OZONE_BUCKET_URI_DESCRIPTION)
+  private String uri;
+
+  @Option(names = {"--addAcl", "-addAcl"},
+      description = "Comma separated list of acl rules to add (eg. " +
+          "user:bilbo:rw)")
+  private String addAcl;
+
+  @Option(names = {"--removeAcl", "-removeAcl"},
+      description = "Comma separated list of acl rules to remove (eg. "
+          + "user:bilbo:rw)")
+  private String removeAcl;
 
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.UPDATE_BUCKET)) {
-      throw new OzoneClientException(
-          "Incorrect call : updateBucket is missing");
-    }
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.UPDATE_BUCKET);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
 
     if (path.getNameCount() < 2) {
@@ -61,28 +68,26 @@ public class UpdateBucketHandler extends Handler {
           "volume and bucket name required in update bucket");
     }
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
+    String volumeName = path.getName(0).toString();
+    String bucketName = path.getName(1).toString();
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Bucket Name : %s%n", bucketName);
     }
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     OzoneBucket bucket = vol.getBucket(bucketName);
-    if (cmd.hasOption(Shell.ADD_ACLS)) {
-      String aclString = cmd.getOptionValue(Shell.ADD_ACLS);
-      String[] aclArray = aclString.split(",");
+    if (addAcl != null) {
+      String[] aclArray = addAcl.split(",");
       List<OzoneAcl> aclList =
           Arrays.stream(aclArray).map(acl -> OzoneAcl.parseAcl(acl))
               .collect(Collectors.toList());
       bucket.addAcls(aclList);
     }
 
-    if (cmd.hasOption(Shell.REMOVE_ACLS)) {
-      String aclString = cmd.getOptionValue(Shell.REMOVE_ACLS);
-      String[] aclArray = aclString.split(",");
+    if (removeAcl != null) {
+      String[] aclArray = removeAcl.split(",");
       List<OzoneAcl> aclList =
           Arrays.stream(aclArray).map(acl -> OzoneAcl.parseAcl(acl))
               .collect(Collectors.toList());
@@ -91,5 +96,6 @@ public class UpdateBucketHandler extends Handler {
 
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
         JsonUtils.toJsonString(OzoneClientUtils.asBucketInfo(bucket))));
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/DeleteKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/DeleteKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/DeleteKeyHandler.java
index fccabe7..8c9aab7 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/DeleteKeyHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/DeleteKeyHandler.java
@@ -18,58 +18,47 @@
 
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
-import org.apache.commons.cli.CommandLine;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
 import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
 
 /**
  * Executes Delete Key.
  */
+@Command(name = "-deleteKey",
+    description = "deletes an existing key")
 public class DeleteKeyHandler extends Handler {
-  private String volumeName;
-  private String bucketName;
-  private String keyName;
+
+  @Parameters(arity = "1..1", description = Shell.OZONE_KEY_URI_DESCRIPTION)
+  private String uri;
 
   /**
    * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.DELETE_KEY)) {
-      throw new OzoneClientException(
-          "Incorrect call : deleteKey is missing");
-    }
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.DELETE_KEY);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 3) {
       throw new OzoneClientException(
           "volume/bucket/key name required in deleteKey");
     }
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
-    keyName = path.getName(2).toString();
-
+    String volumeName = path.getName(0).toString();
+    String bucketName = path.getName(1).toString();
+    String keyName = path.getName(2).toString();
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Bucket Name : %s%n", bucketName);
       System.out.printf("Key Name : %s%n", keyName);
@@ -78,5 +67,6 @@ public class DeleteKeyHandler extends Handler {
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     OzoneBucket bucket = vol.getBucket(bucketName);
     bucket.deleteKey(keyName);
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java
index 2d059e0..9843fd8 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/GetKeyHandler.java
@@ -18,83 +18,67 @@
 
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.codec.digest.DigestUtils;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.net.URI;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CHUNK_SIZE_DEFAULT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CHUNK_SIZE_KEY;
+import org.apache.commons.codec.digest.DigestUtils;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
 
 /**
  * Gets an existing key.
  */
+@Command(name = "-getKey",
+    description = "Gets a specific key from ozone server")
 public class GetKeyHandler extends Handler {
-  private String volumeName;
-  private String bucketName;
-  private String keyName;
 
+  @Parameters(arity = "1..1", description = Shell.OZONE_KEY_URI_DESCRIPTION)
+  private String uri;
+
+  @Option(names = {"-f", "--file", "-file"},
+      description = "File path to download the key to",
+      required = true)
+  private String fileName;
 
   /**
    * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   *
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.GET_KEY)) {
-      throw new OzoneClientException("Incorrect call : getKey is missing");
-    }
+  public Void call() throws Exception {
 
-    if (!cmd.hasOption(Shell.FILE)) {
-      throw new OzoneClientException(
-          "get key needs a file path to download to");
-    }
-
-    String ozoneURIString = cmd.getOptionValue(Shell.GET_KEY);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 3) {
       throw new OzoneClientException(
           "volume/bucket/key name required in putKey");
     }
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
-    keyName = path.getName(2).toString();
+    String volumeName = path.getName(0).toString();
+    String bucketName = path.getName(1).toString();
+    String keyName = path.getName(2).toString();
 
-
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Bucket Name : %s%n", bucketName);
       System.out.printf("Key Name : %s%n", keyName);
     }
 
-
-    String fileName = cmd.getOptionValue(Shell.FILE);
     Path dataFilePath = Paths.get(fileName);
     File dataFile = new File(fileName);
 
@@ -120,12 +104,12 @@ public class GetKeyHandler extends Handler {
       throw new OzoneClientException(
           "Can not access the file \"" + fileName + "\"");
     }
-    if(cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       FileInputStream stream = new FileInputStream(dataFile);
       String hash = DigestUtils.md5Hex(stream);
       System.out.printf("Downloaded file hash : %s%n", hash);
       stream.close();
     }
-
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
index 53a3264..18c7591 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
@@ -18,61 +18,55 @@
 
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
-import java.io.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 
-import org.apache.commons.cli.CommandLine;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.*;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
 import org.apache.hadoop.ozone.web.utils.JsonUtils;
 
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
+
 /**
  * Executes Info Object.
  */
+@Command(name = "-infoKey",
+    description = "returns information about an existing key")
 public class InfoKeyHandler extends Handler {
-  private String volumeName;
-  private String bucketName;
-  private String keyName;
 
+  @Parameters(arity = "1..1", description = Shell.OZONE_KEY_URI_DESCRIPTION)
+  private String uri;
   /**
    * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
    */
   @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
-    if (!cmd.hasOption(Shell.INFO_KEY)) {
-      throw new OzoneClientException("Incorrect call : infoKey is missing");
-    }
-
-    String ozoneURIString = cmd.getOptionValue(Shell.INFO_KEY);
-    URI ozoneURI = verifyURI(ozoneURIString);
+  public Void call() throws Exception {
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 3) {
       throw new OzoneClientException(
           "volume/bucket/key name required in infoKey");
     }
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
+    String volumeName = path.getName(0).toString();
+    String bucketName = path.getName(1).toString();
 
     String searchString = volumeName + OzoneConsts.OZONE_URI_DELIMITER +
         bucketName + OzoneConsts.OZONE_URI_DELIMITER;
 
-    keyName = ozoneURIString.substring(ozoneURIString.indexOf(searchString) +
+    String keyName =
+        uri.substring(uri.indexOf(searchString) +
                 searchString.length());
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("Bucket Name : %s%n", bucketName);
       System.out.printf("Key Name : %s%n", keyName);
@@ -84,5 +78,6 @@ public class InfoKeyHandler extends Handler {
 
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
         JsonUtils.toJsonString(OzoneClientUtils.asKeyInfoDetails(key))));
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd792ce5/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
index 6e266fd..f69693d 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
@@ -18,78 +18,72 @@
 
 package org.apache.hadoop.ozone.web.ozShell.keys;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.client.*;
-import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.web.ozShell.Handler;
-import org.apache.hadoop.ozone.web.ozShell.Shell;
-import org.apache.hadoop.ozone.web.utils.JsonUtils;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-
-import java.io.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
+import org.apache.hadoop.ozone.web.ozShell.Handler;
+import org.apache.hadoop.ozone.web.ozShell.Shell;
+import org.apache.hadoop.ozone.web.utils.JsonUtils;
+
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
+
 /**
  * Executes List Keys.
  */
+@Command(name = "-listKey",
+    description = "list all keys in a given bucket")
 public class ListKeyHandler extends Handler {
-  private String volumeName;
-  private String bucketName;
 
-  /**
-   * Executes the Client Calls.
-   *
-   * @param cmd - CommandLine
-   * @throws IOException
-   * @throws OzoneException
-   * @throws URISyntaxException
-   */
-  @Override
-  protected void execute(CommandLine cmd)
-      throws IOException, OzoneException, URISyntaxException {
+  @Parameters(arity = "1..1", description = Shell.OZONE_BUCKET_URI_DESCRIPTION)
+  private String uri;
 
-    if (!cmd.hasOption(Shell.LIST_KEY)) {
-      throw new OzoneClientException(
-          "Incorrect call : listKey is missing");
-    }
+  @Option(names = {"--length", "-length", "-l"},
+      description = "Limit of the max results",
+      defaultValue = "100")
+  private int maxKeys;
 
-    int maxKeys = Integer.MAX_VALUE;
-    if (cmd.hasOption(Shell.LIST_LENGTH)) {
-      String length = cmd.getOptionValue(Shell.LIST_LENGTH);
-      OzoneUtils.verifyMaxKeyLength(length);
-      maxKeys = Integer.parseInt(length);
-    }
+  @Option(names = {"--start", "-start", "-s"},
+      description = "The first key to start the listing")
+  private String startKey;
 
-    String startKey = null;
-    if (cmd.hasOption(Shell.START)) {
-      startKey = cmd.getOptionValue(Shell.START);
-    }
+  @Option(names = {"--prefix", "-prefix", "-p"},
+      description = "Prefix to filter the key")
+  private String prefix;
 
-    String prefix = null;
-    if (cmd.hasOption(Shell.PREFIX)) {
-      prefix = cmd.getOptionValue(Shell.PREFIX);
-    }
+  /**
+   * Executes the Client Calls.
+   */
+  @Override
+  public Void call() throws Exception {
 
-    String ozoneURIString = cmd.getOptionValue(Shell.LIST_KEY);
-    URI ozoneURI = verifyURI(ozoneURIString);
+    URI ozoneURI = verifyURI(uri);
     Path path = Paths.get(ozoneURI.getPath());
     if (path.getNameCount() < 2) {
       throw new OzoneClientException(
           "volume/bucket is required in listKey");
     }
 
-    volumeName = path.getName(0).toString();
-    bucketName = path.getName(1).toString();
+    if (maxKeys < 1) {
+      throw new IllegalArgumentException(
+          "the length should be a positive number");
+    }
 
+    String volumeName = path.getName(0).toString();
+    String bucketName = path.getName(1).toString();
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Volume Name : %s%n", volumeName);
       System.out.printf("bucket Name : %s%n", bucketName);
     }
@@ -105,12 +99,13 @@ public class ListKeyHandler extends Handler {
       maxKeys -= 1;
     }
 
-    if (cmd.hasOption(Shell.VERBOSE)) {
+    if (isVerbose()) {
       System.out.printf("Found : %d keys for bucket %s in volume : %s ",
           keyInfos.size(), bucketName, volumeName);
     }
     System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(
         JsonUtils.toJsonString(keyInfos)));
+    return null;
   }
 
 }


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