Browse Source

HDFS-6158. Clean up dead code for OfflineImageViewer. Contributed by Haohui Mai.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1582164 13f79535-47bb-0310-9956-ffa450edef68
Haohui Mai 11 years ago
parent
commit
fd3d5ab792
17 changed files with 6 additions and 2445 deletions
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 0 172
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DelimitedImageVisitor.java
  3. 0 36
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DepthCounter.java
  4. 0 193
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java
  5. 0 83
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoader.java
  6. 0 835
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  7. 0 111
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/IndentedImageVisitor.java
  8. 0 178
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsImageVisitor.java
  9. 0 118
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/NameDistributionVisitor.java
  10. 0 274
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
  11. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
  12. 0 109
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TextWriterImageVisitor.java
  13. 0 88
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/XmlImageVisitor.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
  15. 0 101
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestDelimitedImageVisitor.java
  16. 0 135
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOIVCanReadOldVersions.java
  17. 1 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -266,6 +266,8 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6119. FSNamesystem code cleanup. (suresh)
 
+    HDFS-6158. Clean up dead code for OfflineImageViewer (wheat9)
+
   OPTIMIZATIONS
 
   BUG FIXES 

+ 0 - 172
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DelimitedImageVisitor.java

@@ -1,172 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.IOException;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-
-/**
- * A DelimitedImageVisitor generates a text representation of the fsimage,
- * with each element separated by a delimiter string.  All of the elements
- * common to both inodes and inodes-under-construction are included. When 
- * processing an fsimage with a layout version that did not include an 
- * element, such as AccessTime, the output file will include a column
- * for the value, but no value will be included.
- * 
- * Individual block information for each file is not currently included.
- * 
- * The default delimiter is tab, as this is an unlikely value to be included
- * an inode path or other text metadata.  The delimiter value can be via the
- * constructor.
- */
-class DelimitedImageVisitor extends TextWriterImageVisitor {
-  private static final String defaultDelimiter = "\t"; 
-  
-  final private LinkedList<ImageElement> elemQ = new LinkedList<ImageElement>();
-  private long fileSize = 0l;
-  // Elements of fsimage we're interested in tracking
-  private final Collection<ImageElement> elementsToTrack;
-  // Values for each of the elements in elementsToTrack
-  private final AbstractMap<ImageElement, String> elements = 
-                                            new HashMap<ImageElement, String>();
-  private final String delimiter;
-
-  {
-    elementsToTrack = new ArrayList<ImageElement>();
-    
-    // This collection determines what elements are tracked and the order
-    // in which they are output
-    Collections.addAll(elementsToTrack,  ImageElement.INODE_PATH,
-                                         ImageElement.REPLICATION,
-                                         ImageElement.MODIFICATION_TIME,
-                                         ImageElement.ACCESS_TIME,
-                                         ImageElement.BLOCK_SIZE,
-                                         ImageElement.NUM_BLOCKS,
-                                         ImageElement.NUM_BYTES,
-                                         ImageElement.NS_QUOTA,
-                                         ImageElement.DS_QUOTA,
-                                         ImageElement.PERMISSION_STRING,
-                                         ImageElement.USER_NAME,
-                                         ImageElement.GROUP_NAME);
-  }
-  
-  public DelimitedImageVisitor(String filename) throws IOException {
-    this(filename, false);
-  }
-
-  public DelimitedImageVisitor(String outputFile, boolean printToScreen) 
-                                                           throws IOException {
-    this(outputFile, printToScreen, defaultDelimiter);
-  }
-  
-  public DelimitedImageVisitor(String outputFile, boolean printToScreen, 
-                               String delimiter) throws IOException {
-    super(outputFile, printToScreen);
-    this.delimiter = delimiter;
-    reset();
-  }
-
-  /**
-   * Reset the values of the elements we're tracking in order to handle
-   * the next file
-   */
-  private void reset() {
-    elements.clear();
-    for(ImageElement e : elementsToTrack) 
-      elements.put(e, null);
-    
-    fileSize = 0l;
-  }
-  
-  @Override
-  void leaveEnclosingElement() throws IOException {
-    ImageElement elem = elemQ.pop();
-
-    // If we're done with an inode, write out our results and start over
-    if(elem == ImageElement.INODE || 
-       elem == ImageElement.INODE_UNDER_CONSTRUCTION) {
-      writeLine();
-      write("\n");
-      reset();
-    }
-  }
-
-  /**
-   * Iterate through all the elements we're tracking and, if a value was
-   * recorded for it, write it out.
-   */
-  private void writeLine() throws IOException {
-    Iterator<ImageElement> it = elementsToTrack.iterator();
-    
-    while(it.hasNext()) {
-      ImageElement e = it.next();
-      
-      String v = null;
-      if(e == ImageElement.NUM_BYTES)
-        v = String.valueOf(fileSize);
-      else
-        v = elements.get(e);
-      
-      if(v != null)
-        write(v);
-      
-      if(it.hasNext())
-        write(delimiter);
-    }
-  }
-
-  @Override
-  void visit(ImageElement element, String value) throws IOException {
-    // Explicitly label the root path
-    if(element == ImageElement.INODE_PATH && value.equals(""))
-      value = "/";
-    
-    // Special case of file size, which is sum of the num bytes in each block
-    if(element == ImageElement.NUM_BYTES)
-      fileSize += Long.parseLong(value);
-    
-    if(elements.containsKey(element) && element != ImageElement.NUM_BYTES)
-      elements.put(element, value);
-    
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element) throws IOException {
-    elemQ.push(element);
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element, ImageElement key,
-      String value) throws IOException {
-    // Special case as numBlocks is an attribute of the blocks element
-    if(key == ImageElement.NUM_BLOCKS 
-        && elements.containsKey(ImageElement.NUM_BLOCKS))
-      elements.put(key, value);
-    
-    elemQ.push(element);
-  }
-  
-  @Override
-  void start() throws IOException { /* Nothing to do */ }
-}

+ 0 - 36
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DepthCounter.java

@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Utility class for tracking descent into the structure of the
- * Visitor class (ImageVisitor, EditsVisitor etc.)
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class DepthCounter {
-  private int depth = 0;
-
-  public void incLevel() { depth++; }
-  public void decLevel() { if(depth >= 1) depth--; }
-  public int  getLevel() { return depth; }
-}
-

+ 0 - 193
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java

@@ -1,193 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.IOException;
-import java.util.LinkedList;
-
-/**
- * File size distribution visitor.
- * 
- * <h3>Description.</h3>
- * This is the tool for analyzing file sizes in the namespace image.
- * In order to run the tool one should define a range of integers
- * <tt>[0, maxSize]</tt> by specifying <tt>maxSize</tt> and a <tt>step</tt>.
- * The range of integers is divided into segments of size <tt>step</tt>: 
- * <tt>[0, s<sub>1</sub>, ..., s<sub>n-1</sub>, maxSize]</tt>,
- * and the visitor calculates how many files in the system fall into 
- * each segment <tt>[s<sub>i-1</sub>, s<sub>i</sub>)</tt>. 
- * Note that files larger than <tt>maxSize</tt> always fall into 
- * the very last segment.
- * 
- * <h3>Input.</h3>
- * <ul>
- * <li><tt>filename</tt> specifies the location of the image file;</li>
- * <li><tt>maxSize</tt> determines the range <tt>[0, maxSize]</tt> of files
- * sizes considered by the visitor;</li>
- * <li><tt>step</tt> the range is divided into segments of size step.</li>
- * </ul>
- *
- * <h3>Output.</h3>
- * The output file is formatted as a tab separated two column table:
- * Size and NumFiles. Where Size represents the start of the segment,
- * and numFiles is the number of files form the image which size falls in 
- * this segment.
- */
-class FileDistributionVisitor extends TextWriterImageVisitor {
-  final private LinkedList<ImageElement> elemS = new LinkedList<ImageElement>();
-
-  private final static long MAX_SIZE_DEFAULT = 0x2000000000L;   // 1/8 TB = 2^37
-  private final static int INTERVAL_DEFAULT = 0x200000;         // 2 MB = 2^21
-
-  private int[] distribution;
-  private long maxSize;
-  private int step;
-
-  private int totalFiles;
-  private int totalDirectories;
-  private int totalBlocks;
-  private long totalSpace;
-  private long maxFileSize;
-
-  private FileContext current;
-
-  private boolean inInode = false;
-
-  /**
-   * File or directory information.
-   */
-  private static class FileContext {
-    String path;
-    long fileSize;
-    int numBlocks;
-    int replication;
-  }
-
-  public FileDistributionVisitor(String filename,
-                                 long maxSize,
-                                 int step) throws IOException {
-    super(filename, false);
-    this.maxSize = (maxSize == 0 ? MAX_SIZE_DEFAULT : maxSize);
-    this.step = (step == 0 ? INTERVAL_DEFAULT : step);
-    long numIntervals = this.maxSize / this.step;
-    if(numIntervals >= Integer.MAX_VALUE)
-      throw new IOException("Too many distribution intervals " + numIntervals);
-    this.distribution = new int[1 + (int)(numIntervals)];
-    this.totalFiles = 0;
-    this.totalDirectories = 0;
-    this.totalBlocks = 0;
-    this.totalSpace = 0;
-    this.maxFileSize = 0;
-  }
-
-  @Override
-  void start() throws IOException {}
-
-  @Override
-  void finish() throws IOException {
-    output();
-    super.finish();
-  }
-
-  @Override
-  void finishAbnormally() throws IOException {
-    System.out.println("*** Image processing finished abnormally.  Ending ***");
-    output();
-    super.finishAbnormally();
-  }
-
-  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");
-    System.out.println("totalFiles = " + totalFiles);
-    System.out.println("totalDirectories = " + totalDirectories);
-    System.out.println("totalBlocks = " + totalBlocks);
-    System.out.println("totalSpace = " + totalSpace);
-    System.out.println("maxFileSize = " + maxFileSize);
-  }
-
-  @Override
-  void leaveEnclosingElement() throws IOException {
-    ImageElement elem = elemS.pop();
-
-    if(elem != ImageElement.INODE &&
-       elem != ImageElement.INODE_UNDER_CONSTRUCTION)
-      return;
-    inInode = false;
-    if(current.numBlocks < 0) {
-      totalDirectories ++;
-      return;
-    }
-    totalFiles++;
-    totalBlocks += current.numBlocks;
-    totalSpace += current.fileSize * current.replication;
-    if(maxFileSize < current.fileSize)
-      maxFileSize = current.fileSize;
-    int high;
-    if(current.fileSize > maxSize)
-      high = distribution.length-1;
-    else
-      high = (int)Math.ceil((double)current.fileSize / step);
-    distribution[high]++;
-    if(totalFiles % 1000000 == 1)
-      System.out.println("Files processed: " + totalFiles
-          + "  Current: " + current.path);
-  }
-
-  @Override
-  void visit(ImageElement element, String value) throws IOException {
-    if(inInode) {
-      switch(element) {
-      case INODE_PATH:
-        current.path = (value.equals("") ? "/" : value);
-        break;
-      case REPLICATION:
-        current.replication = Integer.parseInt(value);
-        break;
-      case NUM_BYTES:
-        current.fileSize += Long.parseLong(value);
-        break;
-      default:
-        break;
-      }
-    }
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element) throws IOException {
-    elemS.push(element);
-    if(element == ImageElement.INODE ||
-       element == ImageElement.INODE_UNDER_CONSTRUCTION) {
-      current = new FileContext();
-      inInode = true;
-    }
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element,
-      ImageElement key, String value) throws IOException {
-    elemS.push(element);
-    if(element == ImageElement.INODE ||
-       element == ImageElement.INODE_UNDER_CONSTRUCTION)
-      inInode = true;
-    else if(element == ImageElement.BLOCKS)
-      current.numBlocks = Integer.parseInt(value);
-  }
-}

