Browse Source

HDFS-4432. Support INodeFileUnderConstructionWithSnapshot in FSImage saving/loading. Contributed by Jing Zhao.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1439682 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 years ago
parent
commit
5988208b7d

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt

@@ -127,3 +127,6 @@ Branch-2802 Snapshot (Unreleased)
 
 
   HDFS-4441. Move INodeDirectoryWithSnapshot.Diff and the related classes to a
   HDFS-4441. Move INodeDirectoryWithSnapshot.Diff and the related classes to a
   package.  (szetszwo)
   package.  (szetszwo)
+
+  HDFS-4432. Support INodeFileUnderConstructionWithSnapshot in FSImage
+  saving/loading. (Jing Zhao via suresh)

+ 34 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -111,8 +112,12 @@ import org.apache.hadoop.io.Text;
  *     [list of BlockInfo] (when {@link Feature#SNAPSHOT} is not supported or 
  *     [list of BlockInfo] (when {@link Feature#SNAPSHOT} is not supported or 
  *     containsBlock is true),
  *     containsBlock is true),
  *     {
  *     {
- *       snapshotFileSize: long,
- *       isINodeFileWithLink: byte (if ComputedFileSize is negative),
+ *       snapshotFileSize: long (negative is the file is not a snapshot copy),
+ *       isINodeFileUnderConstructionSnapshot: byte (if snapshotFileSize 
+ *       is positive), 
+ *       {clientName: short + byte[], clientMachine: short + byte[]} (when 
+ *       isINodeFileUnderConstructionSnapshot is true),
+ *       isINodeFileWithSnapshot: byte (if snapshotFileSize is negative),
  *     } (when {@link Feature#SNAPSHOT} is supported), 
  *     } (when {@link Feature#SNAPSHOT} is supported), 
  *     fsPermission: short, PermissionStatus
  *     fsPermission: short, PermissionStatus
  *   } for INodeFile
  *   } for INodeFile
@@ -236,6 +241,8 @@ public class FSImageFormat {
               "imgVersion " + imgVersion +
               "imgVersion " + imgVersion +
               " expected to be " + getLayoutVersion());
               " expected to be " + getLayoutVersion());
         }
         }
+        boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
+            imgVersion);
 
 
         // read namespaceID: first appeared in version -2
         // read namespaceID: first appeared in version -2
         in.readInt();
         in.readInt();
@@ -254,7 +261,7 @@ public class FSImageFormat {
           imgTxId = 0;
           imgTxId = 0;
         }
         }
         
         
-        if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+        if (supportSnapshot) {
           namesystem.getSnapshotManager().read(in);
           namesystem.getSnapshotManager().read(in);
         }
         }
 
 
@@ -274,7 +281,7 @@ public class FSImageFormat {
         LOG.info("Number of files = " + numFiles);
         LOG.info("Number of files = " + numFiles);
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
             imgVersion)) {
             imgVersion)) {
-          if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+          if (supportSnapshot) {
             loadLocalNameINodesWithSnapshot(in);
             loadLocalNameINodesWithSnapshot(in);
           } else {
           } else {
             loadLocalNameINodes(numFiles, in);
             loadLocalNameINodes(numFiles, in);
@@ -283,7 +290,7 @@ public class FSImageFormat {
           loadFullNameINodes(numFiles, in);
           loadFullNameINodes(numFiles, in);
         }
         }
 
 
-        loadFilesUnderConstruction(in);
+        loadFilesUnderConstruction(in, supportSnapshot);
 
 
         loadSecretManagerState(in);
         loadSecretManagerState(in);
 
 
@@ -558,8 +565,11 @@ public class FSImageFormat {
     int numBlocks = in.readInt();
     int numBlocks = in.readInt();
     BlockInfo blocks[] = null;
     BlockInfo blocks[] = null;
 
 
+    String clientName = "";
+    String clientMachine = "";
+    boolean underConstruction = false;
     if (numBlocks >= 0) {
     if (numBlocks >= 0) {
-      // to indicate INodeFileWithLink, blocks may be set as null while
+      // to indicate INodeFileWithSnapshot, blocks may be set as null while
       // numBlocks is set to 0
       // numBlocks is set to 0
       blocks = LayoutVersion.supports(Feature.SNAPSHOT, imgVersion) ? (in
       blocks = LayoutVersion.supports(Feature.SNAPSHOT, imgVersion) ? (in
             .readBoolean() ? new BlockInfo[numBlocks] : null)
             .readBoolean() ? new BlockInfo[numBlocks] : null)
@@ -573,6 +583,12 @@ public class FSImageFormat {
         computeFileSize = in.readLong();
         computeFileSize = in.readLong();
         if (computeFileSize < 0) {
         if (computeFileSize < 0) {
           withLink = in.readBoolean();
           withLink = in.readBoolean();
+        } else {
+          underConstruction = in.readBoolean();
+          if (underConstruction) {
+            clientName = FSImageSerialization.readString(in);
+            clientMachine = FSImageSerialization.readString(in);
+          }
         }
         }
       }
       }
     }
     }
@@ -603,13 +619,14 @@ public class FSImageFormat {
     
     
     PermissionStatus permissions = PermissionStatus.read(in);
     PermissionStatus permissions = PermissionStatus.read(in);
 
 
-    return INode.newINode(inodeId, permissions, blocks, symlink, replication,
-        modificationTime, atime, nsQuota, dsQuota, blockSize, numBlocks,
-        withLink, computeFileSize, snapshottable, withSnapshot);
+      return INode.newINode(inodeId, permissions, blocks, symlink, replication,
+          modificationTime, atime, nsQuota, dsQuota, blockSize, numBlocks,
+          withLink, computeFileSize, snapshottable, withSnapshot,
+          underConstruction, clientName, clientMachine);
   }
   }
 
 
