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 ar...@apache.org on 2016/06/24 06:05:59 UTC

[28/49] hadoop git commit: HDFS-10517. DiskBalancer: Support help command. Contributed by Anu Engineer.

HDFS-10517. DiskBalancer: Support help command. Contributed by Anu Engineer.


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

Branch: refs/heads/trunk
Commit: af11ab34d0cff3883885a25aa918be4f98566142
Parents: b502102
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Jun 13 14:02:04 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/command/CancelCommand.java     |  20 +-
 .../server/diskbalancer/command/Command.java    |   4 +-
 .../diskbalancer/command/ExecuteCommand.java    |  17 +-
 .../diskbalancer/command/HelpCommand.java       | 108 +++++++++
 .../diskbalancer/command/PlanCommand.java       |  28 ++-
 .../diskbalancer/command/QueryCommand.java      |  15 +-
 .../diskbalancer/command/ReportCommand.java     |  18 +-
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  | 233 ++++++++++++++-----
 8 files changed, 360 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
index 3834d9b..740292d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -126,12 +127,21 @@ public class CancelCommand extends Command {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
   @Override
-  protected String getHelp() {
-    return "Cancels a running command. e.g -cancel <PlanFile> or -cancel " +
-        "<planID> -node <datanode>";
+  public void printHelp() {
+    String header = "Cancel command cancels a running disk balancer operation" +
+        ".\n\n";
+
+    String footer = "\nCancel command can be run via pointing to a plan file," +
+        " or by reading the plan ID using the query command and then using " +
+        "planID and hostname. Examples of how to run this command are \n" +
+        "hdfs diskbalancer -cancel <planfile> \n" +
+        "hdfs diskbalancer -cancel <planID> -node <hostname>";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -cancel <planFile> | -cancel " +
+        "<planID> -node <hostname>",
+        header, DiskBalancer.getCancelOptions(), footer);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index bbf91ca..d2813e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -100,10 +100,8 @@ public abstract class Command extends Configured {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
-  protected abstract String getHelp();
+  public abstract void printHelp();
 
   /**
    * verifies user provided URL.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
index 85f2a86..5fd1f0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -98,12 +99,18 @@ public class ExecuteCommand extends Command {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
   @Override
-  protected String getHelp() {
-    return "Execute command takes a plan and runs it against the node. e.g. " +
-        "hdfs diskbalancer -execute <nodename.plan.json> ";
+  public void printHelp() {
+    String header = "Execute command runs a submits a plan for execution on " +
+        "the given data node.\n\n";
+
+    String footer = "\nExecute command submits the job to data node and " +
+        "returns immediately. The state of job can be monitored via query " +
+        "command. ";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -execute <planfile>",
+        header, DiskBalancer.getExecuteOptions(), footer);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
new file mode 100644
index 0000000..205df3d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
@@ -0,0 +1,108 @@
+/*
+ * 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.hdfs.server.diskbalancer.command;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+
+/**
+ * Help Command prints out detailed help about each command.
+ */
+public class HelpCommand extends Command {
+
+  /**
+   * Constructs a help command.
+   *
+   * @param conf - config
+   */
+  public HelpCommand(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * Executes the Client Calls.
+   *
+   * @param cmd - CommandLine
+   */
+  @Override
+  public void execute(CommandLine cmd) throws Exception {
+    LOG.debug("Processing help Command.");
+    if (cmd == null) {
+      this.printHelp();
+      return;
+    }
+
+    Preconditions.checkState(cmd.hasOption(DiskBalancer.HELP));
+    verifyCommandOptions(DiskBalancer.HELP, cmd);
+    String helpCommand = cmd.getOptionValue(DiskBalancer.HELP);
+    if (helpCommand == null || helpCommand.isEmpty()) {
+      this.printHelp();
+      return;
+    }
+
+    helpCommand = helpCommand.trim();
+    helpCommand = helpCommand.toLowerCase();
+    Command command = null;
+    switch (helpCommand) {
+    case DiskBalancer.PLAN:
+      command = new PlanCommand(getConf());
+      break;
+    case DiskBalancer.EXECUTE:
+      command = new ExecuteCommand(getConf());
+      break;
+    case DiskBalancer.QUERY:
+      command = new QueryCommand(getConf());
+      break;
+    case DiskBalancer.CANCEL:
+      command = new CancelCommand(getConf());
+      break;
+    case DiskBalancer.REPORT:
+      command = new ReportCommand(getConf(), null);
+      break;
+    default:
+      command = this;
+      break;
+    }
+    command.printHelp();
+
+  }
+
+  /**
+   * Gets extended help for this command.
+   */
+  @Override
+  public void printHelp() {
+    String header = "\nDiskBalancer distributes data evenly between " +
+        "different disks on a datanode. " +
+        "DiskBalancer operates by generating a plan, that tells datanode " +
+        "how to move data between disks. Users can execute a plan by " +
+        "submitting it to the datanode. \nTo get specific help on a " +
+        "particular command please run \n\n hdfs diskbalancer -help <command>.";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer [command] [options]",
+        header, DiskBalancer.getHelpOptions(), "");
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index 91ab7fb..76bdc9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -19,15 +19,18 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerVolumeSet;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.apache.hadoop.hdfs.tools.DiskBalancer;
@@ -182,12 +185,19 @@ public class PlanCommand extends Command {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
   @Override
-  protected String getHelp() {
-    return "This commands creates a disk balancer plan for given datanode";
+  public void printHelp() {
+    String header = "creates a plan that describes how much data should be " +
+        "moved between disks.\n\n";
+
+    String footer = "\nPlan command creates a set of steps that represent a " +
+        "planned data move. A plan file can be executed on a data node, which" +
+        " will balance the data.";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -uri <namenode> -plan " +
+        "<hostname> [options]", header, DiskBalancer.getPlanOptions(), footer);
   }
 
   /**
@@ -221,9 +231,9 @@ public class PlanCommand extends Command {
 
     System.out.println(
         StringUtils.center("Source Disk", 30) +
-        StringUtils.center("Dest.Disk", 30) +
-        StringUtils.center("Size", 10) +
-        StringUtils.center("Type", 10));
+            StringUtils.center("Dest.Disk", 30) +
+            StringUtils.center("Size", 10) +
+            StringUtils.center("Type", 10));
 
     for (NodePlan plan : plans) {
       for (Step step : plan.getVolumeSetPlans()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
index ea7dbcc..3a3b97f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -86,11 +87,17 @@ public class QueryCommand extends Command {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
   @Override
-  protected String getHelp() {
-    return "Gets the status of disk balancing on a given node";
+  public void printHelp() {
+    String header = "Query Plan queries a given data node about the " +
+        "current state of disk balancer execution.\n\n";
+
+    String footer = "\nQuery command retrievs the plan ID and the current " +
+        "running state. ";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -query <hostname>  [options]",
+        header, DiskBalancer.getQueryOptions(), footer);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
index acf9ff2..eb6afcc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.ListIterator;
 
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.text.StrBuilder;
 import org.apache.hadoop.conf.Configuration;
@@ -184,14 +185,23 @@ public class ReportCommand extends Command {
     }
   }
 
+  /**
+   * Prints the help message.
+   */
   @Override
-  protected String getHelp() {
-    return "Report volume information for a specific DataNode or top X "
-        + "one(s) benefiting from running DiskBalancer, "
-        + "top defaults to " + getDefaultTop() + ". E.g.:\n"
+  public void printHelp() {
+    String header = "Report command reports the volume information of a given" +
+        " datanode, or prints out the list of nodes that will benefit from " +
+        "running disk balancer. Top defaults to " + getDefaultTop();
+    String footer = ". E.g.:\n"
         + "hdfs diskbalancer -uri http://namenode.uri -report\n"
         + "hdfs diskbalancer -uri http://namenode.uri -report -top 5\n"
         + "hdfs diskbalancer -uri http://namenode.uri -report "
         + "-node {DataNodeID | IP | Hostname}";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -uri http://namenode.uri " +
+        "-report [options]",
+        header, DiskBalancer.getReportOptions(), footer);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index 1251e96..d83a49c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -18,8 +18,8 @@ package org.apache.hadoop.hdfs.tools;
 
 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.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.CancelCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.Command;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.ExecuteCommand;
+import org.apache.hadoop.hdfs.server.diskbalancer.command.HelpCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.PlanCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.QueryCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.ReportCommand;
@@ -139,6 +140,13 @@ public class DiskBalancer extends Configured implements Tool {
   private static final Logger LOG =
       LoggerFactory.getLogger(DiskBalancer.class);
 
+  private static final Options PLAN_OPTIONS = new Options();
+  private static final Options EXECUTE_OPTIONS = new Options();
+  private static final Options QUERY_OPTIONS = new Options();
+  private static final Options HELP_OPTIONS = new Options();
+  private static final Options CANCEL_OPTIONS = new Options();
+  private static final Options REPORT_OPTIONS = new Options();
+
   /**
    * Construct a DiskBalancer.
    *
@@ -182,7 +190,7 @@ public class DiskBalancer extends Configured implements Tool {
    * Execute the command with the given arguments.
    *
    * @param args command specific arguments.
-   * @param out the output stream used for printing
+   * @param out  the output stream used for printing
    * @return exit code.
    * @throws Exception
    */
@@ -200,6 +208,7 @@ public class DiskBalancer extends Configured implements Tool {
   private Options getOpts() {
     Options opts = new Options();
     addPlanCommands(opts);
+    addHelpCommands(opts);
     addExecuteCommands(opts);
     addQueryCommands(opts);
     addCancelCommands(opts);
@@ -208,101 +217,216 @@ public class DiskBalancer extends Configured implements Tool {
   }
 
   /**
+   * Returns Plan options.
+   *
+   * @return Options.
+   */
+  public static Options getPlanOptions() {
+    return PLAN_OPTIONS;
+  }
+
+  /**
+   * Returns help options.
+   *
+   * @return - help options.
+   */
+  public static Options getHelpOptions() {
+    return HELP_OPTIONS;
+  }
+
+  /**
+   * Retuns execute options.
+   *
+   * @return - execute options.
+   */
+  public static Options getExecuteOptions() {
+    return EXECUTE_OPTIONS;
+  }
+
+  /**
+   * Returns Query Options.
+   *
+   * @return query Options
+   */
+  public static Options getQueryOptions() {
+    return QUERY_OPTIONS;
+  }
+
+  /**
+   * Returns Cancel Options.
+   *
+   * @return Options
+   */
+  public static Options getCancelOptions() {
+    return CANCEL_OPTIONS;
+  }
+
+  /**
+   * Returns Report Options.
+   *
+   * @return Options
+   */
+  public static Options getReportOptions() {
+    return REPORT_OPTIONS;
+  }
+
+  /**
    * Adds commands for plan command.
    *
-   * @param opt - Options
+   * @return Options.
    */
   private void addPlanCommands(Options opt) {
 
-    Option nameNodeUri =
-        new Option(NAMENODEURI, true, "NameNode URI. e.g http://namenode" +
-            ".mycluster.com or file:///myCluster" +
-            ".json");
-    opt.addOption(nameNodeUri);
+    Option uri = OptionBuilder.withLongOpt(NAMENODEURI)
+        .withDescription("Address of the Namenode. e,g. hdfs://namenode")
+        .hasArg()
+        .create();
+    getPlanOptions().addOption(uri);
+    opt.addOption(uri);
+
+    Option plan = OptionBuilder.withLongOpt(PLAN)
+        .withDescription("creates a plan for datanode.")
+        .hasArg()
+        .create();
+    getPlanOptions().addOption(plan);
+    opt.addOption(plan);
 
-    Option outFile =
-        new Option(OUTFILE, true, "File to write output to, if not specified " +
-            "defaults will be used." +
-            "e.g -out outfile.txt");
-    opt.addOption(outFile);
 
-    Option plan = new Option(PLAN, true , "create a plan for the given node. " +
-        "e.g -plan <nodename> | <nodeIP> | <nodeUUID>");
-    opt.addOption(plan);
+    Option outFile = OptionBuilder.withLongOpt(OUTFILE)
+        .hasArg()
+        .withDescription("File to write output to, if not specified " +
+            "defaults will be used.")
+        .create();
+    getPlanOptions().addOption(outFile);
+    opt.addOption(outFile);
 
-    Option bandwidth = new Option(BANDWIDTH, true, "Maximum disk bandwidth to" +
-        " be consumed by diskBalancer. " +
-        "Expressed as MBs per second.");
+    Option bandwidth = OptionBuilder.withLongOpt(BANDWIDTH)
+        .hasArg()
+        .withDescription("Maximum disk bandwidth to be consumed by " +
+            "diskBalancer. e.g. 10")
+        .create();
+    getPlanOptions().addOption(bandwidth);
     opt.addOption(bandwidth);
 
-    Option threshold = new Option(THRESHOLD, true, "Percentage skew that we " +
-        "tolerate before diskbalancer starts working or stops when reaching " +
-        "that range.");
+    Option threshold = OptionBuilder.withLongOpt(THRESHOLD)
+        .hasArg()
+        .withDescription("Percentage skew that we" +
+            "tolerate before diskbalancer starts working e.g. 10")
+        .create();
+    getPlanOptions().addOption(threshold);
     opt.addOption(threshold);
 
-    Option maxErrors = new Option(MAXERROR, true, "Describes how many errors " +
-        "can be tolerated while copying between a pair of disks.");
-    opt.addOption(maxErrors);
 
-    Option help =
-        new Option(HELP, true, "Help about a command or this message");
-    opt.addOption(help);
+    Option maxError = OptionBuilder.withLongOpt(MAXERROR)
+        .hasArg()
+        .withDescription("Describes how many errors " +
+            "can be tolerated while copying between a pair of disks.")
+        .create();
+    getPlanOptions().addOption(maxError);
+    opt.addOption(maxError);
 
-    Option verbose = new Option(VERBOSE, "Print out the summary of the plan");
+    Option verbose = OptionBuilder.withLongOpt(VERBOSE)
+        .withDescription("Print out the summary of the plan on console")
+        .create();
+    getPlanOptions().addOption(verbose);
     opt.addOption(verbose);
+  }
 
+  /**
+   * Adds Help to the options.
+   */
+  private void addHelpCommands(Options opt) {
+    Option help = OptionBuilder.withLongOpt(HELP)
+        .hasOptionalArg()
+        .withArgName(HELP)
+        .withDescription("valid commands are plan | execute | query | cancel" +
+            " | report")
+        .create();
+    getHelpOptions().addOption(help);
+    opt.addOption(help);
   }
 
   /**
    * Adds execute command options.
+   *
    * @param opt Options
    */
   private void addExecuteCommands(Options opt) {
-    Option execute = new Option(EXECUTE, true , "Takes a plan file and " +
-        "submits it for execution to the datanode. e.g -execute <planfile>");
+    Option execute = OptionBuilder.withLongOpt(EXECUTE)
+        .hasArg()
+        .withDescription("Takes a plan file and " +
+            "submits it for execution by the datanode.")
+        .create();
+    getExecuteOptions().addOption(execute);
     opt.addOption(execute);
   }
 
   /**
    * Adds query command options.
+   *
    * @param opt Options
    */
   private void addQueryCommands(Options opt) {
-    Option query = new Option(QUERY, true, "Queries the disk balancer " +
-        "status of a given datanode. e.g. -query <nodename>");
+    Option query = OptionBuilder.withLongOpt(QUERY)
+        .hasArg()
+        .withDescription("Queries the disk balancer " +
+            "status of a given datanode.")
+        .create();
+    getQueryOptions().addOption(query);
     opt.addOption(query);
+
+    // Please note: Adding this only to Query options since -v is already
+    // added to global table.
+    Option verbose = OptionBuilder.withLongOpt(VERBOSE)
+        .withDescription("Prints details of the plan that is being executed " +
+            "on the node.")
+        .create();
+    getQueryOptions().addOption(verbose);
   }
 
   /**
    * Adds cancel command options.
+   *
    * @param opt Options
    */
   private void addCancelCommands(Options opt) {
-    Option cancel = new Option(CANCEL, true, "Cancels a running plan. -cancel" +
-        " <planFile> or -cancel <planID> -node <datanode:port>");
+    Option cancel = OptionBuilder.withLongOpt(CANCEL)
+        .hasArg()
+        .withDescription("Cancels a running plan using a plan file.")
+        .create();
+    getCancelOptions().addOption(cancel);
     opt.addOption(cancel);
-    Option node = new Option(NODE, true, "Name of the datanode in name:port " +
-        "format");
+
+    Option node = OptionBuilder.withLongOpt(NODE)
+        .hasArg()
+        .withDescription("Cancels a running plan using a plan ID and hostName")
+        .create();
+
+    getCancelOptions().addOption(node);
     opt.addOption(node);
   }
 
   /**
    * Adds report command options.
+   *
    * @param opt Options
    */
   private void addReportCommands(Options opt) {
-    Option report = new Option(REPORT, false,
-        "Report volume information of DataNode(s)"
-            + " benefiting from running DiskBalancer. "
-            + "-report [top -X] | [-node {DataNodeID | IP | Hostname}].");
+    Option report = OptionBuilder.withLongOpt(REPORT)
+        .withDescription("List nodes that will benefit from running " +
+            "DiskBalancer.")
+        .create();
+    getReportOptions().addOption(report);
     opt.addOption(report);
 
     Option top = new Option(TOP, true,
-        "specify the top number of nodes to be processed.");
+        "specify the number of nodes to be listed which has data imbalance.");
+    getReportOptions().addOption(top);
     opt.addOption(top);
 
     Option node = new Option(NODE, true,
-        "Name of the datanode in the format of DataNodeID, IP or hostname.");
+        "Datanode address, it can be DataNodeID, IP or hostname.");
+    getReportOptions().addOption(node);
     opt.addOption(node);
   }
 
@@ -322,9 +446,9 @@ public class DiskBalancer extends Configured implements Tool {
   /**
    * Dispatches calls to the right command Handler classes.
    *
-   * @param cmd - CommandLine
+   * @param cmd  - CommandLine
    * @param opts options of command line
-   * @param out the output stream used for printing
+   * @param out  the output stream used for printing
    * @throws IOException
    * @throws URISyntaxException
    */
@@ -338,15 +462,15 @@ public class DiskBalancer extends Configured implements Tool {
         currentCommand = new PlanCommand(getConf());
       }
 
-      if(cmd.hasOption(DiskBalancer.EXECUTE)) {
+      if (cmd.hasOption(DiskBalancer.EXECUTE)) {
         currentCommand = new ExecuteCommand(getConf());
       }
 
-      if(cmd.hasOption(DiskBalancer.QUERY)) {
+      if (cmd.hasOption(DiskBalancer.QUERY)) {
         currentCommand = new QueryCommand(getConf());
       }
 
-      if(cmd.hasOption(DiskBalancer.CANCEL)) {
+      if (cmd.hasOption(DiskBalancer.CANCEL)) {
         currentCommand = new CancelCommand(getConf());
       }
 
@@ -354,13 +478,16 @@ public class DiskBalancer extends Configured implements Tool {
         currentCommand = new ReportCommand(getConf(), out);
       }
 
-      if(currentCommand == null) {
-        HelpFormatter helpFormatter = new HelpFormatter();
-        helpFormatter.printHelp(80, "hdfs diskbalancer -uri [args]",
-            "disk balancer commands", opts,
-            "Please correct your command and try again.");
+      if (cmd.hasOption(DiskBalancer.HELP)) {
+        currentCommand = new HelpCommand(getConf());
+      }
+
+      // Invoke Main help here.
+      if (currentCommand == null) {
+        new HelpCommand(getConf()).execute(null);
         return 1;
       }
+
       currentCommand.execute(cmd);
     } catch (Exception ex) {
       System.err.printf(ex.getMessage());


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