+ 0 - 83
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoader.java

@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * An ImageLoader can accept a DataInputStream to an Hadoop FSImage file
- * and walk over its structure using the supplied ImageVisitor.
- *
- * Each implementation of ImageLoader is designed to rapidly process an
- * image file.  As long as minor changes are made from one layout version
- * to another, it is acceptable to tweak one implementation to read the next.
- * However, if the layout version changes enough that it would make a
- * processor slow or difficult to read, another processor should be created.
- * This allows each processor to quickly read an image without getting
- * bogged down in dealing with significant differences between layout versions.
- */
-interface ImageLoader {
-
-  /**
-   * @param in DataInputStream pointing to an Hadoop FSImage file
-   * @param v Visit to apply to the FSImage file
-   * @param enumerateBlocks Should visitor visit each of the file blocks?
-   */
-  public void loadImage(DataInputStream in, ImageVisitor v,
-      boolean enumerateBlocks) throws IOException;
-
-  /**
-   * Can this processor handle the specified version of FSImage file?
-   *
-   * @param version FSImage version file
-   * @return True if this instance can process the file
-   */
-  public boolean canLoadVersion(int version);
-
-  /**
-   * Factory for obtaining version of image loader that can read
-   * a particular image format.
-   */
-  @InterfaceAudience.Private
-  public class LoaderFactory {
-    // Java doesn't support static methods on interfaces, which necessitates
-    // this factory class
-
-    /**
-     * Find an image loader capable of interpreting the specified
-     * layout version number.  If none, return null;
-     *
-     * @param version fsimage layout version number to be processed
-     * @return ImageLoader that can interpret specified version, or null
-     */
-    static public ImageLoader getLoader(int version) {
-      // Easy to add more image processors as they are written
-      ImageLoader[] loaders = { new ImageLoaderCurrent() };
-
-      for (ImageLoader l : loaders) {
-        if (l.canLoadVersion(version))
-          return l;
-      }
-
-      return null;
-    }
-  }
-}

