瀏覽代碼

HDFS-10599. DiskBalancer: Execute CLI via Shell. Contributed by Manoj Govindassamy.

Anu Engineer 8 年之前
父節點
當前提交
e3f7f58a5f

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

@@ -135,13 +135,20 @@ public class DiskBalancerCLI extends Configured implements Tool {
   private static final Options CANCEL_OPTIONS = new Options();
   private static final Options CANCEL_OPTIONS = new Options();
   private static final Options REPORT_OPTIONS = new Options();
   private static final Options REPORT_OPTIONS = new Options();
 
 
+  private final PrintStream printStream;
+
   /**
   /**
    * Construct a DiskBalancer.
    * Construct a DiskBalancer.
    *
    *
    * @param conf
    * @param conf
    */
    */
   public DiskBalancerCLI(Configuration conf) {
   public DiskBalancerCLI(Configuration conf) {
+    this(conf, System.out);
+  }
+
+  public DiskBalancerCLI(Configuration conf, final PrintStream printStream) {
     super(conf);
     super(conf);
+    this.printStream = printStream;
   }
   }
 
 
   /**
   /**
@@ -171,21 +178,9 @@ public class DiskBalancerCLI extends Configured implements Tool {
    */
    */
   @Override
   @Override
   public int run(String[] args) throws Exception {
   public int run(String[] args) throws Exception {
-    return run(args, System.out);
-  }
-
-  /**
-   * Execute the command with the given arguments.
-   *
-   * @param args command specific arguments.
-   * @param out  the output stream used for printing
-   * @return exit code.
-   * @throws Exception
-   */
-  public int run(String[] args, final PrintStream out) throws Exception {
     Options opts = getOpts();
     Options opts = getOpts();
     CommandLine cmd = parseArgs(args, opts);
     CommandLine cmd = parseArgs(args, opts);
-    return dispatch(cmd, opts, out);
+    return dispatch(cmd, opts);
   }
   }
 
 
   /**
   /**
@@ -443,7 +438,7 @@ public class DiskBalancerCLI extends Configured implements Tool {
    * @param opts options of command line
    * @param opts options of command line
    * @param out  the output stream used for printing
    * @param out  the output stream used for printing
    */
    */
-  private int dispatch(CommandLine cmd, Options opts, final PrintStream out)
+  private int dispatch(CommandLine cmd, Options opts)
       throws Exception {
       throws Exception {
     Command currentCommand = null;
     Command currentCommand = null;
     if (cmd.hasOption(DiskBalancerCLI.PLAN)) {
     if (cmd.hasOption(DiskBalancerCLI.PLAN)) {
@@ -463,7 +458,7 @@ public class DiskBalancerCLI extends Configured implements Tool {
     }
     }
 
 
     if (cmd.hasOption(DiskBalancerCLI.REPORT)) {
     if (cmd.hasOption(DiskBalancerCLI.REPORT)) {
-      currentCommand = new ReportCommand(getConf(), out);
+      currentCommand = new ReportCommand(getConf(), this.printStream);
     }
     }
 
 
     if (cmd.hasOption(DiskBalancerCLI.HELP)) {
     if (cmd.hasOption(DiskBalancerCLI.HELP)) {

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

@@ -43,6 +43,8 @@ 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.apache.hadoop.hdfs.tools.DiskBalancerCLI;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -140,6 +142,27 @@ public class TestDiskBalancerCommand {
             containsString("9 volumes with node data density 1.97"))));
             containsString("9 volumes with node data density 1.97"))));
   }
   }
 
 
+  /**
+   * This test simulates DiskBalancerCLI Report command run from a shell
+   * with a generic option 'fs'.
+   * @throws Exception
+   */
+  @Test(timeout = 60000)
+  public void testReportWithGenericOptionFS() throws Exception {
+    final String topReportArg = "5";
+    final String reportArgs = String.format("-%s file:%s -%s -%s %s",
+        "fs", clusterJson.getPath(),
+        REPORT, "top", topReportArg);
+    final String cmdLine = String.format("%s", reportArgs);
+    final List<String> outputs = runCommand(cmdLine);
+
+    assertThat(outputs.get(0), containsString("Processing report command"));
+    assertThat(outputs.get(1),
+        is(allOf(containsString("Reporting top"), containsString(topReportArg),
+            containsString(
+                "DataNode(s) benefiting from running DiskBalancer"))));
+  }
+
   /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
   /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
   @Test(timeout = 60000)
   @Test(timeout = 60000)
   public void testReportMoreThanTotal() throws Exception {
   public void testReportMoreThanTotal() throws Exception {
@@ -389,11 +412,11 @@ 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, ' ');
-    DiskBalancerCLI db = new DiskBalancerCLI(conf);
-
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     PrintStream out = new PrintStream(bufOut);
     PrintStream out = new PrintStream(bufOut);
-    db.run(cmds, out);
+
+    Tool diskBalancerTool = new DiskBalancerCLI(conf, out);
+    ToolRunner.run(conf, diskBalancerTool, cmds);
 
 
     Scanner scanner = new Scanner(bufOut.toString());
     Scanner scanner = new Scanner(bufOut.toString());
     List<String> outputs = Lists.newArrayList();
     List<String> outputs = Lists.newArrayList();