瀏覽代碼

HDFS-11358. DiskBalancer: Report command supports reading nodes from host file. Contributed by Yiqun Lin.

Yiqun Lin 8 年之前
父節點
當前提交
3b908f71c5

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java

@@ -41,6 +41,7 @@ public class DiskBalancerException extends IOException {
     UNKNOWN_KEY,
     INVALID_NODE,
     DATANODE_STATUS_NOT_REGULAR,
+    INVALID_HOST_FILE_PATH,
   }
 
   private final Result result;

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

@@ -49,17 +49,17 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSe
 import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.HostsFileReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.Closeable;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URL;
-import java.nio.charset.Charset;
-import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.text.SimpleDateFormat;
 import java.util.Collections;
@@ -268,16 +268,33 @@ public abstract class Command extends Configured implements Closeable {
     if ((listArg == null) || listArg.isEmpty()) {
       return resultSet;
     }
+
     if (listArg.startsWith("file://")) {
       listURL = new URL(listArg);
-      byte[] data = Files.readAllBytes(Paths.get(listURL.getPath()));
-      nodeData = new String(data, Charset.forName("UTF-8"));
+      try {
+        HostsFileReader.readFileToSet("include",
+            Paths.get(listURL.getPath()).toString(), resultSet);
+      } catch (FileNotFoundException e) {
+        String warnMsg = String
+            .format("The input host file path '%s' is not a valid path. "
+                + "Please make sure the host file exists.", listArg);
+        throw new DiskBalancerException(warnMsg,
+            DiskBalancerException.Result.INVALID_HOST_FILE_PATH);
+      }
     } else {
       nodeData = listArg;
+      String[] nodes = nodeData.split(",");
+
+      if (nodes.length == 0) {
+        String warnMsg = "The number of input nodes is 0. "
+            + "Please input the valid nodes.";
+        throw new DiskBalancerException(warnMsg,
+            DiskBalancerException.Result.INVALID_NODE);
+      }
+
+      Collections.addAll(resultSet, nodes);
     }
 
-    String[] nodes = nodeData.split(",");
-    Collections.addAll(resultSet, nodes);
     return resultSet;
   }
 

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

@@ -148,7 +148,9 @@ public class ReportCommand extends Command {
        * Reporting volume information for specific DataNode(s)
        */
       outputLine = String.format(
-          "Reporting volume information for DataNode(s) '%s'.", nodeVal);
+          "Reporting volume information for DataNode(s). "
+          + "These DataNode(s) are parsed from '%s'.", nodeVal);
+
       recordOutput(result, outputLine);
 
       List<DiskBalancerDataNode> dbdns = Lists.newArrayList();