+ 0 - 835
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -1,835 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
-import org.apache.hadoop.hdfs.protocol.LayoutFlags;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
-import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.security.token.delegation.DelegationKey;
-
-/**
- * ImageLoaderCurrent processes Hadoop FSImage files and walks over
- * them using a provided ImageVisitor, calling the visitor at each element
- * enumerated below.
- *
- * The only difference between v18 and v19 was the utilization of the
- * stickybit.  Therefore, the same viewer can reader either format.
- *
- * Versions -19 fsimage layout (with changes from -16 up):
- * Image version (int)
- * Namepsace ID (int)
- * NumFiles (long)
- * Generation stamp (long)
- * INodes (count = NumFiles)
- *  INode
- *    Path (String)
- *    Replication (short)
- *    Modification Time (long as date)
- *    Access Time (long) // added in -16
- *    Block size (long)
- *    Num blocks (int)
- *    Blocks (count = Num blocks)
- *      Block
- *        Block ID (long)
- *        Num bytes (long)
- *        Generation stamp (long)
- *    Namespace Quota (long)
- *    Diskspace Quota (long) // added in -18
- *    Permissions
- *      Username (String)
- *      Groupname (String)
- *      OctalPerms (short -> String)  // Modified in -19
- *    Symlink (String) // added in -23
- * NumINodesUnderConstruction (int)
- * INodesUnderConstruction (count = NumINodesUnderConstruction)
- *  INodeUnderConstruction
- *    Path (bytes as string)
- *    Replication (short)
- *    Modification time (long as date)
- *    Preferred block size (long)
- *    Num blocks (int)
- *    Blocks
- *      Block
- *        Block ID (long)
- *        Num bytes (long)
- *        Generation stamp (long)
- *    Permissions
- *      Username (String)
- *      Groupname (String)
- *      OctalPerms (short -> String)
- *    Client Name (String)
- *    Client Machine (String)
- *    NumLocations (int)
- *    DatanodeDescriptors (count = numLocations) // not loaded into memory
- *      short                                    // but still in file
- *      long
- *      string
- *      long
- *      int
- *      string
- *      string
- *      enum
- *    CurrentDelegationKeyId (int)
- *    NumDelegationKeys (int)
- *      DelegationKeys (count = NumDelegationKeys)
- *        DelegationKeyLength (vint)
- *        DelegationKey (bytes)
- *    DelegationTokenSequenceNumber (int)
- *    NumDelegationTokens (int)
- *    DelegationTokens (count = NumDelegationTokens)
- *      DelegationTokenIdentifier
- *        owner (String)
- *        renewer (String)
- *        realUser (String)
- *        issueDate (vlong)
- *        maxDate (vlong)
- *        sequenceNumber (vint)
- *        masterKeyId (vint)
- *      expiryTime (long)     
- *
- */
-class ImageLoaderCurrent implements ImageLoader {
-  protected final DateFormat dateFormat = 
-                                      new SimpleDateFormat("yyyy-MM-dd HH:mm");
-  private static final int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
-      -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49, -50, -51, -52 };
-  private int imageVersion = 0;
-  
-  private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
-  private final Map<Long, String> dirNodeMap = new HashMap<Long, String>();
-
-  /* (non-Javadoc)
-   * @see ImageLoader#canProcessVersion(int)
-   */
-  @Override
-  public boolean canLoadVersion(int version) {
-    for(int v : versions)
-      if(v == version) return true;
-
-    return false;
-  }
-
-  /* (non-Javadoc)
-   * @see ImageLoader#processImage(java.io.DataInputStream, ImageVisitor, boolean)
-   */
-  @Override
-  public void loadImage(DataInputStream in, ImageVisitor v,
-      boolean skipBlocks) throws IOException {
-    boolean done = false;
-    try {
-      v.start();
-      v.visitEnclosingElement(ImageElement.FS_IMAGE);
-
-      imageVersion = in.readInt();
-      if( !canLoadVersion(imageVersion))
-        throw new IOException("Cannot process fslayout version " + imageVersion);
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.ADD_LAYOUT_FLAGS, imageVersion)) {
-        LayoutFlags.read(in);
-      }
-
-      v.visit(ImageElement.IMAGE_VERSION, imageVersion);
-      v.visit(ImageElement.NAMESPACE_ID, in.readInt());
-
-      long numInodes = in.readLong();
-
-      v.visit(ImageElement.GENERATION_STAMP, in.readLong());
-
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imageVersion)) {
-        v.visit(ImageElement.GENERATION_STAMP_V2, in.readLong());
-        v.visit(ImageElement.GENERATION_STAMP_V1_LIMIT, in.readLong());
-        v.visit(ImageElement.LAST_ALLOCATED_BLOCK_ID, in.readLong());
-      }
-
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.STORED_TXIDS, imageVersion)) {
-        v.visit(ImageElement.TRANSACTION_ID, in.readLong());
-      }
-      
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.ADD_INODE_ID, imageVersion)) {
-        v.visit(ImageElement.LAST_INODE_ID, in.readLong());
-      }
-      
-      boolean supportSnapshot = NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.SNAPSHOT, imageVersion);
-      if (supportSnapshot) {
-        v.visit(ImageElement.SNAPSHOT_COUNTER, in.readInt());
-        int numSnapshots = in.readInt();
-        v.visit(ImageElement.NUM_SNAPSHOTS_TOTAL, numSnapshots);
-        for (int i = 0; i < numSnapshots; i++) {
-          processSnapshot(in, v);
-        }
-      }
-      
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.FSIMAGE_COMPRESSION, imageVersion)) {
-        boolean isCompressed = in.readBoolean();
-        v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
-        if (isCompressed) {
-          String codecClassName = Text.readString(in);
-          v.visit(ImageElement.COMPRESS_CODEC, codecClassName);
-          CompressionCodecFactory codecFac = new CompressionCodecFactory(
-              new Configuration());
-          CompressionCodec codec = codecFac.getCodecByClassName(codecClassName);
-          if (codec == null) {
-            throw new IOException("Image compression codec not supported: "
-                + codecClassName);
-          }
-          in = new DataInputStream(codec.createInputStream(in));
-        }
-      }
-      processINodes(in, v, numInodes, skipBlocks, supportSnapshot);
-      subtreeMap.clear();
-      dirNodeMap.clear();
-
-      processINodesUC(in, v, skipBlocks);
-
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.DELEGATION_TOKEN, imageVersion)) {
-        processDelegationTokens(in, v);
-      }
-      
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.CACHING, imageVersion)) {
-        processCacheManagerState(in, v);
-      }
-      v.leaveEnclosingElement(); // FSImage
-      done = true;
-    } finally {
-      if (done) {
-        v.finish();
-      } else {
-        v.finishAbnormally();
-      }
-    }
-  }
-
-  /**
-   * Process CacheManager state from the fsimage.
-   */
-  private void processCacheManagerState(DataInputStream in, ImageVisitor v)
-      throws IOException {
-    v.visit(ImageElement.CACHE_NEXT_ENTRY_ID, in.readLong());
-    final int numPools = in.readInt();
-    for (int i=0; i<numPools; i++) {
-      v.visit(ImageElement.CACHE_POOL_NAME, Text.readString(in));
-      processCachePoolPermission(in, v);
-      v.visit(ImageElement.CACHE_POOL_WEIGHT, in.readInt());
-    }
-    final int numEntries = in.readInt();
-    for (int i=0; i<numEntries; i++) {
-      v.visit(ImageElement.CACHE_ENTRY_PATH, Text.readString(in));
-      v.visit(ImageElement.CACHE_ENTRY_REPLICATION, in.readShort());
-      v.visit(ImageElement.CACHE_ENTRY_POOL_NAME, Text.readString(in));
-    }
-  }
-  /**
-   * Process the Delegation Token related section in fsimage.
-   * 
-   * @param in DataInputStream to process
-   * @param v Visitor to walk over records
-   */
-  private void processDelegationTokens(DataInputStream in, ImageVisitor v)
-      throws IOException {
-    v.visit(ImageElement.CURRENT_DELEGATION_KEY_ID, in.readInt());
-    int numDKeys = in.readInt();
-    v.visitEnclosingElement(ImageElement.DELEGATION_KEYS,
-        ImageElement.NUM_DELEGATION_KEYS, numDKeys);
-    for(int i =0; i < numDKeys; i++) {
-      DelegationKey key = new DelegationKey();
-      key.readFields(in);
-      v.visit(ImageElement.DELEGATION_KEY, key.toString());
-    }
-    v.leaveEnclosingElement();
-    v.visit(ImageElement.DELEGATION_TOKEN_SEQUENCE_NUMBER, in.readInt());
-    int numDTokens = in.readInt();
-    v.visitEnclosingElement(ImageElement.DELEGATION_TOKENS,
-        ImageElement.NUM_DELEGATION_TOKENS, numDTokens);
-    for(int i=0; i<numDTokens; i++){
-      DelegationTokenIdentifier id = new  DelegationTokenIdentifier();
-      id.readFields(in);
-      long expiryTime = in.readLong();
-      v.visitEnclosingElement(ImageElement.DELEGATION_TOKEN_IDENTIFIER);
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_KIND,
-          id.getKind().toString());
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_SEQNO,
-          id.getSequenceNumber());
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_OWNER,
-          id.getOwner().toString());
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_RENEWER,
-          id.getRenewer().toString());
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_REALUSER,
-          id.getRealUser().toString());
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_ISSUE_DATE,
-          id.getIssueDate());
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_MAX_DATE,
-          id.getMaxDate());
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_EXPIRY_TIME,
-          expiryTime);
-      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
-          id.getMasterKeyId());
-      v.leaveEnclosingElement(); // DELEGATION_TOKEN_IDENTIFIER
-    }
-    v.leaveEnclosingElement(); // DELEGATION_TOKENS
-  }
-
-  /**
-   * Process the INodes under construction section of the fsimage.
-   *
-   * @param in DataInputStream to process
-   * @param v Visitor to walk over inodes
-   * @param skipBlocks Walk over each block?
-   */
-  private void processINodesUC(DataInputStream in, ImageVisitor v,
-      boolean skipBlocks) throws IOException {
-    int numINUC = in.readInt();
-
-    v.visitEnclosingElement(ImageElement.INODES_UNDER_CONSTRUCTION,
-                           ImageElement.NUM_INODES_UNDER_CONSTRUCTION, numINUC);
-
-    for(int i = 0; i < numINUC; i++) {
-      v.visitEnclosingElement(ImageElement.INODE_UNDER_CONSTRUCTION);
-      byte [] name = FSImageSerialization.readBytes(in);
-      String n = new String(name, "UTF8");
-      v.visit(ImageElement.INODE_PATH, n);
-      
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.ADD_INODE_ID, imageVersion)) {
-        long inodeId = in.readLong();
-        v.visit(ImageElement.INODE_ID, inodeId);
-      }
-      
-      v.visit(ImageElement.REPLICATION, in.readShort());
-      v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
-
-      v.visit(ImageElement.PREFERRED_BLOCK_SIZE, in.readLong());
-      int numBlocks = in.readInt();
-      processBlocks(in, v, numBlocks, skipBlocks);
-
-      processPermission(in, v);
-      v.visit(ImageElement.CLIENT_NAME, FSImageSerialization.readString(in));
-      v.visit(ImageElement.CLIENT_MACHINE, FSImageSerialization.readString(in));
-
-      // Skip over the datanode descriptors, which are still stored in the
-      // file but are not used by the datanode or loaded into memory
-      int numLocs = in.readInt();
-      for(int j = 0; j < numLocs; j++) {
-        in.readShort();
-        in.readLong();
-        in.readLong();
-        in.readLong();
-        in.readInt();
-        FSImageSerialization.readString(in);
-        FSImageSerialization.readString(in);
-        WritableUtils.readEnum(in, AdminStates.class);
-      }
-
-      v.leaveEnclosingElement(); // INodeUnderConstruction
-    }
-
-    v.leaveEnclosingElement(); // INodesUnderConstruction
-  }
-
-  /**
-   * Process the blocks section of the fsimage.
-   *
-   * @param in Datastream to process
-   * @param v Visitor to walk over inodes
-   * @param skipBlocks Walk over each block?
-   */
-  private void processBlocks(DataInputStream in, ImageVisitor v,
-      int numBlocks, boolean skipBlocks) throws IOException {
-    v.visitEnclosingElement(ImageElement.BLOCKS,
-                            ImageElement.NUM_BLOCKS, numBlocks);
-    
-    // directory or symlink or reference node, no blocks to process    
-    if(numBlocks < 0) { 
-      v.leaveEnclosingElement(); // Blocks
-      return;
-    }
-    
-    if(skipBlocks) {
-      int bytesToSkip = ((Long.SIZE * 3 /* fields */) / 8 /*bits*/) * numBlocks;
-      if(in.skipBytes(bytesToSkip) != bytesToSkip)
-        throw new IOException("Error skipping over blocks");
-      
-    } else {
-      for(int j = 0; j < numBlocks; j++) {
-        v.visitEnclosingElement(ImageElement.BLOCK);
-        v.visit(ImageElement.BLOCK_ID, in.readLong());
-        v.visit(ImageElement.NUM_BYTES, in.readLong());
-        v.visit(ImageElement.GENERATION_STAMP, in.readLong());
-        v.leaveEnclosingElement(); // Block
-      }
-    }
-    v.leaveEnclosingElement(); // Blocks
-  }
-
-  /**
-   * Extract the INode permissions stored in the fsimage file.
-   *
-   * @param in Datastream to process
-   * @param v Visitor to walk over inodes
-   */
-  private void processPermission(DataInputStream in, ImageVisitor v)
-      throws IOException {
-    v.visitEnclosingElement(ImageElement.PERMISSIONS);
-    v.visit(ImageElement.USER_NAME, Text.readString(in));
-    v.visit(ImageElement.GROUP_NAME, Text.readString(in));
-    FsPermission fsp = new FsPermission(in.readShort());
-    v.visit(ImageElement.PERMISSION_STRING, fsp.toString());
-    v.leaveEnclosingElement(); // Permissions
-  }
-
-  /**
-   * Extract CachePool permissions stored in the fsimage file.
-   *
-   * @param in Datastream to process
-   * @param v Visitor to walk over inodes
-   */
-  private void processCachePoolPermission(DataInputStream in, ImageVisitor v)
-      throws IOException {
-    v.visitEnclosingElement(ImageElement.PERMISSIONS);
-    v.visit(ImageElement.CACHE_POOL_OWNER_NAME, Text.readString(in));
-    v.visit(ImageElement.CACHE_POOL_GROUP_NAME, Text.readString(in));
-    FsPermission fsp = new FsPermission(in.readShort());
-    v.visit(ImageElement.CACHE_POOL_PERMISSION_STRING, fsp.toString());
-    v.leaveEnclosingElement(); // Permissions
-  }
-
-  /**
-   * Process the INode records stored in the fsimage.
-   *
-   * @param in Datastream to process
-   * @param v Visitor to walk over INodes
-   * @param numInodes Number of INodes stored in file
-   * @param skipBlocks Process all the blocks within the INode?
-   * @param supportSnapshot Whether or not the imageVersion supports snapshot
-   * @throws VisitException
-   * @throws IOException
-   */
-  private void processINodes(DataInputStream in, ImageVisitor v,
-      long numInodes, boolean skipBlocks, boolean supportSnapshot)
-      throws IOException {
-    v.visitEnclosingElement(ImageElement.INODES,
-        ImageElement.NUM_INODES, numInodes);
-    
-    if (NameNodeLayoutVersion.supports(
-        LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
-      if (!supportSnapshot) {
-        processLocalNameINodes(in, v, numInodes, skipBlocks);
-      } else {
-        processLocalNameINodesWithSnapshot(in, v, skipBlocks);
-      }
-    } else { // full path name
-      processFullNameINodes(in, v, numInodes, skipBlocks);
-    }
-
-    
-    v.leaveEnclosingElement(); // INodes
-  }
-  
-  /**
-   * Process image with full path name
-   * 
-   * @param in image stream
-   * @param v visitor
-   * @param numInodes number of indoes to read
-   * @param skipBlocks skip blocks or not
-   * @throws IOException if there is any error occurs
-   */
-  private void processLocalNameINodes(DataInputStream in, ImageVisitor v,
-      long numInodes, boolean skipBlocks) throws IOException {
-    // process root
-    processINode(in, v, skipBlocks, "", false);
-    numInodes--;
-    while (numInodes > 0) {
-      numInodes -= processDirectory(in, v, skipBlocks);
-    }
-  }
-  
-  private int processDirectory(DataInputStream in, ImageVisitor v,
-     boolean skipBlocks) throws IOException {
-    String parentName = FSImageSerialization.readString(in);
-    return processChildren(in, v, skipBlocks, parentName);
-  }
-  
-  /**
-   * Process image with local path name and snapshot support
-   * 
-   * @param in image stream
-   * @param v visitor
-   * @param skipBlocks skip blocks or not
-   */
-  private void processLocalNameINodesWithSnapshot(DataInputStream in,
-      ImageVisitor v, boolean skipBlocks) throws IOException {
-    // process root
-    processINode(in, v, skipBlocks, "", false);
-    processDirectoryWithSnapshot(in, v, skipBlocks);
-  }
-  
-  /**
-   * Process directories when snapshot is supported.
-   */
-  private void processDirectoryWithSnapshot(DataInputStream in, ImageVisitor v,
-      boolean skipBlocks) throws IOException {
-    // 1. load dir node id
-    long inodeId = in.readLong();
-    
-    String dirName = dirNodeMap.remove(inodeId);
-    Boolean visitedRef = subtreeMap.get(inodeId);
-    if (visitedRef != null) {
-      if (visitedRef.booleanValue()) { // the subtree has been visited
-        return;
-      } else { // first time to visit
-        subtreeMap.put(inodeId, true);
-      }
-    } // else the dir is not linked by a RefNode, thus cannot be revisited
-    
-    // 2. load possible snapshots
-    processSnapshots(in, v, dirName);
-    // 3. load children nodes
-    processChildren(in, v, skipBlocks, dirName);
-    // 4. load possible directory diff list
-    processDirectoryDiffList(in, v, dirName);
-    // recursively process sub-directories
-    final int numSubTree = in.readInt();
-    for (int i = 0; i < numSubTree; i++) {
-      processDirectoryWithSnapshot(in, v, skipBlocks);
-    }
-  }
-  
-  /**
-   * Process snapshots of a snapshottable directory
-   */
-  private void processSnapshots(DataInputStream in, ImageVisitor v,
-      String rootName) throws IOException {
-    final int numSnapshots = in.readInt();
-    if (numSnapshots >= 0) {
-      v.visitEnclosingElement(ImageElement.SNAPSHOTS,
-          ImageElement.NUM_SNAPSHOTS, numSnapshots);
-      for (int i = 0; i < numSnapshots; i++) {
-        // process snapshot
-        v.visitEnclosingElement(ImageElement.SNAPSHOT);
-        v.visit(ImageElement.SNAPSHOT_ID, in.readInt());
-        v.leaveEnclosingElement();
-      }
-      v.visit(ImageElement.SNAPSHOT_QUOTA, in.readInt());
-      v.leaveEnclosingElement();
-    }
-  }
-  
-  private void processSnapshot(DataInputStream in, ImageVisitor v)
-      throws IOException {
-    v.visitEnclosingElement(ImageElement.SNAPSHOT);
-    v.visit(ImageElement.SNAPSHOT_ID, in.readInt());
-    // process root of snapshot
-    v.visitEnclosingElement(ImageElement.SNAPSHOT_ROOT);
-    processINode(in, v, true, "", false);
-    v.leaveEnclosingElement();
-    v.leaveEnclosingElement();
-  }
-  
-  private void processDirectoryDiffList(DataInputStream in, ImageVisitor v,
-      String currentINodeName) throws IOException {
-    final int numDirDiff = in.readInt();
-    if (numDirDiff >= 0) {
-      v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFFS,
-          ImageElement.NUM_SNAPSHOT_DIR_DIFF, numDirDiff);
-      for (int i = 0; i < numDirDiff; i++) {
-        // process directory diffs in reverse chronological oder
-        processDirectoryDiff(in, v, currentINodeName); 
-      }
-      v.leaveEnclosingElement();
-    }
-  }
-  
-  private void processDirectoryDiff(DataInputStream in, ImageVisitor v,
-      String currentINodeName) throws IOException {
-    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF);
-    int snapshotId = in.readInt();
-    v.visit(ImageElement.SNAPSHOT_DIFF_SNAPSHOTID, snapshotId);
-    v.visit(ImageElement.SNAPSHOT_DIR_DIFF_CHILDREN_SIZE, in.readInt());
-    
-    // process snapshotINode
-    boolean useRoot = in.readBoolean();
-    if (!useRoot) {
-      if (in.readBoolean()) {
-        v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_DIRECTORY_ATTRIBUTES);
-        if (NameNodeLayoutVersion.supports(
-            LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
-          processINodeDirectoryAttributes(in, v, currentINodeName);
-        } else {
-          processINode(in, v, true, currentINodeName, true);
-        }
-        v.leaveEnclosingElement();
-      }
-    }
-    
-    // process createdList
-    int createdSize = in.readInt();
-    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_CREATEDLIST,
-        ImageElement.SNAPSHOT_DIR_DIFF_CREATEDLIST_SIZE, createdSize);
-    for (int i = 0; i < createdSize; i++) {
-      String createdNode = FSImageSerialization.readString(in);
-      v.visit(ImageElement.SNAPSHOT_DIR_DIFF_CREATED_INODE, createdNode);
-    }
-    v.leaveEnclosingElement();
-    
-    // process deletedList
-    int deletedSize = in.readInt();
-    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_DELETEDLIST,
-        ImageElement.SNAPSHOT_DIR_DIFF_DELETEDLIST_SIZE, deletedSize);
-    for (int i = 0; i < deletedSize; i++) {
-      v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_DELETED_INODE);
-      processINode(in, v, false, currentINodeName, true);
-      v.leaveEnclosingElement();
-    }
-    v.leaveEnclosingElement();
-    v.leaveEnclosingElement();
-  }
-
-  private void processINodeDirectoryAttributes(DataInputStream in, ImageVisitor v,
-      String parentName) throws IOException {
-    final String pathName = readINodePath(in, parentName);
-    v.visit(ImageElement.INODE_PATH, pathName);
-    processPermission(in, v);
-    v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
-
-    v.visit(ImageElement.NS_QUOTA, in.readLong());
-    v.visit(ImageElement.DS_QUOTA, in.readLong());
-  }
-
-  /** Process children under a directory */
-  private int processChildren(DataInputStream in, ImageVisitor v,
-      boolean skipBlocks, String parentName) throws IOException {
-    int numChildren = in.readInt();
-    for (int i = 0; i < numChildren; i++) {
-      processINode(in, v, skipBlocks, parentName, false);
-    }
-    return numChildren;
-  }
-  
-  /**
-   * Process image with full path name
-   * 
-   * @param in image stream
-   * @param v visitor
-   * @param numInodes number of indoes to read
-   * @param skipBlocks skip blocks or not
-   * @throws IOException if there is any error occurs
-   */
-  private void processFullNameINodes(DataInputStream in, ImageVisitor v,
-      long numInodes, boolean skipBlocks) throws IOException {
-    for(long i = 0; i < numInodes; i++) {
-      processINode(in, v, skipBlocks, null, false);
-    }
-  }
- 
-  private String readINodePath(DataInputStream in, String parentName)
-      throws IOException {
-    String pathName = FSImageSerialization.readString(in);
-    if (parentName != null) {  // local name
-      pathName = "/" + pathName;
-      if (!"/".equals(parentName)) { // children of non-root directory
-        pathName = parentName + pathName;
-      }
-    }
-    return pathName;
-  }
-
-  /**
-   * Process an INode
-   * 
-   * @param in image stream
-   * @param v visitor
-   * @param skipBlocks skip blocks or not
-   * @param parentName the name of its parent node
-   * @param isSnapshotCopy whether or not the inode is a snapshot copy
-   * @throws IOException
-   */
-  private void processINode(DataInputStream in, ImageVisitor v,
-      boolean skipBlocks, String parentName, boolean isSnapshotCopy)
-      throws IOException {
-    boolean supportSnapshot = NameNodeLayoutVersion.supports(
-        LayoutVersion.Feature.SNAPSHOT, imageVersion);
-    boolean supportInodeId = NameNodeLayoutVersion.supports(
-        LayoutVersion.Feature.ADD_INODE_ID, imageVersion);
-    
-    v.visitEnclosingElement(ImageElement.INODE);
-    final String pathName = readINodePath(in, parentName);
-    v.visit(ImageElement.INODE_PATH, pathName);
-
-    long inodeId = INodeId.GRANDFATHER_INODE_ID;
-    if (supportInodeId) {
-      inodeId = in.readLong();
-      v.visit(ImageElement.INODE_ID, inodeId);
-    }
-    v.visit(ImageElement.REPLICATION, in.readShort());
-    v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
-    if(NameNodeLayoutVersion.supports(
-        LayoutVersion.Feature.FILE_ACCESS_TIME, imageVersion))
-      v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong()));
-    v.visit(ImageElement.BLOCK_SIZE, in.readLong());
-    int numBlocks = in.readInt();
-
-    processBlocks(in, v, numBlocks, skipBlocks);
-    
-    if (numBlocks >= 0) { // File
-      if (supportSnapshot) {
-        // make sure subtreeMap only contains entry for directory
-        subtreeMap.remove(inodeId);
-        // process file diffs
-        processFileDiffList(in, v, parentName);
-        if (isSnapshotCopy) {
-          boolean underConstruction = in.readBoolean();
-          if (underConstruction) {
-            v.visit(ImageElement.CLIENT_NAME,
-                FSImageSerialization.readString(in));
-            v.visit(ImageElement.CLIENT_MACHINE,
-                FSImageSerialization.readString(in));
-          }
-        }
-      }
-      processPermission(in, v);
-    } else if (numBlocks == -1) { // Directory
-      if (supportSnapshot && supportInodeId) {
-        dirNodeMap.put(inodeId, pathName);
-      }
-      v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.DISKSPACE_QUOTA, imageVersion))
-        v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
-      if (supportSnapshot) {
-        boolean snapshottable = in.readBoolean();
-        if (!snapshottable) {
-          boolean withSnapshot = in.readBoolean();
-          v.visit(ImageElement.IS_WITHSNAPSHOT_DIR, Boolean.toString(withSnapshot));
-        } else {
-          v.visit(ImageElement.IS_SNAPSHOTTABLE_DIR, Boolean.toString(snapshottable));
-        }
-      }
-      processPermission(in, v);
-    } else if (numBlocks == -2) {
-      v.visit(ImageElement.SYMLINK, Text.readString(in));
-      processPermission(in, v);
-    } else if (numBlocks == -3) { // reference node
-      final boolean isWithName = in.readBoolean();
-      int snapshotId = in.readInt();
-      if (isWithName) {
-        v.visit(ImageElement.SNAPSHOT_LAST_SNAPSHOT_ID, snapshotId);
-      } else {
-        v.visit(ImageElement.SNAPSHOT_DST_SNAPSHOT_ID, snapshotId);
-      }
-      
-      final boolean firstReferred = in.readBoolean();
-      if (firstReferred) {
-        // if a subtree is linked by multiple "parents", the corresponding dir
-        // must be referred by a reference node. we put the reference node into
-        // the subtreeMap here and let its value be false. when we later visit
-        // the subtree for the first time, we change the value to true.
-        subtreeMap.put(inodeId, false);
-        v.visitEnclosingElement(ImageElement.SNAPSHOT_REF_INODE);
-        processINode(in, v, skipBlocks, parentName, isSnapshotCopy);
-        v.leaveEnclosingElement();  // referred inode    
-      } else {
-        v.visit(ImageElement.SNAPSHOT_REF_INODE_ID, in.readLong());
-      }
-    }
-
-    v.leaveEnclosingElement(); // INode
-  }
-
-  private void processINodeFileAttributes(DataInputStream in, ImageVisitor v,
-      String parentName) throws IOException {
-    final String pathName = readINodePath(in, parentName);
-    v.visit(ImageElement.INODE_PATH, pathName);
-    processPermission(in, v);
-    v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
-    if(NameNodeLayoutVersion.supports(
-        LayoutVersion.Feature.FILE_ACCESS_TIME, imageVersion)) {
-      v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong()));
-    }
-
-    v.visit(ImageElement.REPLICATION, in.readShort());
-    v.visit(ImageElement.BLOCK_SIZE, in.readLong());
-  }
-  
-  private void processFileDiffList(DataInputStream in, ImageVisitor v,
-      String currentINodeName) throws IOException {
-    final int size = in.readInt();
-    if (size >= 0) {
-      v.visitEnclosingElement(ImageElement.SNAPSHOT_FILE_DIFFS,
-          ImageElement.NUM_SNAPSHOT_FILE_DIFF, size);
-      for (int i = 0; i < size; i++) {
-        processFileDiff(in, v, currentINodeName);
-      }
-      v.leaveEnclosingElement();
-    }
-  }
-  
-  private void processFileDiff(DataInputStream in, ImageVisitor v,
-      String currentINodeName) throws IOException {
-    int snapshotId = in.readInt();
-    v.visitEnclosingElement(ImageElement.SNAPSHOT_FILE_DIFF,
-        ImageElement.SNAPSHOT_DIFF_SNAPSHOTID, snapshotId);
-    v.visit(ImageElement.SNAPSHOT_FILE_SIZE, in.readLong());
-    if (in.readBoolean()) {
-      v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_FILE_ATTRIBUTES);
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
-        processINodeFileAttributes(in, v, currentINodeName);
-      } else {
-        processINode(in, v, true, currentINodeName, true);
-      }
-      v.leaveEnclosingElement();
-    }
-    v.leaveEnclosingElement();
-  }
-  
-  /**
-   * Helper method to format dates during processing.
-   * @param date Date as read from image file
-   * @return String version of date format
-   */
-  private String formatDate(long date) {
-    return dateFormat.format(new Date(date));
-  }
-}

