Browse Source

HDFS-10821. DiskBalancer: Report command support with multiple nodes. Contributed by Yiqun Lin.

Anu Engineer 8 years ago
parent
commit
8a93f45a80

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

@@ -38,7 +38,8 @@ public class DiskBalancerException extends IOException {
     INVALID_MOVE,
     INTERNAL_ERROR,
     NO_SUCH_PLAN,
-    UNKNOWN_KEY
+    UNKNOWN_KEY,
+    INVALID_NODE,
   }
 
   private final Result result;

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

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 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.datamodel.DiskBalancerCluster;
@@ -256,6 +257,7 @@ public abstract class Command extends Configured {
       throws IOException {
     Set<String> nodeNames = null;
     List<DiskBalancerDataNode> nodeList = Lists.newArrayList();
+    List<String> invalidNodeList = Lists.newArrayList();
 
     if ((listArg == null) || listArg.isEmpty()) {
       return nodeList;
@@ -269,10 +271,22 @@ public abstract class Command extends Configured {
 
         if (node != null) {
           nodeList.add(node);
+        } else {
+          invalidNodeList.add(name);
         }
       }
     }
 
+    if (!invalidNodeList.isEmpty()) {
+      String invalidNodes = StringUtils.join(invalidNodeList.toArray(), ",");
+      String warnMsg = String.format(
+          "The node(s) '%s' not found. "
+          + "Please make sure that '%s' exists in the cluster.",
+          invalidNodes, invalidNodes);
+      throw new DiskBalancerException(warnMsg,
+          DiskBalancerException.Result.INVALID_NODE);
+    }
+
     return nodeList;
   }
 

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

@@ -27,6 +27,7 @@ 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;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 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;
@@ -59,7 +60,7 @@ public class ReportCommand extends Command {
         "Top number of nodes to be processed. Default: %d", getDefaultTop());
     addValidCommandParameters(DiskBalancerCLI.TOP, desc);
 
-    desc = String.format("Print out volume information for a DataNode.");
+    desc = String.format("Print out volume information for DataNode(s).");
     addValidCommandParameters(DiskBalancerCLI.NODE, desc);
   }
 
