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 ae...@apache.org on 2016/09/09 02:38:28 UTC

[33/43] hadoop git commit: HDFS-10778. Add -format option to make the output of FileDistribution processor human-readable in OfflineImageViewer.

HDFS-10778. Add -format option to make the output of FileDistribution processor human-readable in OfflineImageViewer.


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

Branch: refs/heads/HDFS-7240
Commit: 63f594892ecd4687e37a99790288e36eb278849f
Parents: d355573
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Sep 8 15:13:43 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Sep 8 15:13:43 2016 +0900

----------------------------------------------------------------------
 .../FileDistributionCalculator.java             |  20 +++-
 .../FileDistributionVisitor.java                |  28 ++++-
 .../offlineImageViewer/OfflineImageViewer.java  | 116 ++++++++++---------
 .../OfflineImageViewerPB.java                   |  78 +++++++------
 .../src/site/markdown/HDFSCommands.md           |   1 +
 .../src/site/markdown/HdfsImageViewer.md        |   1 +
 .../TestOfflineImageViewer.java                 |  24 +++-
 7 files changed, 164 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63f59489/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
index 33ab641..71fb822 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionCalculator.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.util.LimitInputStream;
+import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.base.Preconditions;
 
@@ -75,11 +76,14 @@ final class FileDistributionCalculator {
   private long totalSpace;
   private long maxFileSize;
 
+  private boolean formatOutput = false;
+
   FileDistributionCalculator(Configuration conf, long maxSize, int steps,
-      PrintStream out) {
+      boolean formatOutput, PrintStream out) {
     this.conf = conf;
     this.maxSize = maxSize == 0 ? MAX_SIZE_DEFAULT : maxSize;
     this.steps = steps == 0 ? INTERVAL_DEFAULT : steps;
+    this.formatOutput = formatOutput;
     this.out = out;
     long numIntervals = this.maxSize / this.steps;
     // avoid OutOfMemoryError when allocating an array
@@ -148,10 +152,20 @@ final class FileDistributionCalculator {
 
   private void output() {
     // write the distribution into the output file
-    out.print("Size\tNumFiles\n");
+    out.print((formatOutput ? "Size Range" : "Size") + "\tNumFiles\n");
     for (int i = 0; i < distribution.length; i++) {
       if (distribution[i] != 0) {
-        out.print(((long) i * steps) + "\t" + distribution[i]);
+        if (formatOutput) {
+          out.print((i == 0 ? "[" : "(")
+              + StringUtils.byteDesc(((long) (i == 0 ? 0 : i - 1) * steps))
+              + ", "
+              + StringUtils.byteDesc((long)
+                  (i == distribution.length - 1 ? maxFileSize : i * steps))
+                  + "]\t" + distribution[i]);
+        } else {
+          out.print(((long) i * steps) + "\t" + distribution[i]);
+        }
+
         out.print('\n');
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63f59489/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java
index 1cef720..7dcc299 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 import java.io.IOException;
 import java.util.LinkedList;
 
+import org.apache.hadoop.util.StringUtils;
+
 /**
  * File size distribution visitor.
  * 
@@ -67,6 +69,7 @@ class FileDistributionVisitor extends TextWriterImageVisitor {
   private FileContext current;
 
   private boolean inInode = false;
+  private boolean formatOutput = false;
 
   /**
    * File or directory information.
@@ -78,12 +81,12 @@ class FileDistributionVisitor extends TextWriterImageVisitor {
     int replication;
   }
 
-  public FileDistributionVisitor(String filename,
-                                 long maxSize,
-                                 int step) throws IOException {
+  public FileDistributionVisitor(String filename, long maxSize, int step,
+      boolean formatOutput) throws IOException {
     super(filename, false);
     this.maxSize = (maxSize == 0 ? MAX_SIZE_DEFAULT : maxSize);
     this.step = (step == 0 ? INTERVAL_DEFAULT : step);
+    this.formatOutput = formatOutput;
     long numIntervals = this.maxSize / this.step;
     if(numIntervals >= Integer.MAX_VALUE)
       throw new IOException("Too many distribution intervals " + numIntervals);
@@ -113,9 +116,22 @@ class FileDistributionVisitor extends TextWriterImageVisitor {
 
   private void output() throws IOException {
     // write the distribution into the output file
-    write("Size\tNumFiles\n");
-    for(int i = 0; i < distribution.length; i++)
-      write(((long)i * step) + "\t" + distribution[i] + "\n");
+    write((formatOutput ? "Size Range" : "Size") + "\tNumFiles\n");
+    for (int i = 0; i < distribution.length; i++) {
+      if (distribution[i] > 0) {
+        if (formatOutput) {
+          write((i == 0 ? "[" : "(")
+              + StringUtils.byteDesc(((long) (i == 0 ? 0 : i - 1) * step))
+              + ", "
+              + StringUtils.byteDesc((long)
+                  (i == distribution.length - 1 ? maxFileSize : i * step))
+                  + "]\t"
+              + distribution[i] + "\n");
+        } else {
+          write(((long) i * step) + "\t" + distribution[i] + "\n");
+        }
+      }
+    }
     System.out.println("totalFiles = " + totalFiles);
     System.out.println("totalDirectories = " + totalDirectories);
     System.out.println("totalBlocks = " + totalBlocks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63f59489/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
index 770cde1..c542d90 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
@@ -46,61 +46,63 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.PositionTrackingIn
 public class OfflineImageViewer {
   public static final Log LOG = LogFactory.getLog(OfflineImageViewer.class);
   
-  private final static String usage = 
-    "Usage: bin/hdfs oiv_legacy [OPTIONS] -i INPUTFILE -o OUTPUTFILE\n" +
-    "Offline Image Viewer\n" + 
-    "View a Hadoop fsimage INPUTFILE using the specified PROCESSOR,\n" +
-    "saving the results in OUTPUTFILE.\n" +
-    "\n" +
-    "The oiv utility will attempt to parse correctly formed image files\n" +
-    "and will abort fail with mal-formed image files.\n" +
-    "\n" +
-    "The tool works offline and does not require a running cluster in\n" +
-    "order to process an image file.\n" +
-    "\n" +
-    "The following image processors are available:\n" +
-    "  * Ls: The default image processor generates an lsr-style listing\n" +
-    "    of the files in the namespace, with the same fields in the same\n" +
-    "    order.  Note that in order to correctly determine file sizes,\n" +
-    "    this formatter cannot skip blocks and will override the\n" +
-    "    -skipBlocks option.\n" +
-    "  * Indented: This processor enumerates over all of the elements in\n" +
-    "    the fsimage file, using levels of indentation to delineate\n" +
-    "    sections within the file.\n" +
-    "  * Delimited: Generate a text file with all of the elements common\n" +
-    "    to both inodes and inodes-under-construction, separated by a\n" +
-    "    delimiter. The default delimiter is \u0001, though this may be\n" +
-    "    changed via the -delimiter argument. This processor also overrides\n" +
-    "    the -skipBlocks option for the same reason as the Ls processor\n" +
-    "  * XML: This processor creates an XML document with all elements of\n" +
-    "    the fsimage enumerated, suitable for further analysis by XML\n" +
-    "    tools.\n" +
-    "  * FileDistribution: This processor analyzes the file size\n" +
-    "    distribution in the image.\n" +
-    "    -maxSize specifies the range [0, maxSize] of file sizes to be\n" +
-    "     analyzed (128GB by default).\n" +
-    "    -step defines the granularity of the distribution. (2MB by default)\n" +
-    "  * NameDistribution: This processor analyzes the file names\n" +
-    "    in the image and prints total number of file names and how frequently\n" +
-    "    file names are reused.\n" +
-    "\n" + 
-    "Required command line arguments:\n" +
-    "-i,--inputFile <arg>   FSImage file to process.\n" +
-    "-o,--outputFile <arg>  Name of output file. If the specified\n" +
-    "                       file exists, it will be overwritten.\n" +
-    "\n" + 
-    "Optional command line arguments:\n" +
-    "-p,--processor <arg>   Select which type of processor to apply\n" +
-    "                       against image file." +
-    " (Ls|XML|Delimited|Indented|FileDistribution).\n" +
-    "-h,--help              Display usage information and exit\n" +
-    "-printToScreen         For processors that write to a file, also\n" +
-    "                       output to screen. On large image files this\n" +
-    "                       will dramatically increase processing time.\n" +
-    "-skipBlocks            Skip inodes' blocks information. May\n" +
-    "                       significantly decrease output.\n" +
-    "                       (default = false).\n" +
-    "-delimiter <arg>       Delimiting string to use with Delimited processor\n";
+  private final static String usage =
+      "Usage: bin/hdfs oiv_legacy [OPTIONS] -i INPUTFILE -o OUTPUTFILE\n"
+          + "Offline Image Viewer\n"
+          + "View a Hadoop fsimage INPUTFILE using the specified PROCESSOR,\n"
+          + "saving the results in OUTPUTFILE.\n"
+          + "\n"
+          + "The oiv utility will attempt to parse correctly formed image files\n"
+          + "and will abort fail with mal-formed image files.\n"
+          + "\n"
+          + "The tool works offline and does not require a running cluster in\n"
+          + "order to process an image file.\n"
+          + "\n"
+          + "The following image processors are available:\n"
+          + "  * Ls: The default image processor generates an lsr-style listing\n"
+          + "    of the files in the namespace, with the same fields in the same\n"
+          + "    order.  Note that in order to correctly determine file sizes,\n"
+          + "    this formatter cannot skip blocks and will override the\n"
+          + "    -skipBlocks option.\n"
+          + "  * Indented: This processor enumerates over all of the elements in\n"
+          + "    the fsimage file, using levels of indentation to delineate\n"
+          + "    sections within the file.\n"
+          + "  * Delimited: Generate a text file with all of the elements common\n"
+          + "    to both inodes and inodes-under-construction, separated by a\n"
+          + "    delimiter. The default delimiter is \u0001, though this may be\n"
+          + "    changed via the -delimiter argument. This processor also overrides\n"
+          + "    the -skipBlocks option for the same reason as the Ls processor\n"
+          + "  * XML: This processor creates an XML document with all elements of\n"
+          + "    the fsimage enumerated, suitable for further analysis by XML\n"
+          + "    tools.\n"
+          + "  * FileDistribution: This processor analyzes the file size\n"
+          + "    distribution in the image.\n"
+          + "    -maxSize specifies the range [0, maxSize] of file sizes to be\n"
+          + "     analyzed (128GB by default).\n"
+          + "    -step defines the granularity of the distribution. (2MB by default)\n"
+          + "    -format formats the output result in a human-readable fashion\n"
+          + "     rather than a number of bytes. (false by default)\n"
+          + "  * NameDistribution: This processor analyzes the file names\n"
+          + "    in the image and prints total number of file names and how frequently\n"
+          + "    file names are reused.\n"
+          + "\n"
+          + "Required command line arguments:\n"
+          + "-i,--inputFile <arg>   FSImage file to process.\n"
+          + "-o,--outputFile <arg>  Name of output file. If the specified\n"
+          + "                       file exists, it will be overwritten.\n"
+          + "\n"
+          + "Optional command line arguments:\n"
+          + "-p,--processor <arg>   Select which type of processor to apply\n"
+          + "                       against image file."
+          + " (Ls|XML|Delimited|Indented|FileDistribution).\n"
+          + "-h,--help              Display usage information and exit\n"
+          + "-printToScreen         For processors that write to a file, also\n"
+          + "                       output to screen. On large image files this\n"
+          + "                       will dramatically increase processing time.\n"
+          + "-skipBlocks            Skip inodes' blocks information. May\n"
+          + "                       significantly decrease output.\n"
+          + "                       (default = false).\n"
+          + "-delimiter <arg>       Delimiting string to use with Delimited processor\n";
 
   private final boolean skipBlocks;
   private final String inputFile;
@@ -188,6 +190,7 @@ public class OfflineImageViewer {
     options.addOption("h", "help", false, "");
     options.addOption("maxSize", true, "");
     options.addOption("step", true, "");
+    options.addOption("format", false, "");
     options.addOption("skipBlocks", false, "");
     options.addOption("printToScreen", false, "");
     options.addOption("delimiter", true, "");
@@ -253,7 +256,8 @@ public class OfflineImageViewer {
     } else if (processor.equals("FileDistribution")) {
       long maxSize = Long.parseLong(cmd.getOptionValue("maxSize", "0"));
       int step = Integer.parseInt(cmd.getOptionValue("step", "0"));
-      v = new FileDistributionVisitor(outputFile, maxSize, step);
+      boolean formatOutput = cmd.hasOption("format");
+      v = new FileDistributionVisitor(outputFile, maxSize, step, formatOutput);
     } else if (processor.equals("NameDistribution")) {
       v = new NameDistributionVisitor(outputFile, printToScreen);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63f59489/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
index b514b3f..c1141f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
@@ -67,6 +67,8 @@ public class OfflineImageViewerPB {
       + "    -maxSize specifies the range [0, maxSize] of file sizes to be\n"
       + "     analyzed (128GB by default).\n"
       + "    -step defines the granularity of the distribution. (2MB by default)\n"
+      + "    -format formats the output result in a human-readable fashion\n"
+      + "     rather than a number of bytes. (false by default)\n"
       + "  * Web: Run a viewer to expose read-only WebHDFS API.\n"
       + "    -addr specifies the address to listen. (localhost:5978 by default)\n"
       + "  * Delimited (experimental): Generate a text file with all of the elements common\n"
@@ -111,6 +113,7 @@ public class OfflineImageViewerPB {
     options.addOption("h", "help", false, "");
     options.addOption("maxSize", true, "");
     options.addOption("step", true, "");
+    options.addOption("format", false, "");
     options.addOption("addr", true, "");
     options.addOption("delimiter", true, "");
     options.addOption("t", "temp", true, "");
@@ -172,43 +175,44 @@ public class OfflineImageViewerPB {
     try (PrintStream out = outputFile.equals("-") ?
         System.out : new PrintStream(outputFile, "UTF-8")) {
       switch (processor) {
-        case "FileDistribution":
-          long maxSize = Long.parseLong(cmd.getOptionValue("maxSize", "0"));
-          int step = Integer.parseInt(cmd.getOptionValue("step", "0"));
-          new FileDistributionCalculator(conf, maxSize, step, out).visit(
-              new RandomAccessFile(inputFile, "r"));
-          break;
-        case "XML":
-          new PBImageXmlWriter(conf, out).visit(
-              new RandomAccessFile(inputFile, "r"));
-          break;
-        case "ReverseXML":
-          try {
-            OfflineImageReconstructor.run(inputFile, outputFile);
-          } catch (Exception e) {
-            System.err.println("OfflineImageReconstructor failed: " +
-                e.getMessage());
-            e.printStackTrace(System.err);
-            System.exit(1);
-          }
-          break;
-        case "Web":
-          String addr = cmd.getOptionValue("addr", "localhost:5978");
-          try (WebImageViewer viewer = new WebImageViewer(
-              NetUtils.createSocketAddr(addr))) {
-            viewer.start(inputFile);
-          }
-          break;
-        case "Delimited":
-          try (PBImageDelimitedTextWriter writer =
-              new PBImageDelimitedTextWriter(out, delimiter, tempPath)) {
-            writer.visit(new RandomAccessFile(inputFile, "r"));
-          }
-          break;
-        default:
-          System.err.println("Invalid processor specified : " + processor);
-          printUsage();
-          return -1;
+      case "FileDistribution":
+        long maxSize = Long.parseLong(cmd.getOptionValue("maxSize", "0"));
+        int step = Integer.parseInt(cmd.getOptionValue("step", "0"));
+        boolean formatOutput = cmd.hasOption("format");
+        new FileDistributionCalculator(conf, maxSize, step, formatOutput, out)
+            .visit(new RandomAccessFile(inputFile, "r"));
+        break;
+      case "XML":
+        new PBImageXmlWriter(conf, out).visit(new RandomAccessFile(inputFile,
+            "r"));
+        break;
+      case "ReverseXML":
+        try {
+          OfflineImageReconstructor.run(inputFile, outputFile);
+        } catch (Exception e) {
+          System.err.println("OfflineImageReconstructor failed: "
+              + e.getMessage());
+          e.printStackTrace(System.err);
+          System.exit(1);
+        }
+        break;
+      case "Web":
+        String addr = cmd.getOptionValue("addr", "localhost:5978");
+        try (WebImageViewer viewer =
+            new WebImageViewer(NetUtils.createSocketAddr(addr))) {
+          viewer.start(inputFile);
+        }
+        break;
+      case "Delimited":
+        try (PBImageDelimitedTextWriter writer =
+            new PBImageDelimitedTextWriter(out, delimiter, tempPath)) {
+          writer.visit(new RandomAccessFile(inputFile, "r"));
+        }
+        break;
+      default:
+        System.err.println("Invalid processor specified : " + processor);
+        printUsage();
+        return -1;
       }
       return 0;
     } catch (EOFException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63f59489/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 22886d3..4075878 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -239,6 +239,7 @@ Usage: `hdfs oiv [OPTIONS] -i INPUT_FILE`
 | `-addr` *address* | Specify the address(host:port) to listen. (localhost:5978 by default). This option is used with Web processor. |
 | `-maxSize` *size* | Specify the range [0, maxSize] of file sizes to be analyzed in bytes (128GB by default). This option is used with FileDistribution processor. |
 | `-step` *size* | Specify the granularity of the distribution in bytes (2MB by default). This option is used with FileDistribution processor. |
+| `-format` | Format the output result in a human-readable fashion rather than a number of bytes. (false by default). This option is used with FileDistribution processor. |
 | `-delimiter` *arg* | Delimiting string to use with Delimited processor. |
 | `-t`,`--temp` *temporary dir* | Use temporary dir to cache intermediate result to generate Delimited outputs. If not set, Delimited processor constructs the namespace in memory before outputting text. |
 | `-h`,`--help` | Display the tool usage and help information and exit. |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63f59489/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
index de27fc2..f991b44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
@@ -150,6 +150,7 @@ Options
 | `-addr` *address* | Specify the address(host:port) to listen. (localhost:5978 by default). This option is used with Web processor. |
 | `-maxSize` *size* | Specify the range [0, maxSize] of file sizes to be analyzed in bytes (128GB by default). This option is used with FileDistribution processor. |
 | `-step` *size* | Specify the granularity of the distribution in bytes (2MB by default). This option is used with FileDistribution processor. |
+| `-format` | Format the output result in a human-readable fashion rather than a number of bytes. (false by default). This option is used with FileDistribution processor. |
 | `-delimiter` *arg* | Delimiting string to use with Delimited processor. |
 | `-t`\|`--temp` *temporary dir* | Use temporary dir to cache intermediate result to generate Delimited outputs. If not set, Delimited processor constructs the namespace in memory before outputting text. |
 | `-h`\|`--help` | Display the tool usage and help information and exit. |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63f59489/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
index a7c30ec..740a8ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
@@ -237,7 +237,7 @@ public class TestOfflineImageViewer {
     File truncatedFile = new File(tempDir, "truncatedFsImage");
     PrintStream output = new PrintStream(NullOutputStream.NULL_OUTPUT_STREAM);
     copyPartOfFile(originalFsimage, truncatedFile);
-    new FileDistributionCalculator(new Configuration(), 0, 0, output)
+    new FileDistributionCalculator(new Configuration(), 0, 0, false, output)
         .visit(new RandomAccessFile(truncatedFile, "r"));
   }
 
@@ -259,7 +259,7 @@ public class TestOfflineImageViewer {
   public void testFileDistributionCalculator() throws IOException {
     ByteArrayOutputStream output = new ByteArrayOutputStream();
     PrintStream o = new PrintStream(output);
-    new FileDistributionCalculator(new Configuration(), 0, 0, o)
+    new FileDistributionCalculator(new Configuration(), 0, 0, false, o)
         .visit(new RandomAccessFile(originalFsimage, "r"));
     o.close();
 
@@ -620,4 +620,24 @@ public class TestOfflineImageViewer {
       IOUtils.closeStream(out);
     }
   }
+
+  @Test
+  public void testOfflineImageViewerWithFormatOption() throws Exception {
+    final ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    final PrintStream out = new PrintStream(bytes);
+    final PrintStream oldOut = System.out;
+    try {
+      System.setOut(out);
+      int status =
+          OfflineImageViewerPB.run(new String[] {"-i",
+              originalFsimage.getAbsolutePath(), "-o", "-", "-p",
+              "FileDistribution", "-maxSize", "512", "-step", "8",
+              "-format"});
+      assertEquals(0, status);
+      Assert.assertTrue(bytes.toString().contains("(0 B, 8 B]"));
+    } finally {
+      System.setOut(oldOut);
+      IOUtils.closeStream(out);
+    }
+  }
 }


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