+ 0 - 111
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/IndentedImageVisitor.java

@@ -1,111 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.IOException;
-import java.util.Date;
-
-/**
- * IndentedImageVisitor walks over an FSImage and displays its structure 
- * using indenting to organize sections within the image file.
- */
-class IndentedImageVisitor extends TextWriterImageVisitor {
-  
-  public IndentedImageVisitor(String filename) throws IOException {
-    super(filename);
-  }
-
-  public IndentedImageVisitor(String filename, boolean printToScreen) throws IOException {
-    super(filename, printToScreen);
-  }
-
-  final private DepthCounter dc = new DepthCounter();// to track leading spacing
-
-  @Override
-  void start() throws IOException {}
-
-  @Override
-  void finish() throws IOException { super.finish(); }
-
-  @Override
-  void finishAbnormally() throws IOException {
-    System.out.println("*** Image processing finished abnormally.  Ending ***");
-    super.finishAbnormally();
-  }
-
-  @Override
-  void leaveEnclosingElement() throws IOException {
-    dc.decLevel();
-  }
-
-  @Override
-  void visit(ImageElement element, String value) throws IOException {
-    printIndents();
-    write(element + " = " + value + "\n");
-  }
-
-  @Override
-  void visit(ImageElement element, long value) throws IOException {
-    if ((element == ImageElement.DELEGATION_TOKEN_IDENTIFIER_EXPIRY_TIME) || 
-        (element == ImageElement.DELEGATION_TOKEN_IDENTIFIER_ISSUE_DATE) || 
-        (element == ImageElement.DELEGATION_TOKEN_IDENTIFIER_MAX_DATE)) {
-      visit(element, new Date(value).toString());
-    } else {
-      visit(element, Long.toString(value));
-    }
-  }
-  
-  @Override
-  void visitEnclosingElement(ImageElement element) throws IOException {
-    printIndents();
-    write(element + "\n");
-    dc.incLevel();
-  }
-
-  // Print element, along with associated key/value pair, in brackets
-  @Override
-  void visitEnclosingElement(ImageElement element,
-      ImageElement key, String value)
-      throws IOException {
-    printIndents();
-    write(element + " [" + key + " = " + value + "]\n");
-    dc.incLevel();
-  }
-
-  /**
-  * Print an appropriate number of spaces for the current level.
-  * FsImages can potentially be millions of lines long, so caching can
-  * significantly speed up output.
-  */
-  final private static String [] indents = { "",
-                                             "  ",
-                                             "    ",
-                                             "      ",
-                                             "        ",
-                                             "          ",
-                                             "            "};
-  private void printIndents() throws IOException {
-    try {
-      write(indents[dc.getLevel()]);
-    } catch (IndexOutOfBoundsException e) {
-      // There's no reason in an fsimage would need a deeper indent
-      for(int i = 0; i < dc.getLevel(); i++)
-        write(" ");
-    }
-   }
-}

