Browse Source

HDFS-10553. DiskBalancer: Rename Tools/DiskBalancer class to Tools/DiskBalancerCLI. Contributed by Manoj Govindassamy.

Anu Engineer 8 years ago
parent
commit
35c5943b8b

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -127,7 +127,7 @@ function hdfscmd_case
       HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
       HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
     ;;
     ;;
     diskbalancer)
     diskbalancer)
-      HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.DiskBalancer
+      HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.DiskBalancerCLI
       hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS"
       hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS"
       HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
       HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
     ;;
     ;;

+ 12 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java

@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
@@ -44,9 +44,10 @@ public class CancelCommand extends Command {
    */
    */
   public CancelCommand(Configuration conf) {
   public CancelCommand(Configuration conf) {
     super(conf);
     super(conf);
-    addValidCommandParameters(DiskBalancer.CANCEL, "Cancels a running plan.");
-    addValidCommandParameters(DiskBalancer.NODE, "Node to run the command " +
-        "against in node:port format.");
+    addValidCommandParameters(DiskBalancerCLI.CANCEL,
+        "Cancels a running plan.");
+    addValidCommandParameters(DiskBalancerCLI.NODE,
+        "Node to run the command against in node:port format.");
   }
   }
 
 
   /**
   /**
@@ -57,20 +58,20 @@ public class CancelCommand extends Command {
   @Override
   @Override
   public void execute(CommandLine cmd) throws Exception {
   public void execute(CommandLine cmd) throws Exception {
     LOG.info("Executing \"Cancel plan\" command.");
     LOG.info("Executing \"Cancel plan\" command.");
-    Preconditions.checkState(cmd.hasOption(DiskBalancer.CANCEL));
-    verifyCommandOptions(DiskBalancer.CANCEL, cmd);
+    Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.CANCEL));
+    verifyCommandOptions(DiskBalancerCLI.CANCEL, cmd);
 
 
     // We can cancel a plan using datanode address and plan ID
     // We can cancel a plan using datanode address and plan ID
     // that you can read from a datanode using queryStatus
     // that you can read from a datanode using queryStatus
-    if(cmd.hasOption(DiskBalancer.NODE)) {
-      String nodeAddress = cmd.getOptionValue(DiskBalancer.NODE);
-      String planHash = cmd.getOptionValue(DiskBalancer.CANCEL);
+    if(cmd.hasOption(DiskBalancerCLI.NODE)) {
+      String nodeAddress = cmd.getOptionValue(DiskBalancerCLI.NODE);
+      String planHash = cmd.getOptionValue(DiskBalancerCLI.CANCEL);
       cancelPlanUsingHash(nodeAddress, planHash);
       cancelPlanUsingHash(nodeAddress, planHash);
     } else {
     } else {
       // Or you can cancel a plan using the plan file. If the user
       // Or you can cancel a plan using the plan file. If the user
       // points us to the plan file, we can compute the hash as well as read
       // points us to the plan file, we can compute the hash as well as read
       // the address of the datanode from the plan file.
       // the address of the datanode from the plan file.
-      String planFile = cmd.getOptionValue(DiskBalancer.CANCEL);
+      String planFile = cmd.getOptionValue(DiskBalancerCLI.CANCEL);
       Preconditions.checkArgument(planFile != null && !planFile.isEmpty(),
       Preconditions.checkArgument(planFile != null && !planFile.isEmpty(),
           "Invalid plan file specified.");
           "Invalid plan file specified.");
       String planData = null;
       String planData = null;
@@ -142,6 +143,6 @@ public class CancelCommand extends Command {
     HelpFormatter helpFormatter = new HelpFormatter();
     HelpFormatter helpFormatter = new HelpFormatter();
     helpFormatter.printHelp("hdfs diskbalancer -cancel <planFile> | -cancel " +
     helpFormatter.printHelp("hdfs diskbalancer -cancel <planFile> | -cancel " +
         "<planID> -node <hostname>",
         "<planID> -node <hostname>",
-        header, DiskBalancer.getCancelOptions(), footer);
+        header, DiskBalancerCLI.getCancelOptions(), footer);
   }
   }
 }
 }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 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.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -418,7 +418,7 @@ public abstract class Command extends Configured {
    * @return default top number of nodes.
    * @return default top number of nodes.
    */
    */
   protected int getDefaultTop() {
   protected int getDefaultTop() {
-    return DiskBalancer.DEFAULT_TOP;
+    return DiskBalancerCLI.DEFAULT_TOP;
   }
   }
 
 
   /**
   /**
@@ -437,7 +437,7 @@ public abstract class Command extends Configured {
   protected int parseTopNodes(final CommandLine cmd, final StrBuilder result) {
   protected int parseTopNodes(final CommandLine cmd, final StrBuilder result) {
     String outputLine = "";
     String outputLine = "";
     int nodes = 0;
     int nodes = 0;
-    final String topVal = cmd.getOptionValue(DiskBalancer.TOP);
+    final String topVal = cmd.getOptionValue(DiskBalancerCLI.TOP);
     if (StringUtils.isBlank(topVal)) {
     if (StringUtils.isBlank(topVal)) {
       outputLine = String.format(
       outputLine = String.format(
           "No top limit specified, using default top value %d.",
           "No top limit specified, using default top value %d.",

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java

@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
@@ -46,7 +46,8 @@ public class ExecuteCommand extends Command {
    */
    */
   public ExecuteCommand(Configuration conf) {
   public ExecuteCommand(Configuration conf) {
     super(conf);
     super(conf);
-    addValidCommandParameters(DiskBalancer.EXECUTE, "Executes a given plan.");
+    addValidCommandParameters(DiskBalancerCLI.EXECUTE,
+        "Executes a given plan.");
   }
   }
 
 
   /**
   /**
@@ -57,10 +58,10 @@ public class ExecuteCommand extends Command {
   @Override
   @Override
   public void execute(CommandLine cmd) throws Exception {
   public void execute(CommandLine cmd) throws Exception {
     LOG.info("Executing \"execute plan\" command");
     LOG.info("Executing \"execute plan\" command");
-    Preconditions.checkState(cmd.hasOption(DiskBalancer.EXECUTE));
-    verifyCommandOptions(DiskBalancer.EXECUTE, cmd);
+    Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.EXECUTE));
+    verifyCommandOptions(DiskBalancerCLI.EXECUTE, cmd);
 
 
-    String planFile = cmd.getOptionValue(DiskBalancer.EXECUTE);
+    String planFile = cmd.getOptionValue(DiskBalancerCLI.EXECUTE);
     Preconditions.checkArgument(planFile != null && !planFile.isEmpty(),
     Preconditions.checkArgument(planFile != null && !planFile.isEmpty(),
         "Invalid plan file specified.");
         "Invalid plan file specified.");
 
 
@@ -88,7 +89,7 @@ public class ExecuteCommand extends Command {
     String planHash = DigestUtils.shaHex(planData);
     String planHash = DigestUtils.shaHex(planData);
     try {
     try {
       // TODO : Support skipping date check.
       // TODO : Support skipping date check.
-      dataNode.submitDiskBalancerPlan(planHash, DiskBalancer.PLAN_VERSION,
+      dataNode.submitDiskBalancerPlan(planHash, DiskBalancerCLI.PLAN_VERSION,
                                       planFile, planData, false);
                                       planFile, planData, false);
     } catch (DiskBalancerException ex) {
     } catch (DiskBalancerException ex) {
       LOG.error("Submitting plan on  {} failed. Result: {}, Message: {}",
       LOG.error("Submitting plan on  {} failed. Result: {}, Message: {}",
@@ -111,6 +112,6 @@ public class ExecuteCommand extends Command {
 
 
     HelpFormatter helpFormatter = new HelpFormatter();
     HelpFormatter helpFormatter = new HelpFormatter();
     helpFormatter.printHelp("hdfs diskbalancer -execute <planfile>",
     helpFormatter.printHelp("hdfs diskbalancer -execute <planfile>",
-        header, DiskBalancer.getExecuteOptions(), footer);
+        header, DiskBalancerCLI.getExecuteOptions(), footer);
   }
   }
 }
 }

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java

@@ -23,7 +23,7 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 
 
 /**
 /**
  * Help Command prints out detailed help about each command.
  * Help Command prints out detailed help about each command.
@@ -37,7 +37,7 @@ public class HelpCommand extends Command {
    */
    */
   public HelpCommand(Configuration conf) {
   public HelpCommand(Configuration conf) {
     super(conf);
     super(conf);
-    addValidCommandParameters(DiskBalancer.HELP, "Help Command");
+    addValidCommandParameters(DiskBalancerCLI.HELP, "Help Command");
   }
   }
 
 
   /**
   /**
@@ -53,9 +53,9 @@ public class HelpCommand extends Command {
       return;
       return;
     }
     }
 
 
-    Preconditions.checkState(cmd.hasOption(DiskBalancer.HELP));
-    verifyCommandOptions(DiskBalancer.HELP, cmd);
-    String helpCommand = cmd.getOptionValue(DiskBalancer.HELP);
+    Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.HELP));
+    verifyCommandOptions(DiskBalancerCLI.HELP, cmd);
+    String helpCommand = cmd.getOptionValue(DiskBalancerCLI.HELP);
     if (helpCommand == null || helpCommand.isEmpty()) {
     if (helpCommand == null || helpCommand.isEmpty()) {
       this.printHelp();
       this.printHelp();
       return;
       return;
@@ -65,19 +65,19 @@ public class HelpCommand extends Command {
     helpCommand = helpCommand.toLowerCase();
     helpCommand = helpCommand.toLowerCase();
     Command command = null;
     Command command = null;
     switch (helpCommand) {
     switch (helpCommand) {
-    case DiskBalancer.PLAN:
+    case DiskBalancerCLI.PLAN:
       command = new PlanCommand(getConf());
       command = new PlanCommand(getConf());
       break;
       break;
-    case DiskBalancer.EXECUTE:
+    case DiskBalancerCLI.EXECUTE:
       command = new ExecuteCommand(getConf());
       command = new ExecuteCommand(getConf());
       break;
       break;
-    case DiskBalancer.QUERY:
+    case DiskBalancerCLI.QUERY:
       command = new QueryCommand(getConf());
       command = new QueryCommand(getConf());
       break;
       break;
-    case DiskBalancer.CANCEL:
+    case DiskBalancerCLI.CANCEL:
       command = new CancelCommand(getConf());
       command = new CancelCommand(getConf());
       break;
       break;
-    case DiskBalancer.REPORT:
+    case DiskBalancerCLI.REPORT:
       command = new ReportCommand(getConf(), null);
       command = new ReportCommand(getConf(), null);
       break;
       break;
     default:
     default:
@@ -102,7 +102,7 @@ public class HelpCommand extends Command {
 
 
     HelpFormatter helpFormatter = new HelpFormatter();
     HelpFormatter helpFormatter = new HelpFormatter();
     helpFormatter.printHelp("hdfs diskbalancer [command] [options]",
     helpFormatter.printHelp("hdfs diskbalancer [command] [options]",
-        header, DiskBalancer.getHelpOptions(), "");
+        header, DiskBalancerCLI.getHelpOptions(), "");
   }
   }
 
 
 
 

+ 32 - 31
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
     .DiskBalancerDataNode;
     .DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 import java.nio.charset.StandardCharsets;
 import java.nio.charset.StandardCharsets;
 import java.util.List;
 import java.util.List;
 
 
@@ -53,18 +53,18 @@ public class PlanCommand extends Command {
     this.thresholdPercentage = 1;
     this.thresholdPercentage = 1;
     this.bandwidth = 0;
     this.bandwidth = 0;
     this.maxError = 0;
     this.maxError = 0;
-    addValidCommandParameters(DiskBalancer.OUTFILE, "Output directory in " +
+    addValidCommandParameters(DiskBalancerCLI.OUTFILE, "Output directory in " +
         "HDFS. The generated plan will be written to a file in this " +
         "HDFS. The generated plan will be written to a file in this " +
         "directory.");
         "directory.");
-    addValidCommandParameters(DiskBalancer.BANDWIDTH, "Maximum Bandwidth to " +
-        "be used while copying.");
-    addValidCommandParameters(DiskBalancer.THRESHOLD, "Percentage skew that " +
-        "we tolerate before diskbalancer starts working.");
-    addValidCommandParameters(DiskBalancer.MAXERROR, "Max errors to tolerate " +
-        "between 2 disks");
-    addValidCommandParameters(DiskBalancer.VERBOSE, "Run plan command in " +
+    addValidCommandParameters(DiskBalancerCLI.BANDWIDTH,
+        "Maximum Bandwidth to be used while copying.");
+    addValidCommandParameters(DiskBalancerCLI.THRESHOLD,
+        "Percentage skew that we tolerate before diskbalancer starts working.");
+    addValidCommandParameters(DiskBalancerCLI.MAXERROR,
+        "Max errors to tolerate between 2 disks");
+    addValidCommandParameters(DiskBalancerCLI.VERBOSE, "Run plan command in " +
         "verbose mode.");
         "verbose mode.");
-    addValidCommandParameters(DiskBalancer.PLAN, "Plan Command");
+    addValidCommandParameters(DiskBalancerCLI.PLAN, "Plan Command");
   }
   }
 
 
   /**
   /**
@@ -77,36 +77,37 @@ public class PlanCommand extends Command {
   @Override
   @Override
   public void execute(CommandLine cmd) throws Exception {
   public void execute(CommandLine cmd) throws Exception {
     LOG.debug("Processing Plan Command.");
     LOG.debug("Processing Plan Command.");
-    Preconditions.checkState(cmd.hasOption(DiskBalancer.PLAN));
-    verifyCommandOptions(DiskBalancer.PLAN, cmd);
+    Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.PLAN));
+    verifyCommandOptions(DiskBalancerCLI.PLAN, cmd);
 
 
-    if (cmd.getOptionValue(DiskBalancer.PLAN) == null) {
+    if (cmd.getOptionValue(DiskBalancerCLI.PLAN) == null) {
       throw new IllegalArgumentException("A node name is required to create a" +
       throw new IllegalArgumentException("A node name is required to create a" +
           " plan.");
           " plan.");
     }
     }
 
 
-    if (cmd.hasOption(DiskBalancer.BANDWIDTH)) {
-      this.bandwidth = Integer.parseInt(cmd.getOptionValue(DiskBalancer
+    if (cmd.hasOption(DiskBalancerCLI.BANDWIDTH)) {
+      this.bandwidth = Integer.parseInt(cmd.getOptionValue(DiskBalancerCLI
           .BANDWIDTH));
           .BANDWIDTH));
     }
     }
 
 
-    if (cmd.hasOption(DiskBalancer.MAXERROR)) {
-      this.maxError = Integer.parseInt(cmd.getOptionValue(DiskBalancer
+    if (cmd.hasOption(DiskBalancerCLI.MAXERROR)) {
+      this.maxError = Integer.parseInt(cmd.getOptionValue(DiskBalancerCLI
           .MAXERROR));
           .MAXERROR));
     }
     }
 
 
     readClusterInfo(cmd);
     readClusterInfo(cmd);
     String output = null;
     String output = null;
-    if (cmd.hasOption(DiskBalancer.OUTFILE)) {
-      output = cmd.getOptionValue(DiskBalancer.OUTFILE);
+    if (cmd.hasOption(DiskBalancerCLI.OUTFILE)) {
+      output = cmd.getOptionValue(DiskBalancerCLI.OUTFILE);
     }
     }
     setOutputPath(output);
     setOutputPath(output);
 
 
     // -plan nodename is the command line argument.
     // -plan nodename is the command line argument.
-    DiskBalancerDataNode node = getNode(cmd.getOptionValue(DiskBalancer.PLAN));
+    DiskBalancerDataNode node =
+        getNode(cmd.getOptionValue(DiskBalancerCLI.PLAN));
     if (node == null) {
     if (node == null) {
       throw new IllegalArgumentException("Unable to find the specified node. " +
       throw new IllegalArgumentException("Unable to find the specified node. " +
-          cmd.getOptionValue(DiskBalancer.PLAN));
+          cmd.getOptionValue(DiskBalancerCLI.PLAN));
     }
     }
     this.thresholdPercentage = getThresholdPercentage(cmd);
     this.thresholdPercentage = getThresholdPercentage(cmd);
 
 
@@ -124,8 +125,8 @@ public class PlanCommand extends Command {
 
 
 
 
     try (FSDataOutputStream beforeStream = create(String.format(
     try (FSDataOutputStream beforeStream = create(String.format(
-        DiskBalancer.BEFORE_TEMPLATE,
-        cmd.getOptionValue(DiskBalancer.PLAN)))) {
+        DiskBalancerCLI.BEFORE_TEMPLATE,
+        cmd.getOptionValue(DiskBalancerCLI.PLAN)))) {
       beforeStream.write(getCluster().toJson()
       beforeStream.write(getCluster().toJson()
           .getBytes(StandardCharsets.UTF_8));
           .getBytes(StandardCharsets.UTF_8));
     }
     }
@@ -133,17 +134,17 @@ public class PlanCommand extends Command {
     if (plan != null && plan.getVolumeSetPlans().size() > 0) {
     if (plan != null && plan.getVolumeSetPlans().size() > 0) {
       LOG.info("Writing plan to : {}", getOutputPath());
       LOG.info("Writing plan to : {}", getOutputPath());
       try (FSDataOutputStream planStream = create(String.format(
       try (FSDataOutputStream planStream = create(String.format(
-          DiskBalancer.PLAN_TEMPLATE,
-          cmd.getOptionValue(DiskBalancer.PLAN)))) {
+          DiskBalancerCLI.PLAN_TEMPLATE,
+          cmd.getOptionValue(DiskBalancerCLI.PLAN)))) {
         planStream.write(plan.toJson().getBytes(StandardCharsets.UTF_8));
         planStream.write(plan.toJson().getBytes(StandardCharsets.UTF_8));
       }
       }
     } else {
     } else {
       LOG.info("No plan generated. DiskBalancing not needed for node: {} " +
       LOG.info("No plan generated. DiskBalancing not needed for node: {} " +
-              "threshold used: {}", cmd.getOptionValue(DiskBalancer.PLAN),
+              "threshold used: {}", cmd.getOptionValue(DiskBalancerCLI.PLAN),
           this.thresholdPercentage);
           this.thresholdPercentage);
     }
     }
 
 
-    if (cmd.hasOption(DiskBalancer.VERBOSE) && plans.size() > 0) {
+    if (cmd.hasOption(DiskBalancerCLI.VERBOSE) && plans.size() > 0) {
       printToScreen(plans);
       printToScreen(plans);
     }
     }
   }
   }
@@ -162,8 +163,8 @@ public class PlanCommand extends Command {
         " will balance the data.";
         " will balance the data.";
 
 
     HelpFormatter helpFormatter = new HelpFormatter();
     HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp("hdfs diskbalancer -plan " +
-        "<hostname> [options]", header, DiskBalancer.getPlanOptions(), footer);
+    helpFormatter.printHelp("hdfs diskbalancer -plan <hostname> [options]",
+        header, DiskBalancerCLI.getPlanOptions(), footer);
   }
   }
 
 
   /**
   /**
@@ -174,8 +175,8 @@ public class PlanCommand extends Command {
    */
    */
   private double getThresholdPercentage(CommandLine cmd) {
   private double getThresholdPercentage(CommandLine cmd) {
     Double value = 0.0;
     Double value = 0.0;
-    if (cmd.hasOption(DiskBalancer.THRESHOLD)) {
-      value = Double.parseDouble(cmd.getOptionValue(DiskBalancer.THRESHOLD));
+    if (cmd.hasOption(DiskBalancerCLI.THRESHOLD)) {
+      value = Double.parseDouble(cmd.getOptionValue(DiskBalancerCLI.THRESHOLD));
     }
     }
 
 
     if ((value <= 0.0) || (value > 100.0)) {
     if ((value <= 0.0) || (value > 100.0)) {

+ 10 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
 import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 
 
 /**
 /**
@@ -42,9 +42,10 @@ public class QueryCommand extends Command {
    */
    */
   public QueryCommand(Configuration conf) {
   public QueryCommand(Configuration conf) {
     super(conf);
     super(conf);
-    addValidCommandParameters(DiskBalancer.QUERY, "Queries the status of disk" +
-        " plan running on a given datanode.");
-    addValidCommandParameters(DiskBalancer.VERBOSE, "Prints verbose results.");
+    addValidCommandParameters(DiskBalancerCLI.QUERY,
+        "Queries the status of disk plan running on a given datanode.");
+    addValidCommandParameters(DiskBalancerCLI.VERBOSE,
+        "Prints verbose results.");
   }
   }
 
 
   /**
   /**
@@ -55,9 +56,9 @@ public class QueryCommand extends Command {
   @Override
   @Override
   public void execute(CommandLine cmd) throws Exception {
   public void execute(CommandLine cmd) throws Exception {
     LOG.info("Executing \"query plan\" command.");
     LOG.info("Executing \"query plan\" command.");
-    Preconditions.checkState(cmd.hasOption(DiskBalancer.QUERY));
-    verifyCommandOptions(DiskBalancer.QUERY, cmd);
-    String nodeName = cmd.getOptionValue(DiskBalancer.QUERY);
+    Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.QUERY));
+    verifyCommandOptions(DiskBalancerCLI.QUERY, cmd);
+    String nodeName = cmd.getOptionValue(DiskBalancerCLI.QUERY);
     Preconditions.checkNotNull(nodeName);
     Preconditions.checkNotNull(nodeName);
     nodeName = nodeName.trim();
     nodeName = nodeName.trim();
     String nodeAddress = nodeName;
     String nodeAddress = nodeName;
@@ -79,7 +80,7 @@ public class QueryCommand extends Command {
               workStatus.getPlanID(),
               workStatus.getPlanID(),
               workStatus.getResult().toString());
               workStatus.getResult().toString());
 
 
-      if (cmd.hasOption(DiskBalancer.VERBOSE)) {
+      if (cmd.hasOption(DiskBalancerCLI.VERBOSE)) {
         System.out.printf("%s", workStatus.currentStateString());
         System.out.printf("%s", workStatus.currentStateString());
       }
       }
     } catch (DiskBalancerException ex) {
     } catch (DiskBalancerException ex) {
@@ -101,6 +102,6 @@ public class QueryCommand extends Command {
 
 
     HelpFormatter helpFormatter = new HelpFormatter();
     HelpFormatter helpFormatter = new HelpFormatter();
     helpFormatter.printHelp("hdfs diskbalancer -query <hostname>  [options]",
     helpFormatter.printHelp("hdfs diskbalancer -query <hostname>  [options]",
-        header, DiskBalancer.getQueryOptions(), footer);
+        header, DiskBalancerCLI.getQueryOptions(), footer);
   }
   }
 }
 }

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java

@@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 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.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
@@ -52,15 +52,15 @@ public class ReportCommand extends Command {
     super(conf);
     super(conf);
     this.out = out;
     this.out = out;
 
 
-    addValidCommandParameters(DiskBalancer.REPORT,
+    addValidCommandParameters(DiskBalancerCLI.REPORT,
         "Report volume information of nodes.");
         "Report volume information of nodes.");
 
 
     String desc = String.format(
     String desc = String.format(
         "Top number of nodes to be processed. Default: %d", getDefaultTop());
         "Top number of nodes to be processed. Default: %d", getDefaultTop());
-    addValidCommandParameters(DiskBalancer.TOP, desc);
+    addValidCommandParameters(DiskBalancerCLI.TOP, desc);
 
 
     desc = String.format("Print out volume information for a DataNode.");
     desc = String.format("Print out volume information for a DataNode.");
-    addValidCommandParameters(DiskBalancer.NODE, desc);
+    addValidCommandParameters(DiskBalancerCLI.NODE, desc);
   }
   }
 
 
   @Override
   @Override
@@ -69,8 +69,8 @@ public class ReportCommand extends Command {
     String outputLine = "Processing report command";
     String outputLine = "Processing report command";
     recordOutput(result, outputLine);
     recordOutput(result, outputLine);
 
 
-    Preconditions.checkState(cmd.hasOption(DiskBalancer.REPORT));
-    verifyCommandOptions(DiskBalancer.REPORT, cmd);
+    Preconditions.checkState(cmd.hasOption(DiskBalancerCLI.REPORT));
+    verifyCommandOptions(DiskBalancerCLI.REPORT, cmd);
     readClusterInfo(cmd);
     readClusterInfo(cmd);
 
 
     final String nodeFormat =
     final String nodeFormat =
@@ -81,7 +81,7 @@ public class ReportCommand extends Command {
         "[%s: volume-%s] - %.2f used: %d/%d, %.2f free: %d/%d, "
         "[%s: volume-%s] - %.2f used: %d/%d, %.2f free: %d/%d, "
         + "isFailed: %s, isReadOnly: %s, isSkip: %s, isTransient: %s.";
         + "isFailed: %s, isReadOnly: %s, isSkip: %s, isTransient: %s.";
 
 
-    if (cmd.hasOption(DiskBalancer.NODE)) {
+    if (cmd.hasOption(DiskBalancerCLI.NODE)) {
       /*
       /*
        * Reporting volume information for a specific DataNode
        * Reporting volume information for a specific DataNode
        */
        */
@@ -136,7 +136,7 @@ public class ReportCommand extends Command {
      * get value that identifies a DataNode from command line, it could be UUID,
      * get value that identifies a DataNode from command line, it could be UUID,
      * IP address or host name.
      * IP address or host name.
      */
      */
-    final String nodeVal = cmd.getOptionValue(DiskBalancer.NODE);
+    final String nodeVal = cmd.getOptionValue(DiskBalancerCLI.NODE);
 
 
     if (StringUtils.isBlank(nodeVal)) {
     if (StringUtils.isBlank(nodeVal)) {
       outputLine = "The value for '-node' is neither specified or empty.";
       outputLine = "The value for '-node' is neither specified or empty.";
@@ -211,6 +211,6 @@ public class ReportCommand extends Command {
     HelpFormatter helpFormatter = new HelpFormatter();
     HelpFormatter helpFormatter = new HelpFormatter();
     helpFormatter.printHelp("hdfs diskbalancer -fs http://namenode.uri " +
     helpFormatter.printHelp("hdfs diskbalancer -fs http://namenode.uri " +
         "-report [options]",
         "-report [options]",
-        header, DiskBalancer.getReportOptions(), footer);
+        header, DiskBalancerCLI.getReportOptions(), footer);
   }
   }
 }
 }

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancerCLI.java

@@ -50,7 +50,7 @@ import java.io.PrintStream;
  * At very high level diskbalancer computes a set of moves that will make disk
  * At very high level diskbalancer computes a set of moves that will make disk
  * utilization equal and then those moves are executed by the datanode.
  * utilization equal and then those moves are executed by the datanode.
  */
  */
-public class DiskBalancer extends Configured implements Tool {
+public class DiskBalancerCLI extends Configured implements Tool {
   /**
   /**
    * Computes a plan for a given set of nodes.
    * Computes a plan for a given set of nodes.
    */
    */
@@ -126,7 +126,7 @@ public class DiskBalancer extends Configured implements Tool {
    */
    */
   public static final String PLAN_TEMPLATE = "%s.plan.json";
   public static final String PLAN_TEMPLATE = "%s.plan.json";
   private static final Logger LOG =
   private static final Logger LOG =
-      LoggerFactory.getLogger(DiskBalancer.class);
+      LoggerFactory.getLogger(DiskBalancerCLI.class);
 
 
   private static final Options PLAN_OPTIONS = new Options();
   private static final Options PLAN_OPTIONS = new Options();
   private static final Options EXECUTE_OPTIONS = new Options();
   private static final Options EXECUTE_OPTIONS = new Options();
@@ -140,7 +140,7 @@ public class DiskBalancer extends Configured implements Tool {
    *
    *
    * @param conf
    * @param conf
    */
    */
-  public DiskBalancer(Configuration conf) {
+  public DiskBalancerCLI(Configuration conf) {
     super(conf);
     super(conf);
   }
   }
 
 
@@ -151,7 +151,7 @@ public class DiskBalancer extends Configured implements Tool {
    * @throws Exception
    * @throws Exception
    */
    */
   public static void main(String[] argv) throws Exception {
   public static void main(String[] argv) throws Exception {
-    DiskBalancer shell = new DiskBalancer(new HdfsConfiguration());
+    DiskBalancerCLI shell = new DiskBalancerCLI(new HdfsConfiguration());
     int res = 0;
     int res = 0;
     try {
     try {
       res = ToolRunner.run(shell, argv);
       res = ToolRunner.run(shell, argv);
@@ -446,27 +446,27 @@ public class DiskBalancer extends Configured implements Tool {
   private int dispatch(CommandLine cmd, Options opts, final PrintStream out)
   private int dispatch(CommandLine cmd, Options opts, final PrintStream out)
       throws Exception {
       throws Exception {
     Command currentCommand = null;
     Command currentCommand = null;
-    if (cmd.hasOption(DiskBalancer.PLAN)) {
+    if (cmd.hasOption(DiskBalancerCLI.PLAN)) {
       currentCommand = new PlanCommand(getConf());
       currentCommand = new PlanCommand(getConf());
     }
     }
 
 
-    if (cmd.hasOption(DiskBalancer.EXECUTE)) {
+    if (cmd.hasOption(DiskBalancerCLI.EXECUTE)) {
       currentCommand = new ExecuteCommand(getConf());
       currentCommand = new ExecuteCommand(getConf());
     }
     }
 
 
-    if (cmd.hasOption(DiskBalancer.QUERY)) {
+    if (cmd.hasOption(DiskBalancerCLI.QUERY)) {
       currentCommand = new QueryCommand(getConf());
       currentCommand = new QueryCommand(getConf());
     }
     }
 
 
-    if (cmd.hasOption(DiskBalancer.CANCEL)) {
+    if (cmd.hasOption(DiskBalancerCLI.CANCEL)) {
       currentCommand = new CancelCommand(getConf());
       currentCommand = new CancelCommand(getConf());
     }
     }
 
 
-    if (cmd.hasOption(DiskBalancer.REPORT)) {
+    if (cmd.hasOption(DiskBalancerCLI.REPORT)) {
       currentCommand = new ReportCommand(getConf(), out);
       currentCommand = new ReportCommand(getConf(), out);
     }
     }
 
 
-    if (cmd.hasOption(DiskBalancer.HELP)) {
+    if (cmd.hasOption(DiskBalancerCLI.HELP)) {
       currentCommand = new HelpCommand(getConf());
       currentCommand = new HelpCommand(getConf());
     }
     }
 
 

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java

@@ -41,18 +41,19 @@ import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 
 
-import static org.apache.hadoop.hdfs.tools.DiskBalancer.CANCEL;
-import static org.apache.hadoop.hdfs.tools.DiskBalancer.HELP;
-import static org.apache.hadoop.hdfs.tools.DiskBalancer.NODE;
-import static org.apache.hadoop.hdfs.tools.DiskBalancer.PLAN;
-import static org.apache.hadoop.hdfs.tools.DiskBalancer.QUERY;
-import static org.apache.hadoop.hdfs.tools.DiskBalancer.REPORT;
+import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.CANCEL;
+import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.HELP;
+import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.NODE;
+import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.PLAN;
+import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.QUERY;
+import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.REPORT;
 
 
 import org.junit.Rule;
 import org.junit.Rule;
 import org.junit.rules.ExpectedException;
 import org.junit.rules.ExpectedException;
@@ -387,8 +388,7 @@ public class TestDiskBalancerCommand {
   private List<String> runCommandInternal(final String cmdLine) throws
   private List<String> runCommandInternal(final String cmdLine) throws
       Exception {
       Exception {
     String[] cmds = StringUtils.split(cmdLine, ' ');
     String[] cmds = StringUtils.split(cmdLine, ' ');
-    org.apache.hadoop.hdfs.tools.DiskBalancer db =
-        new org.apache.hadoop.hdfs.tools.DiskBalancer(conf);
+    DiskBalancerCLI db = new DiskBalancerCLI(conf);
 
 
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     PrintStream out = new PrintStream(bufOut);
     PrintStream out = new PrintStream(bufOut);