-    private void loadFilesUnderConstruction(DataInputStream in)
-    throws IOException {
+    private void loadFilesUnderConstruction(DataInputStream in,
+        boolean supportSnapshot) throws IOException {
       FSDirectory fsDir = namesystem.dir;
       FSDirectory fsDir = namesystem.dir;
       int size = in.readInt();
       int size = in.readInt();
 
 
@@ -617,12 +634,17 @@ public class FSImageFormat {
 
 
       for (int i = 0; i < size; i++) {
       for (int i = 0; i < size; i++) {
         INodeFileUnderConstruction cons =
         INodeFileUnderConstruction cons =
-          FSImageSerialization.readINodeUnderConstruction(in);
+          FSImageSerialization.readINodeUnderConstruction(in, supportSnapshot);
 
 
         // verify that file exists in namespace
         // verify that file exists in namespace
         String path = cons.getLocalName();
         String path = cons.getLocalName();
         final INodesInPath iip = fsDir.getINodesInPath(path);
         final INodesInPath iip = fsDir.getINodesInPath(path);
         INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
         INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
+        cons.setLocalName(oldnode.getLocalNameBytes());
+        if (oldnode instanceof FileWithSnapshot
+            && cons instanceof FileWithSnapshot) {
+          ((FileWithSnapshot) oldnode).insertBefore((FileWithSnapshot) cons);
+        }
         fsDir.unprotectedReplaceINodeFile(path, oldnode, cons,
         fsDir.unprotectedReplaceINodeFile(path, oldnode, cons,
             iip.getLatestSnapshot());
             iip.getLatestSnapshot());
         namesystem.leaseManager.addLease(cons.getClientName(), path); 
         namesystem.leaseManager.addLease(cons.getClientName(), path); 

+ 28 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -33,10 +33,12 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileSnapshot;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -84,7 +86,8 @@ public class FSImageSerialization {
   // from the input stream
   // from the input stream
   //
   //
   static INodeFileUnderConstruction readINodeUnderConstruction(
   static INodeFileUnderConstruction readINodeUnderConstruction(
-                            DataInputStream in) throws IOException {
+      DataInputStream in, boolean supportSnapshot) throws IOException {
+    boolean withSnapshot = false;
     byte[] name = readBytes(in);
     byte[] name = readBytes(in);
     short blockReplication = in.readShort();
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long modificationTime = in.readLong();
@@ -103,6 +106,9 @@ public class FSImageSerialization {
       blocks[i] = new BlockInfoUnderConstruction(
       blocks[i] = new BlockInfoUnderConstruction(
         blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
         blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
     }
     }
+    if (supportSnapshot) {
+      withSnapshot = in.readBoolean();
+    }
     PermissionStatus perm = PermissionStatus.read(in);
     PermissionStatus perm = PermissionStatus.read(in);
     String clientName = readString(in);
     String clientName = readString(in);
     String clientMachine = readString(in);
     String clientMachine = readString(in);
@@ -113,16 +119,11 @@ public class FSImageSerialization {
     assert numLocs == 0 : "Unexpected block locations";
     assert numLocs == 0 : "Unexpected block locations";
 
 
     //TODO: get inodeId from fsimage after inodeId is persisted
     //TODO: get inodeId from fsimage after inodeId is persisted
-    return new INodeFileUnderConstruction(INodeId.GRANDFATHER_INODE_ID,
-                                          name,
-                                          blockReplication, 
-                                          modificationTime,
-                                          preferredBlockSize,
-                                          blocks,
-                                          perm,
-                                          clientName,
-                                          clientMachine,
-                                          null);
+    INodeFileUnderConstruction node = new INodeFileUnderConstruction(
+        INodeId.GRANDFATHER_INODE_ID, name, blockReplication, modificationTime,
+        preferredBlockSize, blocks, perm, clientName, clientMachine, null);
+    return withSnapshot ? new INodeFileUnderConstructionWithSnapshot(node)
+        : node;
   }
   }
 
 
   // Helper function that writes an INodeUnderConstruction
   // Helper function that writes an INodeUnderConstruction
@@ -141,6 +142,7 @@ public class FSImageSerialization {
     for (int i = 0; i < nrBlocks; i++) {
     for (int i = 0; i < nrBlocks; i++) {
       cons.getBlocks()[i].write(out);
       cons.getBlocks()[i].write(out);
     }
     }
+    out.writeBoolean(cons instanceof INodeFileUnderConstructionWithSnapshot);
     cons.getPermissionStatus().write(out);
     cons.getPermissionStatus().write(out);
     writeString(cons.getClientName(), out);
     writeString(cons.getClientName(), out);
     writeString(cons.getClientMachine(), out);
     writeString(cons.getClientMachine(), out);
@@ -204,10 +206,10 @@ public class FSImageSerialization {
   /**
   /**
    * Serialize a {@link INodeFile} node
    * Serialize a {@link INodeFile} node
    * @param node The node to write
    * @param node The node to write
-   * @param out The {@link DataOutput} where the fields are written
+   * @param out The {@link DataOutputStream} where the fields are written
    * @param writeBlock Whether to write block information
    * @param writeBlock Whether to write block information
    */
    */
-  public static void writeINodeFile(INodeFile node, DataOutput out,
+  public static void writeINodeFile(INodeFile node, DataOutputStream out,
       boolean writeBlock) throws IOException {
       boolean writeBlock) throws IOException {
     byte[] name = node.getLocalNameBytes();
     byte[] name = node.getLocalNameBytes();
     out.writeShort(name.length);
     out.writeShort(name.length);
@@ -227,11 +229,19 @@ public class FSImageSerialization {
       out.writeInt(0); // # of blocks
       out.writeInt(0); // # of blocks
       out.writeBoolean(false);
       out.writeBoolean(false);
     }
     }
-    if (node instanceof INodeFileSnapshot) {
-      out.writeLong(((INodeFileSnapshot) node).computeFileSize(true));
+    if (node instanceof INodeFileSnapshot
+        || node instanceof INodeFileUnderConstructionSnapshot) {
+      out.writeLong(node.computeFileSize(true));
+      if (node instanceof INodeFileUnderConstructionSnapshot) {
+        out.writeBoolean(true);
+        writeString(((INodeFileUnderConstruction) node).getClientName(), out);
+        writeString(((INodeFileUnderConstruction) node).getClientMachine(), out);
+      } else {
+        out.writeBoolean(false);
+      }
     } else {
     } else {
       out.writeLong(-1);
       out.writeLong(-1);
-      out.writeBoolean(node instanceof INodeFileWithSnapshot);
+      out.writeBoolean(node instanceof FileWithSnapshot);
     }
     }
     FsPermission filePerm = TL_DATA.get().FILE_PERM;
     FsPermission filePerm = TL_DATA.get().FILE_PERM;
     filePerm.fromShort(fileINode.getFsPermissionShort());
     filePerm.fromShort(fileINode.getFsPermissionShort());
@@ -243,7 +253,7 @@ public class FSImageSerialization {
   /**
   /**
    * Save one inode's attributes to the image.
    * Save one inode's attributes to the image.
    */
    */
-  static void saveINode2Image(INode node, DataOutput out)
+  static void saveINode2Image(INode node, DataOutputStream out)
       throws IOException {
       throws IOException {
     if (node.isDirectory()) {
     if (node.isDirectory()) {
       writeINodeDirectory((INodeDirectory) node, out);
       writeINodeDirectory((INodeDirectory) node, out);

+ 16 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.diff.Diff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.diff.Diff;
@@ -623,14 +624,20 @@ public abstract class INode implements Diff.Element<byte[]> {
    * @param computeFileSize non-negative computeFileSize means the node is 
    * @param computeFileSize non-negative computeFileSize means the node is 
    *                        INodeFileSnapshot
    *                        INodeFileSnapshot
    * @param snapshottable whether the node is {@link INodeDirectorySnapshottable}
    * @param snapshottable whether the node is {@link INodeDirectorySnapshottable}
-   * @param withSnapshot whether the node is {@link INodeDirectoryWithSnapshot}                       
+   * @param withSnapshot whether the node is {@link INodeDirectoryWithSnapshot}
+   * @param underConstruction whether the node is 
+   *                          {@link INodeFileUnderConstructionSnapshot}
+   * @param clientName clientName of {@link INodeFileUnderConstructionSnapshot}
+   * @param clientMachine clientMachine of 
+   *                      {@link INodeFileUnderConstructionSnapshot}
    * @return an inode
    * @return an inode
    */
    */
   static INode newINode(long id, PermissionStatus permissions,
   static INode newINode(long id, PermissionStatus permissions,
       BlockInfo[] blocks, String symlink, short replication,
       BlockInfo[] blocks, String symlink, short replication,
       long modificationTime, long atime, long nsQuota, long dsQuota,
       long modificationTime, long atime, long nsQuota, long dsQuota,
       long preferredBlockSize, int numBlocks, boolean withLink,
       long preferredBlockSize, int numBlocks, boolean withLink,
-      long computeFileSize, boolean snapshottable, boolean withSnapshot) {
+      long computeFileSize, boolean snapshottable, boolean withSnapshot, 
+      boolean underConstruction, String clientName, String clientMachine) {
     if (symlink.length() != 0) { // check if symbolic link
     if (symlink.length() != 0) { // check if symbolic link
       return new INodeSymlink(id, symlink, modificationTime, atime, permissions);
       return new INodeSymlink(id, symlink, modificationTime, atime, permissions);
     }  else if (blocks == null && numBlocks < 0) { 
     }  else if (blocks == null && numBlocks < 0) { 
@@ -650,9 +657,13 @@ public abstract class INode implements Diff.Element<byte[]> {
     // file
     // file
     INodeFile fileNode = new INodeFile(id, permissions, blocks, replication,
     INodeFile fileNode = new INodeFile(id, permissions, blocks, replication,
         modificationTime, atime, preferredBlockSize);
         modificationTime, atime, preferredBlockSize);
-    return computeFileSize >= 0 ? new INodeFileSnapshot(fileNode,
-        computeFileSize) : (withLink ? new INodeFileWithSnapshot(fileNode)
-        : fileNode);
+    if (computeFileSize >= 0) {
+      return underConstruction ? new INodeFileUnderConstructionSnapshot(
+          fileNode, computeFileSize, clientName, clientMachine)
+          : new INodeFileSnapshot(fileNode, computeFileSize); 
+    } else {
+      return withLink ? new INodeFileWithSnapshot(fileNode) : fileNode;
+    }
   }
   }
 
 
   /**
   /**

+ 10 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
-import java.util.Arrays;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -54,6 +53,14 @@ public class INodeFile extends INode implements BlockCollection {
 
 
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
 
 
+  /**
+   * Check the first block to see if two INodes are about the same file
+   */
+  public static boolean isOfSameFile(INodeFile file1, INodeFile file2) {
+    BlockInfo[] blk1 = file1.getBlocks();
+    BlockInfo[] blk2 = file2.getBlocks();
+    return blk1 != null && blk2 != null && blk1[0] == blk2[0];
+  }
 
 
   /** Format: [16 bits for replication][48 bits for PreferredBlockSize] */
   /** Format: [16 bits for replication][48 bits for PreferredBlockSize] */
   private static class HeaderFormat {
   private static class HeaderFormat {
@@ -332,7 +339,8 @@ public class INodeFile extends INode implements BlockCollection {
       final Snapshot snapshot) {
       final Snapshot snapshot) {
     super.dumpTreeRecursively(out, prefix, snapshot);
     super.dumpTreeRecursively(out, prefix, snapshot);
     out.print(", fileSize=" + computeFileSize(true));
     out.print(", fileSize=" + computeFileSize(true));
-    out.print(", blocks=" + (blocks == null? null: Arrays.asList(blocks)));
+    // only compare the first block
+    out.print(", blocks=" + (blocks == null? null: blocks[0]));
     if (this instanceof FileWithSnapshot) {
     if (this instanceof FileWithSnapshot) {
       final FileWithSnapshot withSnapshot = (FileWithSnapshot) this;
       final FileWithSnapshot withSnapshot = (FileWithSnapshot) this;
       final FileWithSnapshot next = withSnapshot.getNext();
       final FileWithSnapshot next = withSnapshot.getNext();

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java

@@ -17,7 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 
-import java.io.DataOutput;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collections;
@@ -57,7 +57,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     }
 
 
     /** Serialize {@link #created} */
     /** Serialize {@link #created} */
-    private void writeCreated(DataOutput out) throws IOException {
+    private void writeCreated(DataOutputStream out) throws IOException {
         final List<INode> created = getCreatedList();
         final List<INode> created = getCreatedList();
         out.writeInt(created.size());
         out.writeInt(created.size());
         for (INode node : created) {
         for (INode node : created) {
@@ -69,7 +69,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     }
     
     
     /** Serialize {@link #deleted} */
     /** Serialize {@link #deleted} */
-    private void writeDeleted(DataOutput out) throws IOException {
+    private void writeDeleted(DataOutputStream out) throws IOException {
         final List<INode> deleted = getDeletedList();
         final List<INode> deleted = getDeletedList();
         out.writeInt(deleted.size());
         out.writeInt(deleted.size());
         for (INode node : deleted) {
         for (INode node : deleted) {
@@ -90,7 +90,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
               // SnapshotDiff, we may put two inodes sharing the same name but
               // SnapshotDiff, we may put two inodes sharing the same name but
               // with totally different blocks in the created and deleted list of
               // with totally different blocks in the created and deleted list of
               // the same SnapshotDiff.
               // the same SnapshotDiff.
-              if (cNode.getBlocks() == dNode.getBlocks()) {
+              if (INodeFile.isOfSameFile(cNode, dNode)) {
                 FSImageSerialization.writeINodeFile(dNode, out, false);
                 FSImageSerialization.writeINodeFile(dNode, out, false);
               } else {
               } else {
                 FSImageSerialization.writeINodeFile(dNode, out, true);
                 FSImageSerialization.writeINodeFile(dNode, out, true);
@@ -101,7 +101,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     }
     
     
     /** Serialize to out */
     /** Serialize to out */
-    private void write(DataOutput out) throws IOException {
+    private void write(DataOutputStream out) throws IOException {
       writeCreated(out);
       writeCreated(out);
       writeDeleted(out);    
       writeDeleted(out);    
     }
     }
@@ -283,7 +283,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     }
     
     
     /** Serialize fields to out */
     /** Serialize fields to out */
-    void write(DataOutput out) throws IOException {
+    void write(DataOutputStream out) throws IOException {
       out.writeInt(childrenSize);
       out.writeInt(childrenSize);
       // No need to write all fields of Snapshot here, since the snapshot must
       // No need to write all fields of Snapshot here, since the snapshot must
       // have been recorded before when writing the FSImage. We only need to
       // have been recorded before when writing the FSImage. We only need to

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionSnapshot.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
 
 
 /**
 /**
@@ -34,6 +36,17 @@ public class INodeFileUnderConstructionSnapshot
     this.size = f.computeFileSize(true);
     this.size = f.computeFileSize(true);
     f.insertAfter(this);
     f.insertAfter(this);
   }
   }
+  
+  /**
+   * A constructor generating an {@link INodeFileUnderConstructionSnapshot}
+   * based on an {@link INodeFile}, the file size at the snapshot time, client
+   * name, and client machine. Used while loading {@link FSImage}
+   */
+  public INodeFileUnderConstructionSnapshot(INodeFile f, long size,
+      String clientName, String clientMachine) {
+    super(f, clientName, clientMachine, null);
+    this.size = size;
+  }
 
 
   @Override
   @Override
   public long computeFileSize(boolean includesBlockInfoUnderConstruction) {
   public long computeFileSize(boolean includesBlockInfoUnderConstruction) {

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java

@@ -133,8 +133,7 @@ public class SnapshotFSImageFormat {
       // 
       // 
       // For case 1), c and d should be both INodeFile and should share
       // For case 1), c and d should be both INodeFile and should share
       // the same blockInfo list.
       // the same blockInfo list.
-      if (c.isFile()
-          && ((INodeFile) c).getBlocks() == ((INodeFile) d).getBlocks()) {
+      if (c.isFile() && INodeFile.isOfSameFile((INodeFile) c, (INodeFile) d)) {
         return c;
         return c;
       } else {
       } else {
         return d;
         return d;
@@ -200,11 +199,12 @@ public class SnapshotFSImageFormat {
                   + DFSUtil.bytes2String(deletedNodeName)
                   + DFSUtil.bytes2String(deletedNodeName)
                   + " in deleted list while loading FSImage.");
                   + " in deleted list while loading FSImage.");
         }
         }
-        // deleted must be an INodeFileSnapshot
-        INodeFileSnapshot deletedWithLink = (INodeFileSnapshot) deleted;
+        // deleted must be an FileWithSnapshot (INodeFileSnapshot or 
+        // INodeFileUnderConstructionSnapshot)
+        FileWithSnapshot deletedWithLink = (FileWithSnapshot) deleted;
         INodeFile cNode = (INodeFile) createdList.get(c);
         INodeFile cNode = (INodeFile) createdList.get(c);
         INodeFileWithSnapshot cNodeWithLink = (INodeFileWithSnapshot) cNode;
         INodeFileWithSnapshot cNodeWithLink = (INodeFileWithSnapshot) cNode;
-        deletedWithLink.setBlocks(cNode.getBlocks());
+        ((INodeFile) deleted).setBlocks(cNode.getBlocks());
         // insert deleted into the circular list
         // insert deleted into the circular list
         cNodeWithLink.insertBefore(deletedWithLink);
         cNodeWithLink.insertBefore(deletedWithLink);
       }
       }

+ 142 - 35
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java

@@ -17,16 +17,22 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import static org.junit.Assert.assertEquals;
-
 import java.io.File;
 import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.EnumSet;
+import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
@@ -38,9 +44,10 @@ import org.junit.Test;
 public class TestFSImageWithSnapshot {
 public class TestFSImageWithSnapshot {
   static final long seed = 0;
   static final long seed = 0;
   static final short REPLICATION = 3;
   static final short REPLICATION = 3;
-  static final long BLOCKSIZE = 1024;
+  static final int BLOCKSIZE = 1024;
   static final long txid = 1;
   static final long txid = 1;
 
 
+  private final Path rootDir = new Path("/");
   private final Path dir = new Path("/TestSnapshot");
   private final Path dir = new Path("/TestSnapshot");
   private static String testDir =
   private static String testDir =
       System.getProperty("test.build.data", "build/test/data");
       System.getProperty("test.build.data", "build/test/data");
@@ -67,6 +74,81 @@ public class TestFSImageWithSnapshot {
     }
     }
   }
   }
   
   
+  /**
+   * Create a temp fsimage file for testing.
+   * @param dir The directory where the fsimage file resides
+   * @param imageTxId The transaction id of the fsimage
+   * @return The file of the image file
+   */
+  private File getImageFile(String dir, long imageTxId) {
+    return new File(dir, String.format("%s_%019d", NameNodeFile.IMAGE,
+        imageTxId));
+  }
+  
+  /** 
+   * Create a temp file for dumping the fsdir
+   * @param dir directory for the temp file
+   * @param suffix suffix of of the temp file
+   * @return the temp file
+   */
+  private File getDumpTreeFile(String dir, String suffix) {
+    return new File(dir, String.format("dumpTree_%s", suffix));
+  }
+  
+  /** 
+   * Dump the fsdir tree to a temp file
+   * @param fileSuffix suffix of the temp file for dumping
+   * @return the temp file
+   */
+  private File dumpTree2File(String fileSuffix) throws IOException {
+    File file = getDumpTreeFile(testDir, fileSuffix);
+    PrintWriter out = new PrintWriter(new FileWriter(file, false), true);
+    fsn.getFSDirectory().getINode(rootDir.toString())
+        .dumpTreeRecursively(out, new StringBuilder(), null);
+    out.close();
+    return file;
+  }
+  
+  /** Append a file without closing the output stream */
+  private HdfsDataOutputStream appendFileWithoutClosing(Path file, int length)
+      throws IOException {
+    byte[] toAppend = new byte[length];
+    Random random = new Random();
+    random.nextBytes(toAppend);
+    HdfsDataOutputStream out = (HdfsDataOutputStream) hdfs.append(file);
+    out.write(toAppend);
+    return out;
+  }
+  
+  /** Save the fsimage to a temp file */
+  private File saveFSImageToTempFile() throws IOException {
+    SaveNamespaceContext context = new SaveNamespaceContext(fsn, txid,
+        new Canceler());
+    FSImageFormat.Saver saver = new FSImageFormat.Saver(context);
+    FSImageCompression compression = FSImageCompression.createCompression(conf);
+    File imageFile = getImageFile(testDir, txid);
+    fsn.readLock();
+    try {
+      saver.save(imageFile, compression);
+    } finally {
+      fsn.readUnlock();
+    }
+    return imageFile;
+  }
+  
+  /** Load the fsimage from a temp file */
+  private void loadFSImageFromTempFile(File imageFile) throws IOException {
+    FSImageFormat.Loader loader = new FSImageFormat.Loader(conf, fsn);
+    fsn.writeLock();
+    fsn.getFSDirectory().writeLock();
+    try {
+      loader.load(imageFile);
+    } finally {
+      fsn.getFSDirectory().writeUnlock();
+      fsn.writeUnlock();
+    }
+  }
+  
   /**
   /**
    * Testing steps:
    * Testing steps:
    * <pre>
    * <pre>
@@ -106,21 +188,10 @@ public class TestFSImageWithSnapshot {
     hdfs.delete(sub2file2, true);
     hdfs.delete(sub2file2, true);
     
     
     // dump the fsdir tree
     // dump the fsdir tree
-    StringBuffer fsnStrBefore = fsn.getFSDirectory().rootDir
-        .dumpTreeRecursively();
+    File fsnBefore = dumpTree2File("before");
     
     
     // save the namesystem to a temp file
     // save the namesystem to a temp file
-    SaveNamespaceContext context = new SaveNamespaceContext(fsn, txid,
-        new Canceler());
-    FSImageFormat.Saver saver = new FSImageFormat.Saver(context);
-    FSImageCompression compression = FSImageCompression.createCompression(conf);
-    File dstFile = getStorageFile(testDir, txid);
-    fsn.readLock();
-    try {
-      saver.save(dstFile, compression);
-    } finally {
-      fsn.readUnlock();
-    }
+    File imageFile = saveFSImageToTempFile();
 
 
     // restart the cluster, and format the cluster
     // restart the cluster, and format the cluster
     cluster.shutdown();
     cluster.shutdown();
@@ -131,32 +202,68 @@ public class TestFSImageWithSnapshot {
     hdfs = cluster.getFileSystem();
     hdfs = cluster.getFileSystem();
     
     
     // load the namesystem from the temp file
     // load the namesystem from the temp file
-    FSImageFormat.Loader loader = new FSImageFormat.Loader(conf, fsn);
-    fsn.writeLock();
-    try {
-      loader.load(dstFile);
-    } finally {
-      fsn.writeUnlock();
-    }
+    loadFSImageFromTempFile(imageFile);
     
     
     // dump the fsdir tree again
     // dump the fsdir tree again
-    StringBuffer fsnStrAfter = fsn.getFSDirectory().rootDir
-        .dumpTreeRecursively();
+    File fsnAfter = dumpTree2File("after");
     
     
     // compare two dumped tree
     // compare two dumped tree
-    System.out.println(fsnStrBefore.toString());
-    System.out.println("\n" + fsnStrAfter.toString());
-    assertEquals(fsnStrBefore.toString(), fsnStrAfter.toString());
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter);
   }
   }
   
   
   /**
   /**
-   * Create a temp fsimage file for testing.
-   * @param dir The directory where the fsimage file resides
-   * @param imageTxId The transaction id of the fsimage
-   * @return The file of the image file
+   * Test the fsimage saving/loading while file appending.
    */
    */
-  private File getStorageFile(String dir, long imageTxId) {
-    return new File(dir, String.format("%s_%019d", NameNodeFile.IMAGE,
-        imageTxId));
+  @Test
+  public void testSaveLoadImageWithAppending() throws Exception {
+    Path sub1 = new Path(dir, "sub1");
+    Path sub1file1 = new Path(sub1, "sub1file1");
+    Path sub1file2 = new Path(sub1, "sub1file2");
+    DFSTestUtil.createFile(hdfs, sub1file1, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, sub1file2, BLOCKSIZE, REPLICATION, seed);
+    
+    // 1. create snapshot s0
+    hdfs.allowSnapshot(dir.toString());
+    hdfs.createSnapshot(dir, "s0");
+    
+    // 2. create snapshot s1 before appending sub1file1 finishes
+    HdfsDataOutputStream out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+    // also append sub1file2
+    DFSTestUtil.appendFile(hdfs, sub1file2, BLOCKSIZE);
+    hdfs.createSnapshot(dir, "s1");
+    out.close();
+    
+    // 3. create snapshot s2 before appending finishes
+    out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+    hdfs.createSnapshot(dir, "s2");
+    out.close();
+    
+    // 4. save fsimage before appending finishes
+    out = appendFileWithoutClosing(sub1file1, BLOCKSIZE);
+    out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+    // dump fsdir
+    File fsnBefore = dumpTree2File("before");
+    // save the namesystem to a temp file
+    File imageFile = saveFSImageToTempFile();
+    
+    // 5. load fsimage and compare
+    // first restart the cluster, and format the cluster
+    out.close();
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(true)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    // then load the fsimage
+    loadFSImageFromTempFile(imageFile);
+    
+    // dump the fsdir tree again
+    File fsnAfter = dumpTree2File("after");
+    
+    // compare two dumped tree
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter);
   }
   }
 }
 }

+ 83 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java

@@ -20,6 +20,10 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -35,7 +39,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.junit.Assert;
 
 
 /**
 /**
  * Helper for writing snapshot related tests
  * Helper for writing snapshot related tests
@@ -90,6 +99,80 @@ public class SnapshotTestHelper {
     FileStatus[] snapshotFiles = hdfs.listStatus(snapshotRoot);
     FileStatus[] snapshotFiles = hdfs.listStatus(snapshotRoot);
     assertEquals(currentFiles.length, snapshotFiles.length);
     assertEquals(currentFiles.length, snapshotFiles.length);
   }
   }
+  
+  /**
+   * Compare two dumped trees that are stored in two files. The following is an
+   * example of the dumped tree:
+   * 
+   * <pre>
+   * information of root
+   * +- the first child of root (e.g., /foo)
+   *   +- the first child of /foo
+   *   ...
+   *   \- the last child of /foo (e.g., /foo/bar)
+   *     +- the first child of /foo/bar
+   *     ...
+   *   snapshots of /foo
+   *   +- snapshot s_1
+   *   ...
+   *   \- snapshot s_n
+   * +- second child of root
+   *   ...
+   * \- last child of root
+   * 
+   * The following information is dumped for each inode:
+   * localName (className@hashCode) parent permission group user
+   * 
+   * Specific information for different types of INode: 
+   * {@link INodeDirectory}:childrenSize 
+   * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()} 
+   * and {@link BlockInfoUnderConstruction#toString()} for detailed information.
+   * {@link FileWithSnapshot}: next link
+   * </pre>
+   * @see INode#dumpTreeRecursively()
+   */
+  public static void compareDumpedTreeInFile(File file1, File file2)
+      throws IOException {
+    BufferedReader reader1 = new BufferedReader(new FileReader(file1));
+    BufferedReader reader2 = new BufferedReader(new FileReader(file2));
+    try {
+      String line1 = "";
+      String line2 = "";
+      while ((line1 = reader1.readLine()) != null
+          && (line2 = reader2.readLine()) != null) {
+        // skip the hashCode part of the object string during the comparison,
+        // also ignore the difference between INodeFile/INodeFileWithSnapshot
+        line1 = line1.replaceAll("INodeFileWithSnapshot", "INodeFile");
+        line2 = line2.replaceAll("INodeFileWithSnapshot", "INodeFile");
+        line1 = line1.replaceAll("@[\\dabcdef]+", "");
+        line2 = line2.replaceAll("@[\\dabcdef]+", "");
+        
+        // skip the replica field of the last block of an
+        // INodeFileUnderConstruction
+        line1 = line1.replaceAll("replicas=\\[.*\\]", "replicas=[]");
+        line2 = line2.replaceAll("replicas=\\[.*\\]", "replicas=[]");
+        
+        // skip the specific fields of BlockInfoUnderConstruction when the node
+        // is an INodeFileSnapshot or an INodeFileUnderConstructionSnapshot
+        if (line1.contains("(INodeFileSnapshot)")
+            || line1.contains("(INodeFileUnderConstructionSnapshot)")) {
+          line1 = line1.replaceAll(
+           "\\{blockUCState=\\w+, primaryNodeIndex=[-\\d]+, replicas=\\[\\]\\}",
+           "");
+          line2 = line2.replaceAll(
+           "\\{blockUCState=\\w+, primaryNodeIndex=[-\\d]+, replicas=\\[\\]\\}",
+           "");
+        }
+        
+        assertEquals(line1, line2);
+      }
+      Assert.assertNull(reader1.readLine());
+      Assert.assertNull(reader2.readLine());
+    } finally {
+      reader1.close();
+      reader2.close();
+    }
+  }
 
 
   /**
   /**
    * Generate the path for a snapshot file.
    * Generate the path for a snapshot file.

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

@@ -20,11 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
-import java.io.BufferedReader;
 import java.io.File;
 import java.io.File;
-import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.FileWriter;
-import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
@@ -33,12 +30,12 @@ import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -48,7 +45,6 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDi
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
@@ -94,6 +90,7 @@ public class TestSnapshot {
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     conf = new Configuration();
     conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
         .build();
         .build();
     cluster.waitActive();
     cluster.waitActive();
@@ -201,33 +198,8 @@ public class TestSnapshot {
         .dumpTreeRecursively(out, new StringBuilder(), null);
         .dumpTreeRecursively(out, new StringBuilder(), null);
     out.close();
     out.close();
     
     
-    compareFile(fsnBefore, fsnMiddle);
-    compareFile(fsnBefore, fsnAfter);
-  }
-  
-  /** compare two file's content */
-  private void compareFile(File file1, File file2) throws IOException {
-    BufferedReader reader1 = new BufferedReader(new FileReader(file1));
-    BufferedReader reader2 = new BufferedReader(new FileReader(file2));
-    try {
-      String line1 = "";
-      String line2 = "";
-      while ((line1 = reader1.readLine()) != null
-          && (line2 = reader2.readLine()) != null) {
-        // skip the hashCode part of the object string during the comparison,
-        // also ignore the difference between INodeFile/INodeFileWithSnapshot
-        line1 = line1.replaceAll("INodeFileWithSnapshot", "INodeFile");
-        line2 = line2.replaceAll("INodeFileWithSnapshot", "INodeFile");
-        line1 = line1.replaceAll("@[\\dabcdef]+", "");
-        line2 = line2.replaceAll("@[\\dabcdef]+", "");
-        assertEquals(line1, line2);
-      }
-      Assert.assertNull(reader1.readLine());
-      Assert.assertNull(reader2.readLine());
-    } finally {
-      reader1.close();
-      reader2.close();
-    }
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnMiddle);
+    SnapshotTestHelper.compareDumpedTreeInFile(fsnBefore, fsnAfter);
   }
   }
   
   
   /**
   /**
@@ -334,7 +306,7 @@ public class TestSnapshot {
       // If the node does not have files in it, create files
       // If the node does not have files in it, create files
       if (node.fileList == null) {
       if (node.fileList == null) {
         node.initFileList(hdfs, node.nodePath.getName(), BLOCKSIZE,
         node.initFileList(hdfs, node.nodePath.getName(), BLOCKSIZE,
-            REPLICATION, seed, 5);
+            REPLICATION, seed, 6);
       }
       }
       
       
       //
       //
@@ -361,20 +333,18 @@ public class TestSnapshot {
           node.fileList.get((node.nullFileIndex + 1) % node.fileList.size()),
           node.fileList.get((node.nullFileIndex + 1) % node.fileList.size()),
           hdfs);
           hdfs);
 
 
-      // TODO: temporarily disable file append testing before supporting
-      // INodeFileUnderConstructionWithSnapshot in FSImage saving/loading
-//      Modification append = new FileAppend(
-//          node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
-//          hdfs, (int) BLOCKSIZE);
-      Modification chmod = new FileChangePermission(
+      Modification append = new FileAppend(
           node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
           node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
+          hdfs, (int) BLOCKSIZE);
+      Modification chmod = new FileChangePermission(
+          node.fileList.get((node.nullFileIndex + 3) % node.fileList.size()),
           hdfs, genRandomPermission());
           hdfs, genRandomPermission());
       String[] userGroup = genRandomOwner();
       String[] userGroup = genRandomOwner();
       Modification chown = new FileChown(
       Modification chown = new FileChown(
-          node.fileList.get((node.nullFileIndex + 3) % node.fileList.size()),
+          node.fileList.get((node.nullFileIndex + 4) % node.fileList.size()),
           hdfs, userGroup[0], userGroup[1]);
           hdfs, userGroup[0], userGroup[1]);
       Modification replication = new FileChangeReplication(
       Modification replication = new FileChangeReplication(
-          node.fileList.get((node.nullFileIndex + 4) % node.fileList.size()),
+          node.fileList.get((node.nullFileIndex + 5) % node.fileList.size()),
           hdfs, (short) (random.nextInt(REPLICATION) + 1));
           hdfs, (short) (random.nextInt(REPLICATION) + 1));
       node.nullFileIndex = (node.nullFileIndex + 1) % node.fileList.size();
       node.nullFileIndex = (node.nullFileIndex + 1) % node.fileList.size();
       Modification dirChange = new DirCreationOrDeletion(node.nodePath, hdfs,
       Modification dirChange = new DirCreationOrDeletion(node.nodePath, hdfs,
@@ -382,8 +352,7 @@ public class TestSnapshot {
       
       
       mList.add(create);
       mList.add(create);
       mList.add(delete);
       mList.add(delete);
-      //TODO
-      //mList.add(append);
+      mList.add(append);
       mList.add(chmod);
       mList.add(chmod);
       mList.add(chown);
       mList.add(chown);
       mList.add(replication);
       mList.add(replication);
@@ -580,11 +549,7 @@ public class TestSnapshot {
     @Override
     @Override
     void modify() throws Exception {
     void modify() throws Exception {
       assertTrue(fs.exists(file));
       assertTrue(fs.exists(file));
-      FSDataOutputStream out = fs.append(file);
-      byte[] buffer = new byte[appendLen];
-      random.nextBytes(buffer);
-      out.write(buffer);
-      out.close();
+      DFSTestUtil.appendFile(fs, file, appendLen);
     }
     }
 
 
     @Override
     @Override