+ 0 - 178
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsImageVisitor.java

@@ -1,178 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.IOException;
-import java.util.Formatter;
-import java.util.LinkedList;
-
-/**
- * LsImageVisitor displays the blocks of the namespace in a format very similar
- * to the output of ls/lsr.  Entries are marked as directories or not,
- * permissions listed, replication, username and groupname, along with size,
- * modification date and full path.
- *
- * Note: A significant difference between the output of the lsr command
- * and this image visitor is that this class cannot sort the file entries;
- * they are listed in the order they are stored within the fsimage file. 
- * Therefore, the output of this class cannot be directly compared to the
- * output of the lsr command.
- */
-class LsImageVisitor extends TextWriterImageVisitor {
-  final private LinkedList<ImageElement> elemQ = new LinkedList<ImageElement>();
-
-  private int numBlocks;
-  private String perms;
-  private int replication;
-  private String username;
-  private String group;
-  private long filesize;
-  private String modTime;
-  private String path;
-  private String linkTarget;
-
-  private boolean inInode = false;
-  final private StringBuilder sb = new StringBuilder();
-  final private Formatter formatter = new Formatter(sb);
-
-  public LsImageVisitor(String filename) throws IOException {
-    super(filename);
-  }
-
-  public LsImageVisitor(String filename, boolean printToScreen) throws IOException {
-    super(filename, printToScreen);
-  }
-
-  /**
-   * Start a new line of output, reset values.
-   */
-  private void newLine() {
-    numBlocks = 0;
-    perms = username = group = path = linkTarget = "";
-    filesize = 0l;
-    replication = 0;
-
-    inInode = true;
-  }
-
-  /**
-   * All the values have been gathered.  Print them to the console in an
-   * ls-style format.
-   */
-  private final static int widthRepl = 2;  
-  private final static int widthUser = 8; 
-  private final static int widthGroup = 10; 
-  private final static int widthSize = 10;
-  private final static int widthMod = 10;
-  private final static String lsStr = " %" + widthRepl + "s %" + widthUser + 
-                                       "s %" + widthGroup + "s %" + widthSize +
-                                       "d %" + widthMod + "s %s";
-  private void printLine() throws IOException {
-    sb.append(numBlocks < 0 ? "d" : "-");
-    sb.append(perms);
-
-    if (0 != linkTarget.length()) {
-      path = path + " -> " + linkTarget; 
-    }
-    formatter.format(lsStr, replication > 0 ? replication : "-",
-                           username, group, filesize, modTime, path);
-    sb.append("\n");
-
-    write(sb.toString());
-    sb.setLength(0); // clear string builder
-
-    inInode = false;
-  }
-
-  @Override
-  void start() throws IOException {}
-
-  @Override
-  void finish() throws IOException {
-    super.finish();
-  }
-
-  @Override
-  void finishAbnormally() throws IOException {
-    System.out.println("Input ended unexpectedly.");
-    super.finishAbnormally();
-  }
-
-  @Override
-  void leaveEnclosingElement() throws IOException {
-    ImageElement elem = elemQ.pop();
-
-    if(elem == ImageElement.INODE)
-      printLine();
-  }
-
-  // Maintain state of location within the image tree and record
-  // values needed to display the inode in ls-style format.
-  @Override
-  void visit(ImageElement element, String value) throws IOException {
-    if(inInode) {
-      switch(element) {
-      case INODE_PATH:
-        if(value.equals("")) path = "/";
-        else path = value;
-        break;
-      case PERMISSION_STRING:
-        perms = value;
-        break;
-      case REPLICATION:
-        replication = Integer.parseInt(value);
-        break;
-      case USER_NAME:
-        username = value;
-        break;
-      case GROUP_NAME:
-        group = value;
-        break;
-      case NUM_BYTES:
-        filesize += Long.parseLong(value);
-        break;
-      case MODIFICATION_TIME:
-        modTime = value;
-        break;
-      case SYMLINK:
-        linkTarget = value;
-        break;
-      default:
-        // This is OK.  We're not looking for all the values.
-        break;
-      }
-    }
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element) throws IOException {
-    elemQ.push(element);
-    if(element == ImageElement.INODE)
-      newLine();
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element,
-      ImageElement key, String value) throws IOException {
-    elemQ.push(element);
-    if(element == ImageElement.INODE)
-      newLine();
-    else if (element == ImageElement.BLOCKS)
-      numBlocks = Integer.parseInt(value);
-  }
-}