@@ -224,7 +226,7 @@ public class ReportCommand extends Command {
         + "hdfs diskbalancer -report\n"
         + "hdfs diskbalancer -report -top 5\n"
         + "hdfs diskbalancer -report "
-        + "-node [<DataNodeID|IP|Hostname>,...]";
+        + "-node <file://> | [<DataNodeID|IP|Hostname>,...]";
 
     HelpFormatter helpFormatter = new HelpFormatter();
     helpFormatter.printHelp("hdfs diskbalancer -fs http://namenode.uri " +

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -418,7 +418,7 @@ Usage:
          [-query <datanode>]
          [-cancel <planfile>]
          [-cancel <planID> -node <datanode>]
-         [-report -node [<DataNodeID|IP|Hostname>,...]]
+         [-report -node <file://> | [<DataNodeID|IP|Hostname>,...]]
          [-report -node -top <topnum>]
 
 | COMMAND\_OPTION | Description |

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md

@@ -102,9 +102,9 @@ or
 Plan ID can be read from datanode using query command.
 
 ### Report
-Report command provides detailed report of specified node(s) or top nodes that will benefit from running disk balancer.
+Report command provides detailed report of specified node(s) or top nodes that will benefit from running disk balancer. The node(s) can be specified by a host file or comma-separated list of nodes.
 
-`hdfs diskbalancer -fs http://namenode.uri -report -node [<DataNodeID|IP|Hostname>,...]`
+`hdfs diskbalancer -fs http://namenode.uri -report -node <file://> | [<DataNodeID|IP|Hostname>,...]`
 
 or
 

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

@@ -34,6 +34,8 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileWriter;
 import java.io.PrintStream;
 import java.net.URI;
 import java.util.List;
@@ -675,14 +677,18 @@ public class TestDiskBalancerCommand {
         REPORT, NODE, dataNodeUuid1, dataNodeUuid2);
     final String cmdLine = String.format("hdfs diskbalancer %s", planArg);
     List<String> outputs = runCommand(cmdLine, cluster);
+    verifyOutputsOfReportCommand(outputs, dataNodeUuid1, dataNodeUuid2, true);
+  }
+
+  private void verifyOutputsOfReportCommand(List<String> outputs,
+      String dataNodeUuid1, String dataNodeUuid2, boolean inputNodesStr) {
+    assertThat(outputs.get(0), containsString("Processing report command"));
+    if (inputNodesStr) {
+      assertThat(outputs.get(1),
+          is(allOf(containsString("Reporting volume information for DataNode"),
+              containsString(dataNodeUuid1), containsString(dataNodeUuid2))));
+    }
 
-    assertThat(
-        outputs.get(0),
-        containsString("Processing report command"));
-    assertThat(
-        outputs.get(1),
-        is(allOf(containsString("Reporting volume information for DataNode"),
-            containsString(dataNodeUuid1), containsString(dataNodeUuid2))));
     // Since the order of input nodes will be disrupted when parse
     // the node string, we should compare UUID with both output lines.
     assertTrue(outputs.get(2).contains(dataNodeUuid1)
@@ -714,4 +720,55 @@ public class TestDiskBalancerCommand {
             , invalidNode, invalidNode);
     assertTrue(outputs.get(2).contains(invalidNodeInfo));
   }
+
+  @Test(timeout = 60000)
+  public void testReportCommandWithNullNodes() throws Exception {
+    // don't input nodes
+    final String planArg = String.format("-%s -%s ,", REPORT, NODE);
+    final String cmdLine = String.format("hdfs diskbalancer %s", planArg);
+    List<String> outputs = runCommand(cmdLine, cluster);
+
+    String invalidNodeInfo = "The number of input nodes is 0. "
+        + "Please input the valid nodes.";
+    assertTrue(outputs.get(2).contains(invalidNodeInfo));
+  }
+
+  @Test(timeout = 60000)
+  public void testReportCommandWithReadingHostFile() throws Exception {
+    final String testDir = GenericTestUtils.getTestDir().getAbsolutePath();
+    File includeFile = new File(testDir, "diskbalancer.include");
+    String filePath = testDir + "/diskbalancer.include";
+
+    String dataNodeUuid1 = cluster.getDataNodes().get(0).getDatanodeUuid();
+    String dataNodeUuid2 = cluster.getDataNodes().get(1).getDatanodeUuid();
+
+    FileWriter fw = new FileWriter(filePath);
+    fw.write("#This-is-comment\n");
+    fw.write(dataNodeUuid1 + "\n");
+    fw.write(dataNodeUuid2 + "\n");
+    fw.close();
+
+    final String planArg = String.format("-%s -%s file://%s",
+        REPORT, NODE, filePath);
+    final String cmdLine = String.format("hdfs diskbalancer %s", planArg);
+    List<String> outputs = runCommand(cmdLine, cluster);
+
+    verifyOutputsOfReportCommand(outputs, dataNodeUuid1, dataNodeUuid2, false);
+    includeFile.delete();
+  }
+
+  @Test(timeout = 60000)
+  public void testReportCommandWithInvalidHostFilePath() throws Exception {
+    final String testDir = GenericTestUtils.getTestDir().getAbsolutePath();
+    String invalidFilePath = testDir + "/diskbalancer-invalid.include";
+
+    final String planArg = String.format("-%s -%s file://%s",
+        REPORT, NODE, invalidFilePath);
+    final String cmdLine = String.format("hdfs diskbalancer %s", planArg);
+    List<String> outputs = runCommand(cmdLine, cluster);
+
+    String invalidNodeInfo = String.format(
+        "The input host file path 'file://%s' is not a valid path.", invalidFilePath);
+    assertTrue(outputs.get(2).contains(invalidNodeInfo));
+  }
 }