@@ -83,7 +84,7 @@ public class ReportCommand extends Command {
 
     if (cmd.hasOption(DiskBalancerCLI.NODE)) {
       /*
-       * Reporting volume information for a specific DataNode
+       * Reporting volume information for specific DataNode(s)
        */
       handleNodeReport(cmd, result, nodeFormatWithoutSequence, volumeFormat);
 
@@ -133,8 +134,8 @@ public class ReportCommand extends Command {
       final String nodeFormat, final String volumeFormat) throws Exception {
     String outputLine = "";
     /*
-     * get value that identifies a DataNode from command line, it could be UUID,
-     * IP address or host name.
+     * get value that identifies DataNode(s) from command line, it could be
+     * UUID, IP address or host name.
      */
     final String nodeVal = cmd.getOptionValue(DiskBalancerCLI.NODE);
 
@@ -143,55 +144,71 @@ public class ReportCommand extends Command {
       recordOutput(result, outputLine);
     } else {
       /*
-       * Reporting volume information for a specific DataNode
+       * Reporting volume information for specific DataNode(s)
        */
       outputLine = String.format(
-          "Reporting volume information for DataNode '%s'.", nodeVal);
+          "Reporting volume information for DataNode(s) '%s'.", nodeVal);
       recordOutput(result, outputLine);
 
-      final String trueStr = "True";
-      final String falseStr = "False";
-      DiskBalancerDataNode dbdn = getNode(nodeVal);
-      // get storage path of datanode
-      populatePathNames(dbdn);
-
-      if (dbdn == null) {
-        outputLine = String.format(
-            "Can't find a DataNode that matches '%s'.", nodeVal);
-        recordOutput(result, outputLine);
-      } else {
-        result.appendln(String.format(nodeFormat,
-            dbdn.getDataNodeName(),
-            dbdn.getDataNodeIP(),
-            dbdn.getDataNodePort(),
-            dbdn.getDataNodeUUID(),
-            dbdn.getVolumeCount(),
-            dbdn.getNodeDataDensity()));
-
-        List<String> volumeList = Lists.newArrayList();
-        for (DiskBalancerVolumeSet vset : dbdn.getVolumeSets().values()) {
-          for (DiskBalancerVolume vol : vset.getVolumes()) {
-            volumeList.add(String.format(volumeFormat,
-                vol.getStorageType(),
-                vol.getPath(),
-                vol.getUsedRatio(),
-                vol.getUsed(),
-                vol.getCapacity(),
-                vol.getFreeRatio(),
-                vol.getFreeSpace(),
-                vol.getCapacity(),
-                vol.isFailed() ? trueStr : falseStr,
-                vol.isReadOnly() ? trueStr : falseStr,
-                vol.isSkip() ? trueStr : falseStr,
-                vol.isTransient() ? trueStr : falseStr));
-          }
+      List<DiskBalancerDataNode> dbdns = Lists.newArrayList();
+      try {
+        dbdns = getNodes(nodeVal);
+      } catch (DiskBalancerException e) {
+        // If there are some invalid nodes that contained in nodeVal,
+        // the exception will be threw.
+        recordOutput(result, e.getMessage());
+        return;
+      }
+
+      if (!dbdns.isEmpty()) {
+        for (DiskBalancerDataNode node : dbdns) {
+          recordNodeReport(result, node, nodeFormat, volumeFormat);
+          result.append(System.lineSeparator());
         }
+      }
+    }
+  }
 
-        Collections.sort(volumeList);
-        result.appendln(
-            StringUtils.join(volumeList.toArray(), System.lineSeparator()));
+  /**
+   * Put node report lines to string buffer.
+   */
+  private void recordNodeReport(StrBuilder result, DiskBalancerDataNode dbdn,
+      final String nodeFormat, final String volumeFormat) throws Exception {
+    final String trueStr = "True";
+    final String falseStr = "False";
+
+    // get storage path of datanode
+    populatePathNames(dbdn);
+    result.appendln(String.format(nodeFormat,
+        dbdn.getDataNodeName(),
+        dbdn.getDataNodeIP(),
+        dbdn.getDataNodePort(),
+        dbdn.getDataNodeUUID(),
+        dbdn.getVolumeCount(),
+        dbdn.getNodeDataDensity()));
+
+    List<String> volumeList = Lists.newArrayList();
+    for (DiskBalancerVolumeSet vset : dbdn.getVolumeSets().values()) {
+      for (DiskBalancerVolume vol : vset.getVolumes()) {
+        volumeList.add(String.format(volumeFormat,
+            vol.getStorageType(),
+            vol.getPath(),
+            vol.getUsedRatio(),
+            vol.getUsed(),
+            vol.getCapacity(),
+            vol.getFreeRatio(),
+            vol.getFreeSpace(),
+            vol.getCapacity(),
+            vol.isFailed() ? trueStr : falseStr,
+            vol.isReadOnly() ? trueStr: falseStr,
+            vol.isSkip() ? trueStr : falseStr,
+            vol.isTransient() ? trueStr : falseStr));
       }
     }
+
+    Collections.sort(volumeList);
+    result.appendln(
+        StringUtils.join(volumeList.toArray(), System.lineSeparator()));
   }
 
   /**
@@ -199,14 +216,14 @@ public class ReportCommand extends Command {
    */
   @Override
   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 header = "Report command reports the volume information of given" +
+        " datanode(s), 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 -report\n"
         + "hdfs diskbalancer -report -top 5\n"
         + "hdfs diskbalancer -report "
-        + "-node {DataNodeID | IP | Hostname}";
+        + "-node [<DataNodeID|IP|Hostname>,...]";
 
     HelpFormatter helpFormatter = new HelpFormatter();
     helpFormatter.printHelp("hdfs diskbalancer -fs http://namenode.uri " +

+ 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 about a node.
+Report command provides detailed report about node(s).
 
-`hdfs diskbalancer -fs http://namenode.uri -report -node {DataNodeID | IP | Hostname}`
+`hdfs diskbalancer -fs http://namenode.uri -report -node [<DataNodeID|IP|Hostname>,...]`
 
 
 Settings

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

@@ -23,6 +23,7 @@ import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
@@ -457,4 +458,52 @@ public class TestDiskBalancerCommand {
     List<DiskBalancerDataNode> nodeList = command.getNodes(listArg.toString());
     assertEquals(nodeNum, nodeList.size());
   }
+
+  @Test(timeout = 60000)
+  public void testReportCommandWithMultipleNodes() throws Exception {
+    String dataNodeUuid1 = cluster.getDataNodes().get(0).getDatanodeUuid();
+    String dataNodeUuid2 = cluster.getDataNodes().get(1).getDatanodeUuid();
+    final String planArg = String.format("-%s -%s %s,%s",
+        REPORT, NODE, dataNodeUuid1, dataNodeUuid2);
+    final String cmdLine = String.format("hdfs diskbalancer %s", planArg);
+    List<String> outputs = runCommand(cmdLine, cluster);
+
+    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)
+        || outputs.get(6).contains(dataNodeUuid1));
+    assertTrue(outputs.get(2).contains(dataNodeUuid2)
+        || outputs.get(6).contains(dataNodeUuid2));
+  }
+
+  @Test(timeout = 60000)
+  public void testReportCommandWithInvalidNode() throws Exception {
+    String dataNodeUuid1 = cluster.getDataNodes().get(0).getDatanodeUuid();
+    String invalidNode = "invalidNode";
+    final String planArg = String.format("-%s -%s %s,%s",
+        REPORT, NODE, dataNodeUuid1, invalidNode);
+    final String cmdLine = String.format("hdfs diskbalancer %s", planArg);
+    List<String> outputs = runCommand(cmdLine, cluster);
+
+    assertThat(
+        outputs.get(0),
+        containsString("Processing report command"));
+    assertThat(
+        outputs.get(1),
+        is(allOf(containsString("Reporting volume information for DataNode"),
+            containsString(dataNodeUuid1), containsString(invalidNode))));
+
+    String invalidNodeInfo =
+        String.format("The node(s) '%s' not found. "
+            + "Please make sure that '%s' exists in the cluster."
+            , invalidNode, invalidNode);
+    assertTrue(outputs.get(2).contains(invalidNodeInfo));
+  }
 }