+ 0 - 118
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/NameDistributionVisitor.java

@@ -1,118 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * File name distribution visitor. 
- * <p>
- * It analyzes file names in fsimage and prints the following information: 
- * <li>Number of unique file names</li> 
- * <li>Number file names and the corresponding number range of files that use 
- * these same names</li>
- * <li>Heap saved if the file name objects are reused</li>
- */
-@InterfaceAudience.Private
-public class NameDistributionVisitor extends TextWriterImageVisitor {
-  final HashMap<String, Integer> counts = new HashMap<String, Integer>();
-
-  public NameDistributionVisitor(String filename, boolean printToScreen)
-      throws IOException {
-    super(filename, printToScreen);
-  }
-
-  @Override
-  void finish() throws IOException {
-    final int BYTEARRAY_OVERHEAD = 24;
-
-    write("Total unique file names " + counts.size());
-    // Columns: Frequency of file occurrence, savings in heap, total files using
-    // the name and number of file names
-    final long stats[][] = { { 100000, 0, 0, 0 },
-                             { 10000, 0, 0, 0 },
-                             { 1000, 0, 0, 0 },
-                             { 100, 0, 0, 0 },
-                             { 10, 0, 0, 0 },
-                             { 5, 0, 0, 0 },
-                             { 4, 0, 0, 0 },
-                             { 3, 0, 0, 0 },
-                             { 2, 0, 0, 0 }};
-
-    int highbound = Integer.MIN_VALUE;
-    for (Entry<String, Integer> entry : counts.entrySet()) {
-      highbound = Math.max(highbound, entry.getValue());
-      for (int i = 0; i < stats.length; i++) {
-        if (entry.getValue() >= stats[i][0]) {
-          stats[i][1] += (BYTEARRAY_OVERHEAD + entry.getKey().length())
-              * (entry.getValue() - 1);
-          stats[i][2] += entry.getValue();
-          stats[i][3]++;
-          break;
-        }
-      }
-    }
-
-    long lowbound = 0;
-    long totalsavings = 0;
-    for (long[] stat : stats) {
-      lowbound = stat[0];
-      totalsavings += stat[1];
-      String range = lowbound == highbound ? " " + lowbound :
-          " between " + lowbound + "-" + highbound;
-      write("\n" + stat[3] + " names are used by " + stat[2] + " files"
-          + range + " times. Heap savings ~" + stat[1] + " bytes.");
-      highbound = (int) stat[0] - 1;
-    }
-    write("\n\nTotal saved heap ~" + totalsavings + "bytes.\n");
-    super.finish();
-  }
-
-  @Override
-  void visit(ImageElement element, String value) throws IOException {
-    if (element == ImageElement.INODE_PATH) {
-      String filename = value.substring(value.lastIndexOf("/") + 1);
-      if (counts.containsKey(filename)) {
-        counts.put(filename, counts.get(filename) + 1);
-      } else {
-        counts.put(filename, 1);
-      }
-    }
-  }
-
-  @Override
-  void leaveEnclosingElement() throws IOException {
-  }
-
-  @Override
-  void start() throws IOException {
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element) throws IOException {
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element, ImageElement key,
-      String value) throws IOException {
-  }
-}

+ 0 - 274
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java

@@ -1,274 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.cli.PosixParser;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.PositionTrackingInputStream;
-
-/**
- * OfflineImageViewer to dump the contents of an Hadoop image file to XML
- * or the console.  Main entry point into utility, either via the
- * command line or programatically.
- */
-@InterfaceAudience.Private
-public class OfflineImageViewer {
-  public static final Log LOG = LogFactory.getLog(OfflineImageViewer.class);
-  
-  private final static String usage = 
-    "Usage: bin/hdfs oiv [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" +
-    "    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;
-  private final ImageVisitor processor;
-  
-  public OfflineImageViewer(String inputFile, ImageVisitor processor, 
-             boolean skipBlocks) {
-    this.inputFile = inputFile;
-    this.processor = processor;
-    this.skipBlocks = skipBlocks;
-  }
-
-  /**
-   * Process image file.
-   */
-  public void go() throws IOException  {
-    DataInputStream in = null;
-    PositionTrackingInputStream tracker = null;
-    ImageLoader fsip = null;
-    boolean done = false;
-    try {
-      tracker = new PositionTrackingInputStream(new BufferedInputStream(
-               new FileInputStream(new File(inputFile))));
-      in = new DataInputStream(tracker);
-
-      int imageVersionFile = findImageVersion(in);
-
-      fsip = ImageLoader.LoaderFactory.getLoader(imageVersionFile);
-
-      if(fsip == null) 
-        throw new IOException("No image processor to read version " +
-            imageVersionFile + " is available.");
-      fsip.loadImage(in, processor, skipBlocks);
-      done = true;
-    } finally {
-      if (!done) {
-        LOG.error("image loading failed at offset " + tracker.getPos());
-      }
-      IOUtils.cleanup(LOG, in, tracker);
-    }
-  }
-
-  /**
-   * Check an fsimage datainputstream's version number.
-   *
-   * The datainput stream is returned at the same point as it was passed in;
-   * this method has no effect on the datainputstream's read pointer.
-   *
-   * @param in Datainputstream of fsimage
-   * @return Filesystem layout version of fsimage represented by stream
-   * @throws IOException If problem reading from in
-   */
-  private int findImageVersion(DataInputStream in) throws IOException {
-    in.mark(42); // arbitrary amount, resetting immediately
-
-    int version = in.readInt();
-    in.reset();
-
-    return version;
-  }
-  
-  /**
-   * Build command-line options and descriptions
-   */
-  public static Options buildOptions() {
-    Options options = new Options();
-
-    // Build in/output file arguments, which are required, but there is no 
-    // addOption method that can specify this
-    OptionBuilder.isRequired();
-    OptionBuilder.hasArgs();
-    OptionBuilder.withLongOpt("outputFile");
-    options.addOption(OptionBuilder.create("o"));
-    
-    OptionBuilder.isRequired();
-    OptionBuilder.hasArgs();
-    OptionBuilder.withLongOpt("inputFile");
-    options.addOption(OptionBuilder.create("i"));
-    
-    options.addOption("p", "processor", true, "");
-    options.addOption("h", "help", false, "");
-    options.addOption("skipBlocks", false, "");
-    options.addOption("printToScreen", false, "");
-    options.addOption("delimiter", true, "");
-
-    return options;
-  }
-  
-  /**
-   * Entry point to command-line-driven operation.  User may specify
-   * options and start fsimage viewer from the command line.  Program
-   * will process image file and exit cleanly or, if an error is
-   * encountered, inform user and exit.
-   *
-   * @param args Command line options
-   * @throws IOException 
-   */
-  public static void main(String[] args) throws IOException {
-    Options options = buildOptions();
-    if(args.length == 0) {
-      printUsage();
-      return;
-    }
-    
-    CommandLineParser parser = new PosixParser();
-    CommandLine cmd;
-
-    try {
-      cmd = parser.parse(options, args);
-    } catch (ParseException e) {
-      System.out.println("Error parsing command-line options: ");
-      printUsage();
-      return;
-    }
-
-    if(cmd.hasOption("h")) { // print help and exit
-      printUsage();
-      return;
-    }
-
-    boolean skipBlocks = cmd.hasOption("skipBlocks");
-    boolean printToScreen = cmd.hasOption("printToScreen");
-    String inputFile = cmd.getOptionValue("i");
-    String processor = cmd.getOptionValue("p", "Ls");
-    String outputFile = cmd.getOptionValue("o");
-    String delimiter = cmd.getOptionValue("delimiter");
-    
-    if( !(delimiter == null || processor.equals("Delimited")) ) {
-      System.out.println("Can only specify -delimiter with Delimited processor");
-      printUsage();
-      return;
-    }
-    
-    ImageVisitor v;
-    if(processor.equals("Indented")) {
-      v = new IndentedImageVisitor(outputFile, printToScreen);
-    } else if (processor.equals("XML")) {
-      v = new XmlImageVisitor(outputFile, printToScreen);
-    } else if (processor.equals("Delimited")) {
-      v = delimiter == null ?  
-                 new DelimitedImageVisitor(outputFile, printToScreen) :
-                 new DelimitedImageVisitor(outputFile, printToScreen, delimiter);
-      skipBlocks = false;
-    } 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);
-    } else if (processor.equals("NameDistribution")) {
-      v = new NameDistributionVisitor(outputFile, printToScreen);
-    } else {
-      v = new LsImageVisitor(outputFile, printToScreen);
-      skipBlocks = false;
-    }
-    
-    try {
-      OfflineImageViewer d = new OfflineImageViewer(inputFile, v, skipBlocks);
-      d.go();
-    } catch (EOFException e) {
-      System.err.println("Input file ended unexpectedly.  Exiting");
-    } catch(IOException e) {
-      System.err.println("Encountered exception.  Exiting: " + e.getMessage());
-    }
-  }
-
-  /**
-   * Print application usage instructions.
-   */
-  private static void printUsage() {
-    System.out.println(usage);
-  }
-}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java

@@ -36,8 +36,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
 
 /**
- * OfflineImageViewer to dump the contents of an Hadoop image file to XML or the
- * console. Main entry point into utility, either via the command line or
+ * OfflineImageViewerPB to dump the contents of an Hadoop image file to XML or
+ * the console. Main entry point into utility, either via the command line or
  * programatically.
  */
 @InterfaceAudience.Private

+ 0 - 109
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TextWriterImageVisitor.java

@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-
-import com.google.common.base.Charsets;
-
-/**
- * TextWriterImageProcessor mixes in the ability for ImageVisitor
- * implementations to easily write their output to a text file.
- *
- * Implementing classes should be sure to call the super methods for the
- * constructors, finish and finishAbnormally methods, in order that the
- * underlying file may be opened and closed correctly.
- *
- * Note, this class does not add newlines to text written to file or (if
- * enabled) screen.  This is the implementing class' responsibility.
- */
-abstract class TextWriterImageVisitor extends ImageVisitor {
-  private boolean printToScreen = false;
-  private boolean okToWrite = false;
-  final private OutputStreamWriter fw;
-
-  /**
-   * Create a processor that writes to the file named.
-   *
-   * @param filename Name of file to write output to
-   */
-  public TextWriterImageVisitor(String filename) throws IOException {
-    this(filename, false);
-  }
-
-  /**
-   * Create a processor that writes to the file named and may or may not
-   * also output to the screen, as specified.
-   *
-   * @param filename Name of file to write output to
-   * @param printToScreen Mirror output to screen?
-   */
-  public TextWriterImageVisitor(String filename, boolean printToScreen)
-         throws IOException {
-    super();
-    this.printToScreen = printToScreen;
-    fw = new OutputStreamWriter(new FileOutputStream(filename), Charsets.UTF_8);
-    okToWrite = true;
-  }
-  
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor#finish()
-   */
-  @Override
-  void finish() throws IOException {
-    close();
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor#finishAbnormally()
-   */
-  @Override
-  void finishAbnormally() throws IOException {
-    close();
-  }
-
-  /**
-   * Close output stream and prevent further writing
-   */
-  private void close() throws IOException {
-    fw.close();
-    okToWrite = false;
-  }
-
-  /**
-   * Write parameter to output file (and possibly screen).
-   *
-   * @param toWrite Text to write to file
-   */
-  protected void write(String toWrite) throws IOException  {
-    if(!okToWrite)
-      throw new IOException("file not open for writing.");
-
-    if(printToScreen)
-      System.out.print(toWrite);
-
-    try {
-      fw.write(toWrite);
-    } catch (IOException e) {
-      okToWrite = false;
-      throw e;
-    }
-  }
-}

+ 0 - 88
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/XmlImageVisitor.java

@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.IOException;
-import java.util.LinkedList;
-
-/**
- * An XmlImageVisitor walks over an fsimage structure and writes out
- * an equivalent XML document that contains the fsimage's components.
- */
-public class XmlImageVisitor extends TextWriterImageVisitor {
-  final private LinkedList<ImageElement> tagQ =
-                                          new LinkedList<ImageElement>();
-
-  public XmlImageVisitor(String filename) throws IOException {
-    super(filename, false);
-  }
-
-  public XmlImageVisitor(String filename, boolean printToScreen)
-       throws IOException {
-    super(filename, printToScreen);
-  }
-
-  @Override
-  void finish() throws IOException {
-    super.finish();
-  }
-
-  @Override
-  void finishAbnormally() throws IOException {
-    write("\n<!-- Error processing image file.  Exiting -->\n");
-    super.finishAbnormally();
-  }
-
-  @Override
-  void leaveEnclosingElement() throws IOException {
-    if(tagQ.size() == 0)
-      throw new IOException("Tried to exit non-existent enclosing element " +
-                "in FSImage file");
-
-    ImageElement element = tagQ.pop();
-    write("</" + element.toString() + ">\n");
-  }
-
-  @Override
-  void start() throws IOException {
-    write("<?xml version=\"1.0\" ?>\n");
-  }
-
-  @Override
-  void visit(ImageElement element, String value) throws IOException {
-    writeTag(element.toString(), value);
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element) throws IOException {
-    write("<" + element.toString() + ">\n");
-    tagQ.push(element);
-  }
-
-  @Override
-  void visitEnclosingElement(ImageElement element,
-      ImageElement key, String value)
-       throws IOException {
-    write("<" + element.toString() + " " + key + "=\"" + value +"\">\n");
-    tagQ.push(element);
-  }
-
-  private void writeTag(String tag, String value) throws IOException {
-    write("<" + tag + ">" + value + "</" + tag + ">\n");
-  }
-}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -243,7 +243,7 @@ public class TestSnapshot {
   }
   
   /**
-   * Test if the OfflineImageViewer can correctly parse a fsimage containing
+   * Test if the OfflineImageViewerPB can correctly parse a fsimage containing
    * snapshots
    */
   @Test

+ 0 - 101
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestDelimitedImageVisitor.java

@@ -1,101 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
-import org.apache.hadoop.test.PathUtils;
-import org.junit.Test;
-
-/**
- * Test that the DelimitedImageVisistor gives the expected output based
- * on predetermined inputs
- */
-public class TestDelimitedImageVisitor {
-  private static final String ROOT = PathUtils.getTestDirName(TestDelimitedImageVisitor.class);
-  private static final String delim = "--";
-  
-  // Record an element in the visitor and build the expected line in the output
-  private void build(DelimitedImageVisitor div, ImageElement elem, String val, 
-                     StringBuilder sb, boolean includeDelim) throws IOException {
-    div.visit(elem, val);
-    sb.append(val);
-    
-    if(includeDelim)
-      sb.append(delim);
-  }
-  
-  @Test
-  public void testDelimitedImageVisistor() {
-    String filename = ROOT + "/testDIV";
-    File f = new File(filename);
-    BufferedReader br = null;
-    StringBuilder sb = new StringBuilder();
-    
-    try {
-      DelimitedImageVisitor div = new DelimitedImageVisitor(filename, true, delim);
-
-      div.visit(ImageElement.FS_IMAGE, "Not in ouput");
-      div.visitEnclosingElement(ImageElement.INODE);
-      div.visit(ImageElement.LAYOUT_VERSION, "not in");
-      div.visit(ImageElement.LAYOUT_VERSION, "the output");
-      
-      build(div, ImageElement.INODE_PATH,        "hartnell", sb, true);
-      build(div, ImageElement.REPLICATION,       "99", sb, true);
-      build(div, ImageElement.MODIFICATION_TIME, "troughton", sb, true);
-      build(div, ImageElement.ACCESS_TIME,       "pertwee", sb, true);
-      build(div, ImageElement.BLOCK_SIZE,        "baker", sb, true);
-      build(div, ImageElement.NUM_BLOCKS,        "davison", sb, true);
-      build(div, ImageElement.NUM_BYTES,         "55", sb, true);
-      build(div, ImageElement.NS_QUOTA,          "baker2", sb, true);
-      build(div, ImageElement.DS_QUOTA,          "mccoy", sb, true);
-      build(div, ImageElement.PERMISSION_STRING, "eccleston", sb, true);
-      build(div, ImageElement.USER_NAME,         "tennant", sb, true);
-      build(div, ImageElement.GROUP_NAME,        "smith", sb, false);
-      
-      div.leaveEnclosingElement(); // INode
-      div.finish();
-      
-      br = new BufferedReader(new FileReader(f));
-      String actual = br.readLine();
-      
-      // Should only get one line
-      assertNull(br.readLine());
-      br.close();
-      
-      String exepcted = sb.toString();
-      System.out.println("Expect to get: " + exepcted);
-      System.out.println("Actually got:  " + actual);
-      assertEquals(exepcted, actual);
-      
-    } catch (IOException e) {
-      fail("Error while testing delmitedImageVisitor" + e.getMessage());
-    } finally {
-      if(f.exists())
-        f.delete();
-    }
-  }
-}

+ 0 - 135
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOIVCanReadOldVersions.java

@@ -1,135 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.hadoop.hdfs.tools.offlineImageViewer.SpotCheckImageVisitor.ImageInfo;
-import org.junit.Test;
-
-public class TestOIVCanReadOldVersions {
-  // Location of fsimage files during testing.
-  public static final String TEST_CACHE_DATA_DIR =
-    System.getProperty("test.cache.data", "build/test/cache");
-  
-  // Verify that the image processor can correctly process prior Hadoop
-  // layout versions.  These fsimages were previously generated and stored
-  // with the test.  Test success indicates that no changes have been made
-  // to the OIV that causes older fsimages to be incorrectly processed.
-  @Test
-  public void testOldFSImages() {
-    // Define the expected values from the prior versions, as they were created
-    // and verified at time of creation
-    Set<String> pathNames = new HashSet<String>();
-    Collections.addAll(pathNames, "", /* root */
-                                  "/bar",
-                                  "/bar/dir0",
-                                  "/bar/dir0/file0",
-                                  "/bar/dir0/file1",
-                                  "/bar/dir1",
-                                  "/bar/dir1/file0",
-                                  "/bar/dir1/file1",
-                                  "/bar/dir2",
-                                  "/bar/dir2/file0",
-                                  "/bar/dir2/file1",
-                                  "/foo",
-                                  "/foo/dir0",
-                                  "/foo/dir0/file0",
-                                  "/foo/dir0/file1",
-                                  "/foo/dir0/file2",
-                                  "/foo/dir0/file3",
-                                  "/foo/dir1",
-                                  "/foo/dir1/file0",
-                                  "/foo/dir1/file1",
-                                  "/foo/dir1/file2",
-                                  "/foo/dir1/file3");
-    
-    Set<String> INUCpaths = new HashSet<String>();
-    Collections.addAll(INUCpaths, "/bar/dir0/file0",
-                                  "/bar/dir0/file1",
-                                  "/bar/dir1/file0",
-                                  "/bar/dir1/file1",
-                                  "/bar/dir2/file0",
-                                  "/bar/dir2/file1");
-    
-    ImageInfo v18Inodes = new ImageInfo(); // Hadoop version 18 inodes
-    v18Inodes.totalNumBlocks = 12;
-    v18Inodes.totalFileSize = 1069548540l;
-    v18Inodes.pathNames = pathNames;
-    v18Inodes.totalReplications = 14;
-    
-    ImageInfo v18INUCs = new ImageInfo(); // Hadoop version 18 inodes under construction
-    v18INUCs.totalNumBlocks = 0;
-    v18INUCs.totalFileSize = 0;
-    v18INUCs.pathNames = INUCpaths;
-    v18INUCs.totalReplications = 6;
-    
-    ImageInfo v19Inodes = new ImageInfo(); // Hadoop version 19 inodes
-    v19Inodes.totalNumBlocks = 12;
-    v19Inodes.totalFileSize = 1069548540l;
-    v19Inodes.pathNames = pathNames;
-    v19Inodes.totalReplications = 14;
-    
-    ImageInfo v19INUCs = new ImageInfo(); // Hadoop version 19 inodes under construction
-    v19INUCs.totalNumBlocks = 0;
-    v19INUCs.totalFileSize = 0;
-    v19INUCs.pathNames = INUCpaths;
-    v19INUCs.totalReplications = 6;
-    
-
-    spotCheck("18", TEST_CACHE_DATA_DIR + "/fsimageV18", v18Inodes, v18INUCs);
-    spotCheck("19", TEST_CACHE_DATA_DIR + "/fsimageV19", v19Inodes, v19INUCs);
-  }
-
-  // Check that running the processor now gives us the same values as before
-  private void spotCheck(String hadoopVersion, String input, 
-       ImageInfo inodes, ImageInfo INUCs) {
-    SpotCheckImageVisitor v = new SpotCheckImageVisitor();
-    OfflineImageViewer oiv = new OfflineImageViewer(input, v, false);
-    try {
-      oiv.go();
-    } catch (IOException e) {
-      fail("Error processing file: " + input);
-    }
-
-    compareSpotCheck(hadoopVersion, v.getINodesInfo(), inodes);
-    compareSpotCheck(hadoopVersion, v.getINUCsInfo(), INUCs);
-    System.out.println("Successfully processed fsimage file from Hadoop version " +
-                                                    hadoopVersion);
-  }
-
-  // Compare the spot check results of what we generated from the image
-  // processor and what we expected to receive
-  private void compareSpotCheck(String hadoopVersion, 
-                     ImageInfo generated, ImageInfo expected) {
-    assertEquals("Version " + hadoopVersion + ": Same number of total blocks", 
-                     expected.totalNumBlocks, generated.totalNumBlocks);
-    assertEquals("Version " + hadoopVersion + ": Same total file size", 
-                     expected.totalFileSize, generated.totalFileSize);
-    assertEquals("Version " + hadoopVersion + ": Same total replication factor", 
-                     expected.totalReplications, generated.totalReplications);
-    assertEquals("Version " + hadoopVersion + ": One-to-one matching of path names", 
-                     expected.pathNames, generated.pathNames);
-  }
-}

+ 1 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -64,16 +64,8 @@ import org.xml.sax.helpers.DefaultHandler;
 
 import com.google.common.collect.Maps;
 
-/**
- * Test function of OfflineImageViewer by: * confirming it can correctly process
- * a valid fsimage file and that the processing generates a correct
- * representation of the namespace * confirming it correctly fails to process an
- * fsimage file with a layout version it shouldn't be able to handle * confirm
- * it correctly bails on malformed image files, in particular, a file that ends
- * suddenly.
- */
 public class TestOfflineImageViewer {
-  private static final Log LOG = LogFactory.getLog(OfflineImageViewer.class);
+  private static final Log LOG = LogFactory.getLog(OfflineImageViewerPB.class);
   private static final int NUM_DIRS = 3;
   private static final int FILES_PER_DIR = 4;
   private static final String TEST_RENEWER = "JobTracker";