Kaynağa Gözat

HDFS-4126. Add reading/writing snapshot information to FSImage. Contributed by Jing Zhao.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1437256 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 yıl önce
ebeveyn
işleme
fe3584aadf
24 değiştirilmiş dosya ile 1473 ekleme ve 182 silme
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 14 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  4. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  5. 322 59
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  6. 99 37
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  7. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  8. 52 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  9. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  10. 17 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java
  11. 11 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  12. 191 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  13. 15 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileSnapshot.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionSnapshot.java
  15. 21 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java
  16. 21 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java
  17. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
  18. 351 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  19. 25 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  20. 162 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
  21. 8 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
  22. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeDirectoryWithSnapshot.java
  23. 66 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
  24. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java

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

@@ -113,3 +113,6 @@ Branch-2802 Snapshot (Unreleased)
   HDFS-4098. Add FileWithSnapshot, INodeFileUnderConstructionWithSnapshot and
   INodeFileUnderConstructionSnapshot for supporting append to snapshotted files.
   (szetszwo)
+
+  HDFS-4126. Add reading/writing snapshot information to FSImage.
+  (Jing Zhao via suresh)

+ 14 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -211,8 +211,21 @@ public class DFSUtil {
    * Converts a byte array to a string using UTF8 encoding.
    */
   public static String bytes2String(byte[] bytes) {
+    return bytes2String(bytes, 0, bytes.length);
+  }
+  
+  /**
+   * Decode a specific range of bytes of the given byte array to a string
+   * using UTF8.
+   * 
+   * @param bytes The bytes to be decoded into characters
+   * @param offset The index of the first byte to decode
+   * @param length The number of bytes to decode
+   * @return The decoded string
+   */
+  public static String bytes2String(byte[] bytes, int offset, int length) {
     try {
-      return new String(bytes, "UTF8");
+      return new String(bytes, offset, length, "UTF8");
     } catch(UnsupportedEncodingException e) {
       assert false : "UTF8 encoding is not supported ";
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1060,13 +1060,13 @@ public class FSDirectory implements Closeable {
     } finally {
       writeUnlock();
     }
+    fsImage.getEditLog().logDelete(src, now);
     if (filesRemoved <= 0) {
       return false;
     }
     incrDeletedFileCount(filesRemoved);
     // Blocks will be deleted later by the caller of this method
     getFSNamesystem().removePathAndBlocks(src, null);
-    fsImage.getEditLog().logDelete(src, now);
     return true;
   }
   

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -36,16 +36,21 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CreateSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteSnapshotOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp;
@@ -57,6 +62,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.Holder;
@@ -489,6 +495,41 @@ public class FSEditLogLoader {
       // no data in here currently.
       break;
     }
+    case OP_CREATE_SNAPSHOT: {
+      CreateSnapshotOp createSnapshotOp = (CreateSnapshotOp) op;
+      fsNamesys.getSnapshotManager().createSnapshot(
+          createSnapshotOp.snapshotRoot, createSnapshotOp.snapshotName);
+      break;
+    }
+    case OP_DELETE_SNAPSHOT: {
+      DeleteSnapshotOp deleteSnapshotOp = (DeleteSnapshotOp) op;
+      BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
+      fsNamesys.getSnapshotManager().deleteSnapshot(
+          deleteSnapshotOp.snapshotRoot, deleteSnapshotOp.snapshotName,
+          collectedBlocks);
+      fsNamesys.removeBlocks(collectedBlocks);
+      collectedBlocks.clear();
+      break;
+    }
+    case OP_RENAME_SNAPSHOT: {
+      RenameSnapshotOp renameSnapshotOp = (RenameSnapshotOp) op;
+      fsNamesys.getSnapshotManager().renameSnapshot(
+          renameSnapshotOp.snapshotRoot, renameSnapshotOp.snapshotOldName,
+          renameSnapshotOp.snapshotNewName);
+      break;
+    }
+    case OP_ALLOW_SNAPSHOT: {
+      AllowSnapshotOp allowSnapshotOp = (AllowSnapshotOp) op;
+      fsNamesys.getSnapshotManager().setSnapshottable(
+          allowSnapshotOp.snapshotRoot);
+      break;
+    }
+    case OP_DISALLOW_SNAPSHOT: {
+      DisallowSnapshotOp disallowSnapshotOp = (DisallowSnapshotOp) op;
+      fsNamesys.getSnapshotManager().resetSnapshottable(
+          disallowSnapshotOp.snapshotRoot);
+      break;
+    }
     default:
       throw new IOException("Invalid operation read " + op.opCode);
     }

+ 322 - 59
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -31,6 +31,9 @@ import java.security.DigestInputStream;
 import java.security.DigestOutputStream;
 import java.security.MessageDigest;
 import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -48,6 +51,10 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
+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.Snapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
@@ -59,13 +66,14 @@ import org.apache.hadoop.io.Text;
  * In particular, the format of the FSImage looks like:
  * <pre>
  * FSImage {
- *   LayoutVersion: int, NamespaceID: int, NumberItemsInFSDirectoryTree: long,
- *   NamesystemGenerationStamp: long, TransactionID: long
+ *   layoutVersion: int, namespaceID: int, numberItemsInFSDirectoryTree: long,
+ *   namesystemGenerationStamp: long, transactionID: long, 
+ *   snapshotCounter: int, numberOfSnapshots: int, numOfSnapshottableDirs: int,
  *   {FSDirectoryTree, FilesUnderConstruction, SecretManagerState} (can be compressed)
  * }
  * 
  * FSDirectoryTree (if {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported) {
- *   INodeInfo of root, NumberOfChildren of root: int
+ *   INodeInfo of root, numberOfChildren of root: int
  *   [list of INodeInfo of root's children],
  *   [list of INodeDirectoryInfo of root's directory children]
  * }
@@ -76,38 +84,76 @@ import org.apache.hadoop.io.Text;
  * 
  * INodeInfo {
  *   {
- *     LocalName: short + byte[]
+ *     localName: short + byte[]
  *   } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported
  *   or 
  *   {
- *     FullPath: byte[]
+ *     fullPath: byte[]
  *   } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is not supported
- *   ReplicationFactor: short, ModificationTime: long,
- *   AccessTime: long, PreferredBlockSize: long,
- *   NumberOfBlocks: int (-1 for INodeDirectory, -2 for INodeSymLink),
+ *   replicationFactor: short, modificationTime: long,
+ *   accessTime: long, preferredBlockSize: long,
+ *   numberOfBlocks: int (-1 for INodeDirectory, -2 for INodeSymLink),
  *   { 
- *     NsQuota: long, DsQuota: long, FsPermission: short, PermissionStatus
+ *     nsQuota: long, dsQuota: long, 
+ *     {
+ *       isINodeSnapshottable: byte,
+ *       isINodeWithSnapshot: byte (if isINodeSnapshottable is false)
+ *     } (when {@link Feature#SNAPSHOT} is supported), 
+ *     fsPermission: short, PermissionStatus
  *   } for INodeDirectory
  *   or 
  *   {
- *     SymlinkString, FsPermission: short, PermissionStatus
+ *     symlinkString, fsPermission: short, PermissionStatus
  *   } for INodeSymlink
  *   or
  *   {
- *     [list of BlockInfo], FsPermission: short, PermissionStatus
+ *     containsBlock: byte (when {@link Feature#SNAPSHOT} is supported),
+ *     [list of BlockInfo] (when {@link Feature#SNAPSHOT} is not supported or 
+ *     containsBlock is true),
+ *     {
+ *       snapshotFileSize: long,
+ *       isINodeFileWithLink: byte (if ComputedFileSize is negative),
+ *     } (when {@link Feature#SNAPSHOT} is supported), 
+ *     fsPermission: short, PermissionStatus
  *   } for INodeFile
  * }
  * 
  * INodeDirectoryInfo {
- *   FullPath of the directory: short + byte[],
- *   NumberOfChildren: int, [list of INodeInfo of children INode]
- *   [list of INodeDirectoryInfo of the directory children]
+ *   fullPath of the directory: short + byte[],
+ *   numberOfChildren: int, [list of INodeInfo of children INode],
+ *   {
+ *     numberOfSnapshots: int,
+ *     [list of Snapshot] (when NumberOfSnapshots is positive),
+ *     numberOfSnapshotDiffs: int,
+ *     [list of SnapshotDiff] (NumberOfSnapshotDiffs is positive),
+ *     number of children that are directories,
+ *     [list of INodeDirectoryInfo of the directory children] (includes
+ *     snapshot copies of deleted sub-directories)
+ *   } (when {@link Feature#SNAPSHOT} is supported), 
+ * }
+ * 
+ * Snapshot {
+ *   snapshotID: int, root of Snapshot: INodeDirectoryInfo (its local name is 
+ *   the name of the snapshot)
+ * }
+ * 
+ * SnapshotDiff {
+ *   childrenSize: int, 
+ *   full path of the root of the associated Snapshot: short + byte[], 
+ *   isSnapshotRoot: byte, 
+ *   snapshotINodeIsNotNull: byte (when isSnapshotRoot is false),
+ *   snapshotINode: INodeDirectory (when SnapshotINodeIsNotNull is true), Diff 
+ * }
+ * 
+ * Diff {
+ *   createdListSize: int, [Local name of INode in created list],
+ *   deletedListSize: int, [INode in deleted list: INodeInfo]
  * }
  * </pre>
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-class FSImageFormat {
+public class FSImageFormat {
   private static final Log LOG = FSImage.LOG;
   
   // Static-only class
@@ -118,7 +164,7 @@ class FSImageFormat {
    * should be called once, after which the getter methods may be used to retrieve
    * information about the image that was loaded, if loading was successful.
    */
-  static class Loader {
+  public static class Loader {
     private final Configuration conf;
     /** which namesystem this loader is working for */
     private final FSNamesystem namesystem;
@@ -168,9 +214,7 @@ class FSImageFormat {
       }
     }
 
-    void load(File curFile)
-      throws IOException
-    {
+    void load(File curFile) throws IOException {
       checkNotLoaded();
       assert curFile != null : "curFile is null";
 
@@ -209,6 +253,10 @@ class FSImageFormat {
         } else {
           imgTxId = 0;
         }
+        
+        if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+          namesystem.getSnapshotManager().read(in);
+        }
 
         // read compression related info
         FSImageCompression compression;
@@ -226,7 +274,11 @@ class FSImageFormat {
         LOG.info("Number of files = " + numFiles);
         if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
             imgVersion)) {
-          loadLocalNameINodes(numFiles, in);
+          if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+            loadLocalNameINodesWithSnapshot(in);
+          } else {
+            loadLocalNameINodes(numFiles, in);
+          }
         } else {
           loadFullNameINodes(numFiles, in);
         }
@@ -260,7 +312,25 @@ class FSImageFormat {
     fsDir.rootDir.cloneModificationTime(root);
     fsDir.rootDir.clonePermissionStatus(root);    
   }
-
+  
+    /**
+     * Load fsimage files when 1) only local names are stored, 
+     * and 2) snapshot is supported.
+     * 
+     * @param in Image input stream
+     */
+    private void loadLocalNameINodesWithSnapshot(DataInputStream in)
+        throws IOException {
+      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
+          getLayoutVersion());
+      assert LayoutVersion.supports(Feature.SNAPSHOT, getLayoutVersion());
+      
+      // load root
+      loadRoot(in);
+      // load rest of the nodes recursively
+      loadDirectoryWithSnapshot(in);
+    }
+    
   /** 
    * load fsimage files assuming only local names are stored
    *   
@@ -275,13 +345,9 @@ class FSImageFormat {
      assert numFiles > 0;
 
      // load root
-     if( in.readShort() != 0) {
-       throw new IOException("First node is not root");
-     }   
-     INode root = loadINode(in);
-     // update the root's attributes
-     updateRootAttr(root);
-     numFiles--;
+     loadRoot(in);
+     // have loaded the first file (the root)
+     numFiles--; 
 
      // load rest of the nodes directory by directory
      while (numFiles > 0) {
@@ -292,6 +358,77 @@ class FSImageFormat {
      }
    }
    
+    /**
+     * Load information about root, and use the information to update the root
+     * directory of NameSystem.
+     * @param in The {@link DataInputStream} instance to read.
+     */
+    private void loadRoot(DataInputStream in) throws IOException {
+      // load root
+      if (in.readShort() != 0) {
+        throw new IOException("First node is not root");
+      }
+      INode root = loadINode(in);
+      // update the root's attributes
+      updateRootAttr(root);
+    }
+   
+    /** Load children nodes for the parent directory. */
+    private void loadChildren(INodeDirectory parent, DataInputStream in)
+        throws IOException {
+      int numChildren = in.readInt();
+      for (int i = 0; i < numChildren; i++) {
+        // load single inode
+        byte[] localName = new byte[in.readShort()];
+        in.readFully(localName); // read local name
+        INode newNode = loadINode(in); // read rest of inode
+        newNode.setLocalName(localName);
+        addToParent(parent, newNode);
+      }
+    }
+    
+    /**
+     * Load a directory when snapshot is supported.
+     * @param in The {@link DataInputStream} instance to read.
+     */
+    private void loadDirectoryWithSnapshot(DataInputStream in)
+        throws IOException {
+      // Step 1. Identify the parent INode
+      String parentPath = FSImageSerialization.readString(in);
+      final INodeDirectory parent = INodeDirectory.valueOf(
+          namesystem.dir.rootDir.getNode(parentPath, false), parentPath);
+      
+      // Step 2. Load children nodes under parent
+      loadChildren(parent, in);
+      
+      // Step 3. Load snapshots if parent is snapshottable
+      int numSnapshots = in.readInt();
+      INodeDirectorySnapshottable snapshottableParent = null;
+      if (numSnapshots >= 0) {
+        snapshottableParent = (INodeDirectorySnapshottable) parent;
+        // load snapshots and snapshotQuota
+        SnapshotFSImageFormat.loadSnapshotList(snapshottableParent,
+            numSnapshots, in, this);
+      }
+      
+      // Step 4. load SnapshotDiff list
+      int numSnapshotDiffs = in.readInt();
+      if (numSnapshotDiffs >= 0) {
+        INodeDirectoryWithSnapshot parentWithSnapshot = 
+            (INodeDirectoryWithSnapshot) parent;
+        // load SnapshotDiff list
+        SnapshotFSImageFormat.loadSnapshotDiffList(parentWithSnapshot,
+            numSnapshotDiffs, in, this);
+      }
+      
+      // Recursively load sub-directories, including snapshot copies of deleted
+      // directories
+      int numSubTree = in.readInt();
+      for (int i = 0; i < numSubTree; i++) {
+        loadDirectoryWithSnapshot(in);
+      }
+    }
+    
    /**
     * Load all children of a directory
     * 
@@ -388,17 +525,25 @@ class FSImageFormat {
     }
   }
 
+    /** @return The FSDirectory of the namesystem where the fsimage is loaded */
+    public FSDirectory getFSDirectoryInLoading() {
+      return namesystem.dir;
+    }
+  
   /**
    * load an inode from fsimage except for its name
    * 
    * @param in data input stream from which image is read
    * @return an inode
    */
-  private INode loadINode(DataInputStream in)
-      throws IOException {
+  public INode loadINode(DataInputStream in) throws IOException {
     long modificationTime = 0;
     long atime = 0;
     long blockSize = 0;
+    long computeFileSize = -1;
+    boolean snapshottable = false;
+    boolean withSnapshot = false;
+    boolean withLink = false;
     
     int imgVersion = getLayoutVersion();
     long inodeId = namesystem.allocateNewInodeId();
@@ -414,11 +559,22 @@ class FSImageFormat {
     BlockInfo blocks[] = null;
 
     if (numBlocks >= 0) {
-      blocks = new BlockInfo[numBlocks];
+      // to indicate INodeFileWithLink, blocks may be set as null while
+      // numBlocks is set to 0
+      blocks = LayoutVersion.supports(Feature.SNAPSHOT, imgVersion) ? (in
+            .readBoolean() ? new BlockInfo[numBlocks] : null)
+            : new BlockInfo[numBlocks];
+      
       for (int j = 0; j < numBlocks; j++) {
         blocks[j] = new BlockInfo(replication);
         blocks[j].readFields(in);
       }
+      if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+        computeFileSize = in.readLong();
+        if (computeFileSize < 0) {
+          withLink = in.readBoolean();
+        }
+      }
     }
     
     // get quota only when the node is a directory
@@ -431,7 +587,14 @@ class FSImageFormat {
         && blocks == null && numBlocks == -1) {
       dsQuota = in.readLong();
     }
-
+    if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)
+        && blocks == null && numBlocks == -1) {
+      snapshottable = in.readBoolean();
+      if (!snapshottable) {
+        withSnapshot = in.readBoolean();
+      }
+    }
+    
     // Read the symlink only when the node is a symlink
     String symlink = "";
     if (numBlocks == -2) {
@@ -441,7 +604,8 @@ class FSImageFormat {
     PermissionStatus permissions = PermissionStatus.read(in);
 
     return INode.newINode(inodeId, permissions, blocks, symlink, replication,
-        modificationTime, atime, nsQuota, dsQuota, blockSize);
+        modificationTime, atime, nsQuota, dsQuota, blockSize, numBlocks,
+        withLink, computeFileSize, snapshottable, withSnapshot);
   }
 
     private void loadFilesUnderConstruction(DataInputStream in)
@@ -557,9 +721,7 @@ class FSImageFormat {
       return savedDigest;
     }
 
-    void save(File newFile,
-              FSImageCompression compression)
-      throws IOException {
+    void save(File newFile, FSImageCompression compression) throws IOException {
       checkNotSaved();
 
       final FSNamesystem sourceNamesystem = context.getSourceNamesystem();
@@ -584,19 +746,19 @@ class FSImageFormat {
         out.writeLong(fsDir.rootDir.numItemsInTree());
         out.writeLong(sourceNamesystem.getGenerationStamp());
         out.writeLong(context.getTxId());
-
+        sourceNamesystem.getSnapshotManager().write(out);
+        
         // write compression info and set up compressed stream
         out = compression.writeHeaderAndWrapStream(fos);
         LOG.info("Saving image file " + newFile +
                  " using " + compression);
 
-
         byte[] byteStore = new byte[4*HdfsConstants.MAX_PATH_LENGTH];
         ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
         // save the root
         FSImageSerialization.saveINode2Image(fsDir.rootDir, out);
         // save the rest of the nodes
-        saveImage(strbuf, fsDir.rootDir, out);
+        saveImage(strbuf, fsDir.rootDir, out, null);
         // save files under construction
         sourceNamesystem.saveFilesUnderConstruction(out);
         context.checkCancelled();
@@ -619,42 +781,143 @@ class FSImageFormat {
     }
 
     /**
-     * Save file tree image starting from the given root.
-     * This is a recursive procedure, which first saves all children of
-     * a current directory and then moves inside the sub-directories.
+     * Save children INodes.
+     * @param children The list of children INodes
+     * @param out The DataOutputStream to write
+     * @return Number of children that are directory
      */
-    private void saveImage(ByteBuffer currentDirName,
-                                  INodeDirectory current,
-                                  DataOutputStream out) throws IOException {
-      final ReadOnlyList<INode> children = current.getChildrenList(null);
-      if (children.isEmpty()) {
-        return;
-      }
-      // print prefix (parent directory name)
-      int prefixLen = currentDirName.position();
-      if (prefixLen == 0) {  // root
-        out.writeShort(PATH_SEPARATOR.length);
-        out.write(PATH_SEPARATOR);
-      } else {  // non-root directories
-        out.writeShort(prefixLen);
-        out.write(currentDirName.array(), 0, prefixLen);
-      }
+    private int saveChildren(ReadOnlyList<INode> children, DataOutputStream out)
+        throws IOException {
+      // Write normal children INode. 
       out.writeInt(children.size());
+      int dirNum = 0;
       int i = 0;
       for(INode child : children) {
         // print all children first
         FSImageSerialization.saveINode2Image(child, out);
+        if (child.isDirectory()) {
+          dirNum++;
+        }
         if (i++ % 50 == 0) {
           context.checkCancelled();
         }
       }
+      return dirNum;
+    }
+    
+    /**
+     * The nonSnapshotPath is a path without snapshot in order to enable buffer
+     * reuse. If the snapshot is not null, we need to compute a snapshot path.
+     * E.g., when nonSnapshotPath is "/test/foo/bar/" and the snapshot is s1 of
+     * /test, we actually want to save image for directory /test/foo/bar/ under
+     * snapshot s1 of /test, and the path to save thus should be
+     * "/test/.snapshot/s1/foo/bar/".
+     * 
+     * @param nonSnapshotPath The path without snapshot related information.
+     * @param snapshot The snapshot associated with the inode that the path 
+     *                 actually leads to.
+     * @return The snapshot path.                
+     */
+    private String computeSnapshotPath(String nonSnapshotPath, 
+        Snapshot snapshot) {
+      String snapshotParentFullPath = snapshot.getRoot().getParent()
+          .getFullPathName();
+      String snapshotName = snapshot.getRoot().getLocalName();
+      String relativePath = nonSnapshotPath.equals(snapshotParentFullPath) ? 
+          Path.SEPARATOR : nonSnapshotPath.substring(
+               snapshotParentFullPath.length());
+      String snapshotFullPath = snapshotParentFullPath + Path.SEPARATOR
+          + HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + snapshotName
+          + relativePath;
+      return snapshotFullPath;
+    }
+    
+    /**
+     * Save file tree image starting from the given root.
+     * This is a recursive procedure, which first saves all children and 
+     * snapshot diffs of a current directory and then moves inside the 
+     * sub-directories.
+     * 
+     * @param currentDirName A ByteBuffer storing the path leading to the 
+     *                       current node. For a snapshot node, the path is
+     *                       (the snapshot path - ".snapshot/snapshot_name")
+     * @param current The current node
+     * @param out The DataoutputStream to write the image
+     * @param snapshot The possible snapshot associated with the current node
+     */
+    private void saveImage(ByteBuffer currentDirName, INodeDirectory current,
+        DataOutputStream out, Snapshot snapshot)
+        throws IOException {
+      final ReadOnlyList<INode> children = current.getChildrenList(null);
+      int dirNum = 0;
+      Map<Snapshot, List<INodeDirectory>> snapshotDirMap = null;
+      if (current instanceof INodeDirectoryWithSnapshot) {
+        snapshotDirMap = new HashMap<Snapshot, List<INodeDirectory>>();
+        dirNum += ((INodeDirectoryWithSnapshot) current).
+            getSnapshotDirectory(snapshotDirMap);
+      }
+      
+      // 1. Print prefix (parent directory name)
+      int prefixLen = currentDirName.position();
+      if (snapshot == null) {
+        if (prefixLen == 0) {  // root
+          out.writeShort(PATH_SEPARATOR.length);
+          out.write(PATH_SEPARATOR);
+        } else {  // non-root directories
+          out.writeShort(prefixLen);
+          out.write(currentDirName.array(), 0, prefixLen);
+        }
+      } else {
+        String nonSnapshotPath = prefixLen == 0 ? Path.SEPARATOR : DFSUtil
+            .bytes2String(currentDirName.array(), 0, prefixLen);
+        String snapshotFullPath = computeSnapshotPath(nonSnapshotPath, 
+            snapshot);
+        byte[] snapshotFullPathBytes = DFSUtil.string2Bytes(snapshotFullPath);
+        out.writeShort(snapshotFullPathBytes.length);
+        out.write(snapshotFullPathBytes);
+      }
+      
+      // 2. Write children INode 
+      dirNum += saveChildren(children, out);
+      
+      // 3. Write INodeDirectorySnapshottable#snapshotsByNames to record all
+      // Snapshots
+      if (current instanceof INodeDirectorySnapshottable) {
+        INodeDirectorySnapshottable snapshottableNode = 
+            (INodeDirectorySnapshottable) current;
+        SnapshotFSImageFormat.saveSnapshots(snapshottableNode, out);
+      } else {
+        out.writeInt(-1); // # of snapshots
+      }
+      
+      // 4. Write SnapshotDiff lists.
+      if (current instanceof INodeDirectoryWithSnapshot) {
+        INodeDirectoryWithSnapshot sNode = (INodeDirectoryWithSnapshot) current;
+        SnapshotFSImageFormat.saveSnapshotDiffs(sNode, out);
+      } else {
+        out.writeInt(-1); // # of SnapshotDiffs
+      }
+      
+      // Write sub-tree of sub-directories, including possible snapshots of 
+      // deleted sub-directories
+      out.writeInt(dirNum); // the number of sub-directories
       for(INode child : children) {
         if(!child.isDirectory())
           continue;
         currentDirName.put(PATH_SEPARATOR).put(child.getLocalNameBytes());
-        saveImage(currentDirName, (INodeDirectory)child, out);
+        saveImage(currentDirName, (INodeDirectory)child, out, snapshot);
         currentDirName.position(prefixLen);
       }
+      if (snapshotDirMap != null) {
+        for (Snapshot ss : snapshotDirMap.keySet()) {
+          List<INodeDirectory> snapshotSubDirs = snapshotDirMap.get(ss);
+          for (INodeDirectory subDir : snapshotSubDirs) {
+            currentDirName.put(PATH_SEPARATOR).put(subDir.getLocalNameBytes());
+            saveImage(currentDirName, subDir, out, ss);
+            currentDirName.position(prefixLen);
+          }
+        }
+      }
     }
   }
 }

+ 99 - 37
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.DataInputStream;
+import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 
@@ -32,6 +33,10 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+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.INodeFileSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
@@ -143,52 +148,109 @@ public class FSImageSerialization {
     out.writeInt(0); //  do not store locations of last block
   }
 
-  /*
-   * Save one inode's attributes to the image.
+  /**
+   * Serialize a {@link INodeDirectory}
+   * @param node The node to write
+   * @param out The {@link DataOutput} where the fields are written 
    */
-  static void saveINode2Image(INode node,
-                              DataOutputStream out) throws IOException {
+  public static void writeINodeDirectory(INodeDirectory node, DataOutput out)
+      throws IOException {
     byte[] name = node.getLocalNameBytes();
     out.writeShort(name.length);
     out.write(name);
-    FsPermission filePerm = TL_DATA.get().FILE_PERM;
-    if (node.isDirectory()) {
-      out.writeShort(0);  // replication
-      out.writeLong(node.getModificationTime());
-      out.writeLong(0);   // access time
-      out.writeLong(0);   // preferred block size
-      out.writeInt(-1);   // # of blocks
-      out.writeLong(node.getNsQuota());
-      out.writeLong(node.getDsQuota());
-      filePerm.fromShort(node.getFsPermissionShort());
-      PermissionStatus.write(out, node.getUserName(),
-                             node.getGroupName(),
-                             filePerm);
-    } else if (node.isSymlink()) {
-      out.writeShort(0);  // replication
-      out.writeLong(0);   // modification time
-      out.writeLong(0);   // access time
-      out.writeLong(0);   // preferred block size
-      out.writeInt(-2);   // # of blocks
-      Text.writeString(out, ((INodeSymlink)node).getSymlinkString());
-      filePerm.fromShort(node.getFsPermissionShort());
-      PermissionStatus.write(out, node.getUserName(),
-                             node.getGroupName(),
-                             filePerm);      
+    out.writeShort(0);  // replication
+    out.writeLong(node.getModificationTime());
+    out.writeLong(0);   // access time
+    out.writeLong(0);   // preferred block size
+    out.writeInt(-1);   // # of blocks
+    out.writeLong(node.getNsQuota());
+    out.writeLong(node.getDsQuota());
+    if (node instanceof INodeDirectorySnapshottable) {
+      out.writeBoolean(true);
     } else {
-      INodeFile fileINode = (INodeFile)node;
-      out.writeShort(fileINode.getFileReplication());
-      out.writeLong(fileINode.getModificationTime());
-      out.writeLong(fileINode.getAccessTime());
-      out.writeLong(fileINode.getPreferredBlockSize());
+      out.writeBoolean(false);
+      out.writeBoolean(node instanceof INodeDirectoryWithSnapshot);
+    }
+    FsPermission filePerm = TL_DATA.get().FILE_PERM;
+    filePerm.fromShort(node.getFsPermissionShort());
+    PermissionStatus.write(out, node.getUserName(),
+                           node.getGroupName(),
+                           filePerm);
+  }
+  
+  /**
+   * Serialize a {@link INodeSymlink} node
+   * @param node The node to write
+   * @param out The {@link DataOutput} where the fields are written
+   */
+  private static void writeINodeSymlink(INodeSymlink node, DataOutput out)
+      throws IOException {
+    byte[] name = node.getLocalNameBytes();
+    out.writeShort(name.length);
+    out.write(name);
+    out.writeShort(0);  // replication
+    out.writeLong(0);   // modification time
+    out.writeLong(0);   // access time
+    out.writeLong(0);   // preferred block size
+    out.writeInt(-2);   // # of blocks
+    Text.writeString(out, node.getSymlinkString());
+    FsPermission filePerm = TL_DATA.get().FILE_PERM;
+    filePerm.fromShort(node.getFsPermissionShort());
+    PermissionStatus.write(out, node.getUserName(),
+                           node.getGroupName(),
+                           filePerm);
+  }
+  
+  /**
+   * Serialize a {@link INodeFile} node
+   * @param node The node to write
+   * @param out The {@link DataOutput} where the fields are written
+   * @param writeBlock Whether to write block information
+   */
+  public static void writeINodeFile(INodeFile node, DataOutput out,
+      boolean writeBlock) throws IOException {
+    byte[] name = node.getLocalNameBytes();
+    out.writeShort(name.length);
+    out.write(name);
+    INodeFile fileINode = node;
+    out.writeShort(fileINode.getFileReplication());
+    out.writeLong(fileINode.getModificationTime());
+    out.writeLong(fileINode.getAccessTime());
+    out.writeLong(fileINode.getPreferredBlockSize());
+    if (writeBlock) {
       Block[] blocks = fileINode.getBlocks();
       out.writeInt(blocks.length);
+      out.writeBoolean(true);
       for (Block blk : blocks)
         blk.write(out);
-      filePerm.fromShort(fileINode.getFsPermissionShort());
-      PermissionStatus.write(out, fileINode.getUserName(),
-                             fileINode.getGroupName(),
-                             filePerm);
+    } else {
+      out.writeInt(0); // # of blocks
+      out.writeBoolean(false);
+    }
+    if (node instanceof INodeFileSnapshot) {
+      out.writeLong(((INodeFileSnapshot) node).computeFileSize(true));
+    } else {
+      out.writeLong(-1);
+      out.writeBoolean(node instanceof INodeFileWithSnapshot);
+    }
+    FsPermission filePerm = TL_DATA.get().FILE_PERM;
+    filePerm.fromShort(fileINode.getFsPermissionShort());
+    PermissionStatus.write(out, fileINode.getUserName(),
+                           fileINode.getGroupName(),
+                           filePerm);
+  }
+  
+  /**
+   * Save one inode's attributes to the image.
+   */
+  static void saveINode2Image(INode node, DataOutput out)
+      throws IOException {
+    if (node.isDirectory()) {
+      writeINodeDirectory((INodeDirectory) node, out);
+    } else if (node.isSymlink()) {
+      writeINodeSymlink((INodeSymlink) node, out);      
+    } else {
+      writeINodeFile((INodeFile) node, out, true);
     }
   }
 

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2824,7 +2824,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    *          An instance of {@link BlocksMapUpdateInfo} which contains a list
    *          of blocks that need to be removed from blocksMap
    */
-  private void removeBlocks(BlocksMapUpdateInfo blocks) {
+  void removeBlocks(BlocksMapUpdateInfo blocks) {
     Iterator<Map.Entry<Block, BlocksMapINodeUpdateEntry>> iter = blocks
         .iterator();
     while (iter.hasNext()) {
@@ -5645,6 +5645,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         .isAvoidingStaleDataNodesForWrite();
   }
   
+  public SnapshotManager getSnapshotManager() {
+    return snapshotManager;
+  }
+  
   /** Allow snapshot on a directroy. */
   public void allowSnapshot(String path) throws SafeModeException, IOException {
     writeLock();

+ 52 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -35,6 +35,11 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+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.INodeDirectoryWithSnapshot;
+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.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.StringUtils;
@@ -619,31 +624,41 @@ public abstract class INode implements Comparable<byte[]> {
    * @param nsQuota namespace quota
    * @param dsQuota disk quota
    * @param preferredBlockSize block size
+   * @param numBlocks number of blocks
+   * @param withLink whether the node is INodeWithLink
+   * @param computeFileSize non-negative computeFileSize means the node is 
+   *                        INodeFileSnapshot
+   * @param snapshottable whether the node is {@link INodeDirectorySnapshottable}
+   * @param withSnapshot whether the node is {@link INodeDirectoryWithSnapshot}                       
    * @return an inode
    */
-  static INode newINode(long id,
-                        PermissionStatus permissions,
-                        BlockInfo[] blocks,
-                        String symlink,
-                        short replication,
-                        long modificationTime,
-                        long atime,
-                        long nsQuota,
-                        long dsQuota,
-                        long preferredBlockSize) {
+  static INode newINode(long id, PermissionStatus permissions,
+      BlockInfo[] blocks, String symlink, short replication,
+      long modificationTime, long atime, long nsQuota, long dsQuota,
+      long preferredBlockSize, int numBlocks, boolean withLink,
+      long computeFileSize, boolean snapshottable, boolean withSnapshot) {
     if (symlink.length() != 0) { // check if symbolic link
       return new INodeSymlink(id, symlink, modificationTime, atime, permissions);
-    }  else if (blocks == null) { //not sym link and blocks null? directory!
+    }  else if (blocks == null && numBlocks < 0) { 
+      //not sym link and numBlocks < 0? directory!
+      INodeDirectory dir = null;
       if (nsQuota >= 0 || dsQuota >= 0) {
-        return new INodeDirectoryWithQuota(
-             id, permissions, modificationTime, nsQuota, dsQuota);
-      } 
-      // regular directory
-      return new INodeDirectory(id, permissions, modificationTime);
+        dir = new INodeDirectoryWithQuota(id, permissions, modificationTime,
+            nsQuota, dsQuota);
+      } else {
+        // regular directory
+        dir = new INodeDirectory(id, permissions, modificationTime);
+      }
+      return snapshottable ? new INodeDirectorySnapshottable(dir)
+          : (withSnapshot ? INodeDirectoryWithSnapshot.newInstance(dir, null)
+              : dir);
     }
     // file
-    return new INodeFile(id, permissions, blocks, replication,
+    INodeFile fileNode = new INodeFile(id, permissions, blocks, replication,
         modificationTime, atime, preferredBlockSize);
+    return computeFileSize >= 0 ? new INodeFileSnapshot(fileNode,
+        computeFileSize) : (withLink ? new INodeFileWithSnapshot(fileNode)
+        : fileNode);
   }
 
   /**
@@ -662,7 +677,8 @@ public abstract class INode implements Comparable<byte[]> {
    * @param prefix The prefix string that each line should print.
    */
   @VisibleForTesting
-  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix, Snapshot snapshot) {
+  public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
+      Snapshot snapshot) {
     out.print(prefix);
     out.print(" ");
     out.print(getLocalName());
@@ -670,10 +686,27 @@ public abstract class INode implements Comparable<byte[]> {
     out.print(getObjectString());
     out.print("), parent=");
     out.print(parent == null? null: parent.getLocalName() + "/");
+    out.print(", permission=" + getFsPermission(snapshot) + ", group="
+        + getGroupName(snapshot) + ", user=" + getUserName(snapshot));
     if (!this.isDirectory()) {
+      if (this.isFile()) {
+        // print block information
+        String blocksInfo = ((INodeFile) this).printBlocksInfo();
+        out.print(", blocks=[" + blocksInfo + "]");
+      }
+      if (this instanceof INodeFileWithSnapshot) {
+        INodeFileWithSnapshot nodeWithLink = (INodeFileWithSnapshot) this;
+        FileWithSnapshot next = nodeWithLink.getNext();
+        out.print(", next="
+            + (next != null ? next.asINodeFile().getObjectString() : "null"));
+        if (this instanceof INodeFileSnapshot) {
+          out.print(", computedSize="
+              + ((INodeFileSnapshot) this).computeFileSize(true));
+        }
+      }
       out.println();
     } else {
-      final INodeDirectory dir = (INodeDirectory)this;
+      final INodeDirectory dir = (INodeDirectory) this;
       out.println(", size=" + dir.getChildrenList(snapshot).size());
     }
   }

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -319,4 +319,18 @@ public class INodeFile extends INode implements BlockCollection {
   public int numBlocks() {
     return blocks == null ? 0 : blocks.length;
   }
+  
+  /**
+   * @return A String containing all the blockInfo
+   */
+  String printBlocksInfo() {
+    if (blocks == null) {
+      return "";
+    }
+    StringBuilder buffer = new StringBuilder();
+    for (BlockInfo blk : blocks) {
+      buffer.append(blk.toString() + " ");
+    }
+    return buffer.toString();
+  }
 }

+ 17 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java

@@ -41,12 +41,27 @@ public interface FileWithSnapshot {
   /** Set the next element. */
   public void setNext(FileWithSnapshot next);
   
-  /** Insert inode to the circular linked list. */
-  public void insert(FileWithSnapshot inode);
+  /** Insert inode to the circular linked list, after the current node. */
+  public void insertAfter(FileWithSnapshot inode);
+  
+  /** Insert inode to the circular linked list, before the current node. */
+  public void insertBefore(FileWithSnapshot inode);
+  
+  /** Remove self from the circular list */
+  public void removeSelf();
   
   /** Utility methods for the classes which implement the interface. */
   static class Util {
 
+    /** @return The previous node in the circular linked list */
+    static FileWithSnapshot getPrevious(FileWithSnapshot file) {
+      FileWithSnapshot previous = file.getNext();
+      while (previous.getNext() != file) {
+        previous = previous.getNext();
+      }
+      return previous;
+    }
+    
     /** Replace the old file with the new file in the circular linked list. */
     static void replace(FileWithSnapshot oldFile, FileWithSnapshot newFile) {
       //set next element

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

@@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
@@ -67,9 +66,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   /**
    * @return {@link #snapshotsByNames}
    */
-  @VisibleForTesting
-  List<Snapshot> getSnapshotsByNames() {
-    return snapshotsByNames;
+  ReadOnlyList<Snapshot> getSnapshotsByNames() {
+    return ReadOnlyList.Util.asReadOnlyList(this.snapshotsByNames);
   }
   
   /** Number of snapshots allowed. */
@@ -82,7 +80,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   
   /** @return the number of existing snapshots. */
   public int getNumSnapshots() {
-    return getSnapshotsByNames().size();
+    return snapshotsByNames.size();
   }
   
   private int searchSnapshot(byte[] snapshotName) {
@@ -153,6 +151,14 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   public boolean isSnapshottable() {
     return true;
   }
+  
+  /**
+   * Simply add a snapshot into the {@link #snapshotsByNames}. Used by FSImage
+   * loading.
+   */
+  void addSnapshot(Snapshot snapshot) {
+    this.snapshotsByNames.add(snapshot);
+  }
 
   /** Add a snapshot. */
   Snapshot addSnapshot(int id, String name) throws SnapshotException {

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

@@ -17,19 +17,23 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import java.io.DataOutput;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
@@ -79,7 +83,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
    *   2.3.3. modify i in current and then modify: replace it in c-list (c', d)
    * </pre>
    */
-  static class Diff {
+  public static class Diff {
     /**
      * Search the inode from the list.
      * @return -1 if the list is null; otherwise, return the insertion point
@@ -105,6 +109,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     /** d-list: inode(s) deleted from current. */
     private List<INode> deleted;
 
+    INode searchCreated(final byte[] name) {
+      int cIndex = search(created, name);
+      return cIndex < 0 ? null : created.get(cIndex);
+    }
+    
+    INode searchDeleted(final byte[] name) {
+      int dIndex = search(deleted, name);
+      return dIndex < 0 ? null : deleted.get(dIndex);
+    }
+    
     /**
      * Insert the inode to created.
      * @param i the insertion point defined
@@ -155,13 +169,18 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
      * Delete an inode from current state.
      * @return a triple for undo.
      */
-    Triple<Integer, INode, Integer> delete(final INode inode) {
+    Triple<Integer, INode, Integer> delete(final INode inode,
+        boolean updateCircularList) {
       final int c = search(created, inode);
       INode previous = null;
       Integer d = null;
       if (c >= 0) {
         // remove a newly created inode
         previous = created.remove(c);
+        if (updateCircularList && previous instanceof FileWithSnapshot) {
+          // also we should remove previous from the circular list
+          ((FileWithSnapshot) previous).removeSelf();
+        }
       } else {
         // not in c-list, it must be in previous
         d = search(deleted, inode);
@@ -184,7 +203,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
      * Modify an inode in current state.
      * @return a triple for undo.
      */
-    Triple<Integer, INode, Integer> modify(final INode oldinode, final INode newinode) {
+    Triple<Integer, INode, Integer> modify(final INode oldinode,
+        final INode newinode, boolean updateCircularList) {
       if (!oldinode.equals(newinode)) {
         throw new AssertionError("The names do not match: oldinode="
             + oldinode + ", newinode=" + newinode);
@@ -196,6 +216,14 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
         // Case 1.1.3 and 2.3.3: inode is already in c-list,
         previous = created.set(c, newinode);
         
+        if (updateCircularList && newinode instanceof FileWithSnapshot) {
+          // also should remove oldinode from the circular list
+          FileWithSnapshot newNodeWithLink = (FileWithSnapshot) newinode;
+          FileWithSnapshot oldNodeWithLink = (FileWithSnapshot) oldinode;
+          newNodeWithLink.setNext(oldNodeWithLink.getNext());
+          oldNodeWithLink.setNext(null);
+        }
+        
         //TODO: fix a bug that previous != oldinode.  Set it to oldinode for now
         previous = oldinode;
       } else {
@@ -328,8 +356,11 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
      * @param the posterior diff to combine
      * @param deletedINodeProcesser Used in case 2.1, 2.3, 3.1, and 3.3
      *                              to process the deleted inodes.
+     * @param updateCircularList Whether to update the circular linked list 
+     *                           while combining the diffs.                             
      */
-    void combinePostDiff(Diff postDiff, Processor deletedINodeProcesser) {
+    void combinePostDiff(Diff postDiff, Processor deletedINodeProcesser,
+        boolean updateCircularList) {
       final List<INode> postCreated = postDiff.created != null?
           postDiff.created: Collections.<INode>emptyList();
       final List<INode> postDeleted = postDiff.deleted != null?
@@ -350,14 +381,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
           c = createdIterator.hasNext()? createdIterator.next(): null;
         } else if (cmp > 0) {
           // case 2: only in d-list
-          Triple<Integer, INode, Integer> triple = delete(d);
+          Triple<Integer, INode, Integer> triple = delete(d, 
+              updateCircularList);
           if (deletedINodeProcesser != null) {
             deletedINodeProcesser.process(triple.middle);
           }
           d = deletedIterator.hasNext()? deletedIterator.next(): null;
         } else {
           // case 3: in both c-list and d-list 
-          final Triple<Integer, INode, Integer> triple = modify(d, c);
+          final Triple<Integer, INode, Integer> triple = modify(d, c,
+              updateCircularList);
           if (deletedINodeProcesser != null) {
             deletedINodeProcesser.process(triple.middle);
           }
@@ -386,6 +419,74 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
           + "{created=" + toString(created)
           + ", deleted=" + toString(deleted) + "}";
     }
+    
+    /** Serialize {@link #created} */
+    private void writeCreated(DataOutput out) throws IOException {
+      if (created != null) {
+        out.writeInt(created.size());
+        for (INode node : created) {
+          // For INode in created list, we only need to record its local name 
+          byte[] name = node.getLocalNameBytes();
+          out.writeShort(name.length);
+          out.write(name);
+        }
+      } else {
+        out.writeInt(0);
+      }     
+    }
+    
+    /** Serialize {@link #deleted} */
+    private void writeDeleted(DataOutput out) throws IOException {
+      if (deleted != null) {
+        out.writeInt(deleted.size());
+        for (INode node : deleted) {
+          if (node.isDirectory()) {
+            FSImageSerialization.writeINodeDirectory((INodeDirectory) node, out);
+          } else { // INodeFile
+            // we write the block information only for INodeFile node when the
+            // node is only stored in the deleted list or the node is not a
+            // snapshot copy
+            int createdIndex = search(created, node);
+            if (createdIndex < 0) {
+              FSImageSerialization.writeINodeFile((INodeFile) node, out, true);
+            } else {
+              INodeFile cNode = (INodeFile) created.get(createdIndex);
+              INodeFile dNode = (INodeFile) node;
+              // A corner case here: after deleting a Snapshot, when combining
+              // SnapshotDiff, we may put two inodes sharing the same name but
+              // with totally different blocks in the created and deleted list of
+              // the same SnapshotDiff.
+              if (cNode.getBlocks() == dNode.getBlocks()) {
+                FSImageSerialization.writeINodeFile(dNode, out, false);
+              } else {
+                FSImageSerialization.writeINodeFile(dNode, out, true);
+              }
+            }
+          }
+        }
+      } else {
+        out.writeInt(0);
+      }
+    }
+    
+    /** Serialize to out */
+    private void write(DataOutput out) throws IOException {
+      writeCreated(out);
+      writeDeleted(out);    
+    }
+    
+    /** @return The list of INodeDirectory contained in the deleted list */
+    private List<INodeDirectory> getDirsInDeleted() {
+      List<INodeDirectory> dirList = new ArrayList<INodeDirectory>();
+      if (deleted != null) {
+        for (INode node : deleted) {
+          if (node.isDirectory()) {
+            dirList.add((INodeDirectory) node);
+          }
+        }
+      }
+      return dirList;
+    }
   }
   
   /**
@@ -406,7 +507,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
    *   s_k     = s_{k+1} - d_k = (current state) - d_n - d_{n-1} - ... - d_k.
    * </pre>
    */
-  class SnapshotDiff implements Comparable<Snapshot> {
+  public class SnapshotDiff implements Comparable<Snapshot> {
     /** The snapshot will be obtained after this diff is applied. */
     final Snapshot snapshot;
     /** The size of the children list at snapshot creation time. */
@@ -419,7 +520,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
      */
     private SnapshotDiff posteriorDiff;
     /** The children list diff. */
-    private final Diff diff = new Diff();
+    private final Diff diff;
     /** The snapshot inode data.  It is null when there is no change. */
     private INodeDirectory snapshotINode = null;
 
@@ -428,6 +529,25 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
 
       this.snapshot = snapshot;
       this.childrenSize = dir.getChildrenList(null).size();
+      this.diff = new Diff();
+    }
+
+    /** Constructor used by FSImage loading */
+    SnapshotDiff(Snapshot snapshot,
+        int childrenSize, INodeDirectory snapshotINode,
+        SnapshotDiff posteriorDiff, List<INode> createdList,
+        List<INode> deletedList) {
+      this.snapshot = snapshot;
+      this.childrenSize = childrenSize;
+      this.snapshotINode = snapshotINode;
+      this.posteriorDiff = posteriorDiff;
+      this.diff = new Diff();
+      diff.created = createdList;
+      diff.deleted = deletedList;
+    }
+    
+    public Diff getDiff() {
+      return diff;
     }
 
     /** Compare diffs with snapshot ID. */
@@ -485,7 +605,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
           if (children == null) {
             final Diff combined = new Diff();
             for(SnapshotDiff d = SnapshotDiff.this; d != null; d = d.posteriorDiff) {
-              combined.combinePostDiff(d.diff, null);
+              combined.combinePostDiff(d.diff, null, false);
             }
             children = combined.apply2Current(ReadOnlyList.Util.asList(
                 INodeDirectoryWithSnapshot.this.getChildrenList(null)));
@@ -538,6 +658,36 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
           + (posteriorDiff == null? null: posteriorDiff.snapshot)
           + ") childrenSize=" + childrenSize + ", " + diff;
     }
+    
+    /** Serialize fields to out */
+    void write(DataOutput out) throws IOException {
+      out.writeInt(childrenSize);
+      // 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
+      // record the full path of its root.
+      byte[] fullPath = DFSUtil.string2Bytes(snapshot.getRoot()
+          .getFullPathName());
+      out.writeShort(fullPath.length);
+      out.write(fullPath);
+      // write snapshotINode
+      if (isSnapshotRoot()) {
+        out.writeBoolean(true);
+      } else {
+        out.writeBoolean(false);
+        if (snapshotINode != null) {
+          out.writeBoolean(true);
+          FSImageSerialization.writeINodeDirectory(snapshotINode, out);
+        } else {
+          out.writeBoolean(false);
+        }
+      }
+      // Write diff. Node need to write poseriorDiff, since diffs is a list.
+      diff.write(out);
+    }
+    
+    private List<INodeDirectory> getSnapshotDirectory() {
+      return diff.getDirsInDeleted();
+    }
   }
   
   /** An interface for passing a method to process inodes. */
@@ -598,7 +748,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
               ((INodeFile)inode).collectSubtreeBlocksAndClear(collectedBlocks);
             }
           }
-        });
+        }, true);
 
         previousDiff.posteriorDiff = diffToRemove.posteriorDiff;
         diffToRemove.posteriorDiff = null;
@@ -606,7 +756,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       return diffToRemove;
     }
   }
-
+  
+  /** Insert a SnapshotDiff to the head of diffs */
+  public void insertDiff(SnapshotDiff diff) {
+    diffs.add(0, diff);
+  }
+  
   /** Add a {@link SnapshotDiff} for the given snapshot and directory. */
   SnapshotDiff addSnapshotDiff(Snapshot snapshot, INodeDirectory dir,
       boolean isSnapshotCreation) {
@@ -623,7 +778,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
     return d;
   }
-
+  
   SnapshotDiff getLastSnapshotDiff() {
     final int n = diffs.size();
     return n == 0? null: diffs.get(n - 1);
@@ -656,7 +811,6 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   /**
    * @return {@link #snapshots}
    */
-  @VisibleForTesting
   List<SnapshotDiff> getSnapshotDiffs() {
     return diffs;
   }
@@ -709,7 +863,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     }
 
     final Pair<? extends INode, ? extends INode> p = child.createSnapshotCopy();
-    diff.diff.modify(p.right, p.left);
+    diff.diff.modify(p.right, p.left, true);
     return p;
   }
 
@@ -734,7 +888,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
     Triple<Integer, INode, Integer> undoInfo = null;
     if (latest != null) {
       diff = checkAndAddLatestDiff(latest);
-      undoInfo = diff.delete(child);
+      undoInfo = diff.delete(child, true);
     }
     final INode removed = super.removeChild(child, null);
     if (removed == null && undoInfo != null) {
@@ -794,4 +948,25 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   public String toString() {
     return super.toString() + ", diffs=" + getSnapshotDiffs();
   }
+  
+  /**
+   * Get all the INodeDirectory stored in the deletes lists.
+   * 
+   * @param snapshotDirMap
+   *          A HashMap storing all the INodeDirectory stored in the deleted
+   *          lists, with their associated full Snapshot.
+   * @return The number of INodeDirectory returned.
+   */
+  public int getSnapshotDirectory(
+      Map<Snapshot, List<INodeDirectory>> snapshotDirMap) {
+    int dirNum = 0;
+    for (SnapshotDiff sdiff : diffs) {
+      List<INodeDirectory> list = sdiff.getSnapshotDirectory();
+      if (list.size() > 0) {
+        snapshotDirMap.put(sdiff.snapshot, list);
+        dirNum += list.size();
+      }
+    }
+    return dirNum;
+  }
 }

+ 15 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileSnapshot.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 
 /**
  *  INode representing a snapshot of a file.
@@ -25,18 +27,27 @@ import org.apache.hadoop.classification.InterfaceAudience;
 @InterfaceAudience.Private
 public class INodeFileSnapshot extends INodeFileWithSnapshot {
   /** The file size at snapshot creation time. */
-  final long size;
+  final long snapshotFileSize;
 
   INodeFileSnapshot(INodeFileWithSnapshot f) {
     super(f);
-    this.size = f.computeFileSize(true);
-    f.insert(this);
+    this.snapshotFileSize = f.computeFileSize(true);
+    f.insertAfter(this);
+  }
+  
+  /**
+   * A constructor that only sets the basic attributes and the size. Used while
+   * loading {@link FSImage}
+   */
+  public INodeFileSnapshot(INodeFile f, long size) {
+    super(f);
+    this.snapshotFileSize = size;
   }
 
   @Override
   public long computeFileSize(boolean includesBlockInfoUnderConstruction) {
     //ignore includesBlockInfoUnderConstruction 
     //since files in a snapshot are considered as closed.
-    return size;
+    return snapshotFileSize;
   }
 }

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

@@ -32,7 +32,7 @@ public class INodeFileUnderConstructionSnapshot
   INodeFileUnderConstructionSnapshot(INodeFileUnderConstructionWithSnapshot f) {
     super(f, f.getClientName(), f.getClientMachine(), f.getClientNode());
     this.size = f.computeFileSize(true);
-    f.insert(this);
+    f.insertAfter(this);
   }
 
   @Override

+ 21 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java

@@ -74,10 +74,30 @@ public class INodeFileUnderConstructionWithSnapshot
   }
 
   @Override
-  public void insert(FileWithSnapshot inode) {
+  public void insertAfter(FileWithSnapshot inode) {
     inode.setNext(this.getNext());
     this.setNext(inode);
   }
+  
+  @Override
+  public void insertBefore(FileWithSnapshot inode) {
+    inode.setNext(this);
+    if (this.next == null || this.next == this) {
+      this.next = inode;
+      return;
+    }
+    FileWithSnapshot previous = Util.getPrevious(this);
+    previous.setNext(inode);
+  }
+
+  @Override
+  public void removeSelf() {
+    if (this.next != null && this.next != this) {
+      FileWithSnapshot previous = Util.getPrevious(this);
+      previous.setNext(next);
+    }
+    this.next = null;
+  }
 
   @Override
   public short getBlockReplication() {

+ 21 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.Util;
 
 /**
  * Represent an {@link INodeFile} that is snapshotted.
@@ -70,10 +69,30 @@ public class INodeFileWithSnapshot extends INodeFile
   }
 
   @Override
-  public void insert(FileWithSnapshot inode) {
+  public void insertAfter(FileWithSnapshot inode) {
     inode.setNext(this.getNext());
     this.setNext(inode);
   }
+  
+  @Override
+  public void insertBefore(FileWithSnapshot inode) {
+    inode.setNext(this);
+    if (this.next == null || this.next == this) {
+      this.next = inode;
+      return;
+    }
+    FileWithSnapshot previous = Util.getPrevious(this);
+    previous.setNext(inode);
+  }
+
+  @Override
+  public void removeSelf() {
+    if (this.next != null && this.next != this) {
+      FileWithSnapshot previous = Util.getPrevious(this);
+      previous.setNext(next);
+    }
+    this.next = null;
+  }
 
   @Override
   public short getBlockReplication() {

+ 26 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java

@@ -17,9 +17,14 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import java.io.DataOutput;
+import java.io.IOException;
 import java.util.Comparator;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -69,6 +74,13 @@ public class Snapshot implements Comparable<byte[]> {
     public INode getChild(byte[] name, Snapshot snapshot) {
       return getParent().getChild(name, snapshot);
     }
+    
+    @Override
+    public String getFullPathName() {
+      return getParent().getFullPathName() + Path.SEPARATOR
+          + HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR
+          + this.getLocalName();
+    }
   }
 
   /** Snapshot ID. */
@@ -83,7 +95,13 @@ public class Snapshot implements Comparable<byte[]> {
     this.root.setLocalName(name);
     this.root.setParent(dir);
   }
-
+  
+  /** Constructor used when loading fsimage */
+  Snapshot(int id, INodeDirectory root) {
+    this.id = id;
+    this.root = new Root(root);
+  }
+  
   /** @return the root directory of the snapshot. */
   public Root getRoot() {
     return root;
@@ -113,4 +131,11 @@ public class Snapshot implements Comparable<byte[]> {
   public String toString() {
     return getClass().getSimpleName() + "." + root.getLocalName();
   }
+  
+  /** Serialize the fields to out */
+  void write(DataOutput out) throws IOException {
+    out.writeInt(id);
+    // write root
+    FSImageSerialization.writeINodeDirectory(root, out);
+  }
 }

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

@@ -0,0 +1,351 @@
+/**
+ * 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.server.namenode.snapshot;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormat.Loader;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
+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.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.Diff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.SnapshotDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+
+/**
+ * A helper class defining static methods for reading/writing snapshot related
+ * information from/to FSImage.
+ */
+public class SnapshotFSImageFormat {
+
+  /**
+   * Save snapshots and snapshot quota for a snapshottable directory.
+   * @param current The directory that the snapshots belongs to.
+   * @param out The {@link DataOutputStream} to write.
+   * @throws IOException
+   */
+  public static void saveSnapshots(INodeDirectorySnapshottable current,
+      DataOutputStream out) throws IOException {
+    // list of snapshots in snapshotsByNames
+    ReadOnlyList<Snapshot> snapshots = current.getSnapshotsByNames();
+    out.writeInt(snapshots.size());
+    for (Snapshot ss : snapshots) {
+      // write the snapshot
+      ss.write(out);
+    }
+    // snapshot quota
+    out.writeInt(current.getSnapshotQuota());
+  }
+  
+  /**
+   * Save SnapshotDiff list for an INodeDirectoryWithSnapshot.
+   * @param sNode The directory that the SnapshotDiff list belongs to.
+   * @param out The {@link DataOutputStream} to write.
+   */
+  public static void saveSnapshotDiffs(INodeDirectoryWithSnapshot sNode,
+      DataOutputStream out) throws IOException {
+    // # of SnapshotDiff
+    List<SnapshotDiff> diffs = sNode.getSnapshotDiffs();
+    // Record the SnapshotDiff in reversed order, so that we can find the
+    // correct reference for INodes in the created list when loading the
+    // FSImage
+    out.writeInt(diffs.size());
+    for (int i = diffs.size() - 1; i >= 0; i--) {
+      SnapshotDiff sdiff = diffs.get(i);
+      sdiff.write(out);
+    }
+  }
+  
+  /**
+   * Load a node stored in the created list from fsimage.
+   * @param createdNodeName The name of the created node.
+   * @param parent The directory that the created list belongs to.
+   * @return The created node.
+   */
+  private static INode loadCreated(byte[] createdNodeName,
+      INodeDirectoryWithSnapshot parent) throws IOException {
+    // the INode in the created list should be a reference to another INode
+    // in posterior SnapshotDiffs or one of the current children
+    for (SnapshotDiff postDiff : parent.getSnapshotDiffs()) {
+      INode created = findCreated(createdNodeName, postDiff.getDiff());
+      if (created != null) {
+        return created;
+      } // else go to the next SnapshotDiff
+    } 
+    // use the current child
+    INode currentChild = parent.getChild(createdNodeName, null);
+    if (currentChild == null) {
+      throw new IOException("Cannot find an INode associated with the INode "
+          + DFSUtil.bytes2String(createdNodeName)
+          + " in created list while loading FSImage.");
+    }
+    return currentChild;
+  }
+  
+  /**
+   * Search the given {@link Diff} to find an inode matching the specific name.
+   * @param createdNodeName The name of the node for searching.
+   * @param diff The given {@link Diff} where to search the node.
+   * @return The matched inode. Return null if no matched inode can be found.
+   */
+  private static INode findCreated(byte[] createdNodeName, Diff diff) {
+    INode c = diff.searchCreated(createdNodeName);
+    INode d = diff.searchDeleted(createdNodeName);
+    if (c == null && d != null) {
+      // if an INode with the same name is only contained in the deleted
+      // list, then the node should be the snapshot copy of a deleted
+      // node, and the node in the created list should be its reference 
+      return d;
+    } else if (c != null && d != null) {
+      // in a posterior SnapshotDiff, if the created/deleted lists both
+      // contains nodes with the same name (c & d), there are two
+      // possibilities:
+      // 
+      // 1) c and d are used to represent a modification, and 
+      // 2) d indicates the deletion of the node, while c was originally
+      // contained in the created list of a later snapshot, but c was
+      // moved here because of the snapshot deletion.
+      // 
+      // For case 1), c and d should be both INodeFile and should share
+      // the same blockInfo list.
+      if (c.isFile()
+          && ((INodeFile) c).getBlocks() == ((INodeFile) d).getBlocks()) {
+        return c;
+      } else {
+        return d;
+      }
+    }
+    return null;
+  }
+  
+  /**
+   * Load the created list from fsimage.
+   * @param parent The directory that the created list belongs to.
+   * @param in The {@link DataInputStream} to read.
+   * @return The created list.
+   */
+  private static List<INode> loadCreatedList(INodeDirectoryWithSnapshot parent,
+      DataInputStream in) throws IOException {
+    // read the size of the created list
+    int createdSize = in.readInt();
+    List<INode> createdList = new ArrayList<INode>(createdSize);
+    for (int i = 0; i < createdSize; i++) {
+      byte[] createdNodeName = new byte[in.readShort()];
+      in.readFully(createdNodeName);
+      INode created = loadCreated(createdNodeName, parent);
+      createdList.add(created);
+    }
+    return createdList;
+  }
+    
+  /**
+   * Load the deleted list from the fsimage.
+   * 
+   * @param parent The directory that the deleted list belongs to.
+   * @param createdList The created list associated with the deleted list in 
+   *                    the same Diff.
+   * @param in The {@link DataInputStream} to read.
+   * @param loader The {@link Loader} instance. Used to call the
+   *               {@link Loader#loadINode(DataInputStream)} method.
+   * @return The deleted list.
+   */
+  private static List<INode> loadDeletedList(INodeDirectoryWithSnapshot parent,
+      List<INode> createdList, DataInputStream in, FSImageFormat.Loader loader)
+      throws IOException {
+    int deletedSize = in.readInt();
+    List<INode> deletedList = new ArrayList<INode>(deletedSize);
+    for (int i = 0; i < deletedSize; i++) {
+      byte[] deletedNodeName = new byte[in.readShort()];
+      in.readFully(deletedNodeName);
+      INode deleted = loader.loadINode(in);
+      deleted.setLocalName(deletedNodeName);
+      deletedList.add(deleted);
+      // set parent: the parent field of an INode in the deleted list is not 
+      // useful, but set the parent here to be consistent with the original 
+      // fsdir tree.
+      deleted.setParent(parent);
+      if (deleted instanceof INodeFile
+          && ((INodeFile) deleted).getBlocks() == null) {
+        // if deleted is an INodeFile, and its blocks is null, then deleted
+        // must be an INodeFileWithLink, and we need to rebuild its next link
+        int c = Collections.binarySearch(createdList, deletedNodeName);
+        if (c < 0) {
+          throw new IOException(
+              "Cannot find the INode linked with the INode "
+                  + DFSUtil.bytes2String(deletedNodeName)
+                  + " in deleted list while loading FSImage.");
+        }
+        // deleted must be an INodeFileSnapshot
+        INodeFileSnapshot deletedWithLink = (INodeFileSnapshot) deleted;
+        INodeFile cNode = (INodeFile) createdList.get(c);
+        INodeFileWithSnapshot cNodeWithLink = (INodeFileWithSnapshot) cNode;
+        deletedWithLink.setBlocks(cNode.getBlocks());
+        // insert deleted into the circular list
+        cNodeWithLink.insertBefore(deletedWithLink);
+      }
+    }
+    return deletedList;
+  }
+  
+  /**
+   * Load snapshots and snapshotQuota for a Snapshottable directory.
+   * @param snapshottableParent The snapshottable directory for loading.
+   * @param numSnapshots The number of snapshots that the directory has.
+   * @param in The {@link DataInputStream} instance to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   */
+  public static void loadSnapshotList(
+      INodeDirectorySnapshottable snapshottableParent, int numSnapshots,
+      DataInputStream in, FSImageFormat.Loader loader) throws IOException {
+    for (int i = 0; i < numSnapshots; i++) {
+      // read snapshots
+      Snapshot ss = loadSnapshot(snapshottableParent, in, loader);
+      snapshottableParent.addSnapshot(ss);
+    }
+    int snapshotQuota = in.readInt();
+    snapshottableParent.setSnapshotQuota(snapshotQuota);
+  }
+  
+  /**
+   * Load a {@link Snapshot} from fsimage.
+   * @param parent The directory that the snapshot belongs to.
+   * @param in The {@link DataInputStream} instance to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   * @return The snapshot.
+   */
+  private static Snapshot loadSnapshot(INodeDirectorySnapshottable parent,
+      DataInputStream in, FSImageFormat.Loader loader) throws IOException {
+    int snapshotId = in.readInt();
+    byte[] snapshotName = new byte[in.readShort()];
+    in.readFully(snapshotName);
+    INode rootNode = loader.loadINode(in);
+    rootNode.setLocalName(snapshotName);
+    rootNode.setParent(parent);
+    return new Snapshot(snapshotId, (INodeDirectory) rootNode);
+  }
+  
+  /**
+   * Load the {@link SnapshotDiff} list for the INodeDirectoryWithSnapshot
+   * directory.
+   * @param snapshottableParent The snapshottable directory for loading.
+   * @param numSnapshotDiffs The number of {@link SnapshotDiff} that the 
+   *                         directory has.
+   * @param in The {@link DataInputStream} instance to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   */
+  public static void loadSnapshotDiffList(
+      INodeDirectoryWithSnapshot parentWithSnapshot, int numSnapshotDiffs,
+      DataInputStream in, FSImageFormat.Loader loader)
+      throws IOException {
+    for (int i = 0; i < numSnapshotDiffs; i++) {
+      SnapshotDiff diff = loadSnapshotDiff(parentWithSnapshot, in, loader);
+      parentWithSnapshot.insertDiff(diff);
+    }
+  }
+  
+  /**
+   * Use the given full path to a {@link Root} directory to find the
+   * associated snapshot.
+   */
+  private static Snapshot findSnapshot(String sRootFullPath, FSDirectory fsdir)
+      throws IOException {
+    // find the root
+    INode root = fsdir.getINode(sRootFullPath);
+    INodeDirectorySnapshottable snapshotRoot = INodeDirectorySnapshottable
+        .valueOf(root.getParent(), root.getParent().getFullPathName());
+    // find the snapshot
+    return snapshotRoot.getSnapshot(root.getLocalNameBytes());
+  }
+  
+  /**
+   * Load the snapshotINode field of {@link SnapshotDiff}.
+   * @param snapshot The Snapshot associated with the {@link SnapshotDiff}.
+   * @param in The {@link DataInputStream} to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   * @return The snapshotINode.
+   */
+  private static INodeDirectory loadSnapshotINodeInSnapshotDiff(
+      Snapshot snapshot, DataInputStream in, FSImageFormat.Loader loader)
+      throws IOException {
+    // read the boolean indicating whether snapshotINode == Snapshot.Root
+    boolean useRoot = in.readBoolean();      
+    if (useRoot) {
+      return snapshot.getRoot();
+    } else {
+      // another boolean is used to indicate whether snapshotINode is non-null
+      if (in.readBoolean()) {
+        byte[] localName = new byte[in.readShort()];
+        in.readFully(localName);
+        INodeDirectory snapshotINode = (INodeDirectory) loader.loadINode(in);
+        snapshotINode.setLocalName(localName);
+        return snapshotINode;
+      }
+    }
+    return null;
+  }
+   
+  /**
+   * Load {@link SnapshotDiff} from fsimage.
+   * @param parent The directory that the SnapshotDiff belongs to.
+   * @param in The {@link DataInputStream} instance to read.
+   * @param loader The {@link Loader} instance that this loading procedure is 
+   *               using.
+   * @return A {@link SnapshotDiff}.
+   */
+  private static SnapshotDiff loadSnapshotDiff(
+      INodeDirectoryWithSnapshot parent, DataInputStream in,
+      FSImageFormat.Loader loader) throws IOException {
+    // 1. Load SnapshotDiff#childrenSize
+    int childrenSize = in.readInt();
+    // 2. Read the full path of the Snapshot's Root, identify 
+    //    SnapshotDiff#Snapshot
+    Snapshot snapshot = findSnapshot(FSImageSerialization.readString(in),
+        loader.getFSDirectoryInLoading());
+    
+    // 3. Load SnapshotDiff#snapshotINode 
+    INodeDirectory snapshotINode = loadSnapshotINodeInSnapshotDiff(snapshot,
+        in, loader);
+    
+    // 4. Load the created list in SnapshotDiff#Diff
+    List<INode> createdList = loadCreatedList(parent, in);
+    
+    // 5. Load the deleted list in SnapshotDiff#Diff
+    List<INode> deletedList = loadDeletedList(parent, createdList, in, loader);
+    
+    // 6. Compose the SnapshotDiff
+    SnapshotDiff sdiff = parent.new SnapshotDiff(snapshot, childrenSize,
+        snapshotINode, parent.getSnapshotDiffs().isEmpty() ? null : parent
+            .getSnapshotDiffs().get(0), createdList, deletedList);
+    return sdiff;
+  }
+  
+}

+ 25 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -48,7 +50,7 @@ public class SnapshotManager implements SnapshotStats {
   private final AtomicInteger numSnapshottableDirs = new AtomicInteger();
   private final AtomicInteger numSnapshots = new AtomicInteger();
 
-  private int snapshotID = 0;
+  private int snapshotCounter = 0;
   
   /** All snapshottable directories in the namesystem. */
   private final List<INodeDirectorySnapshottable> snapshottables
@@ -117,10 +119,10 @@ public class SnapshotManager implements SnapshotStats {
     final INodesInPath i = fsdir.getMutableINodesInPath(path);
     final INodeDirectorySnapshottable srcRoot
         = INodeDirectorySnapshottable.valueOf(i.getLastINode(), path);
-    srcRoot.addSnapshot(snapshotID, snapshotName);
+    srcRoot.addSnapshot(snapshotCounter, snapshotName);
       
     //create success, update id
-    snapshotID++;
+    snapshotCounter++;
     numSnapshots.getAndIncrement();
   }
   
@@ -180,6 +182,26 @@ public class SnapshotManager implements SnapshotStats {
     return numSnapshots.get();
   }
   
+  /**
+   * Write {@link #snapshotCounter}, {@link #numSnapshots}, and
+   * {@link #numSnapshottableDirs} to the DataOutput.
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(snapshotCounter);
+    out.writeInt(numSnapshots.get());
+    out.writeInt(numSnapshottableDirs.get());
+  }
+  
+  /**
+   * Read values of {@link #snapshotCounter}, {@link #numSnapshots}, and
+   * {@link #numSnapshottableDirs} from the DataInput
+   */
+  public void read(DataInput in) throws IOException {
+    snapshotCounter = in.readInt();
+    numSnapshots.set(in.readInt());
+    numSnapshottableDirs.set(in.readInt());
+  }
+  
   /**
    * @return All the current snapshottable directories
    */

+ 162 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java

@@ -0,0 +1,162 @@
+/**
+ * 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.server.namenode;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
+import org.apache.hadoop.hdfs.util.Canceler;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test FSImage save/load when Snapshot is supported
+ */
+public class TestFSImageWithSnapshot {
+  static final long seed = 0;
+  static final short REPLICATION = 3;
+  static final long BLOCKSIZE = 1024;
+  static final long txid = 1;
+
+  private final Path dir = new Path("/TestSnapshot");
+  private static String testDir =
+      System.getProperty("test.build.data", "build/test/data");
+  
+  Configuration conf;
+  MiniDFSCluster cluster;
+  FSNamesystem fsn;
+  DistributedFileSystem hdfs;
+  
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Testing steps:
+   * <pre>
+   * 1. Creating/modifying directories/files while snapshots are being taken.
+   * 2. Dump the FSDirectory tree of the namesystem.
+   * 3. Save the namesystem to a temp file (FSImage saving).
+   * 4. Restart the cluster and format the namesystem.
+   * 5. Load the namesystem from the temp file (FSImage loading).
+   * 6. Dump the FSDirectory again and compare the two dumped string.
+   * </pre>
+   */
+  @Test
+  public void testSaveLoadImage() throws Exception {
+    // make changes to the namesystem
+    hdfs.mkdirs(dir);
+    hdfs.allowSnapshot(dir.toString());
+    hdfs.createSnapshot(dir, "s0");
+    
+    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);
+    
+    hdfs.createSnapshot(dir, "s1");
+    
+    Path sub2 = new Path(dir, "sub2");
+    Path sub2file1 = new Path(sub2, "sub2file1");
+    Path sub2file2 = new Path(sub2, "sub2file2");
+    DFSTestUtil.createFile(hdfs, sub2file1, BLOCKSIZE, REPLICATION, seed);
+    DFSTestUtil.createFile(hdfs, sub2file2, BLOCKSIZE, REPLICATION, seed);
+    hdfs.setReplication(sub1file1, (short) (REPLICATION - 1));
+    hdfs.delete(sub1file2, true);
+    
+    hdfs.createSnapshot(dir, "s2");
+    hdfs.setOwner(sub2, "dr.who", "unknown");
+    hdfs.delete(sub2file2, true);
+    
+    // dump the fsdir tree
+    StringBuffer fsnStrBefore = fsn.getFSDirectory().rootDir
+        .dumpTreeRecursively();
+    
+    // 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();
+    }
+
+    // restart the cluster, and format the cluster
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(true)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    
+    // load the namesystem from the temp file
+    FSImageFormat.Loader loader = new FSImageFormat.Loader(conf, fsn);
+    fsn.writeLock();
+    try {
+      loader.load(dstFile);
+    } finally {
+      fsn.writeUnlock();
+    }
+    
+    // dump the fsdir tree again
+    StringBuffer fsnStrAfter = fsn.getFSDirectory().rootDir
+        .dumpTreeRecursively();
+    
+    // compare two dumped tree
+    System.out.println(fsnStrBefore.toString());
+    System.out.println("\n" + fsnStrAfter.toString());
+    assertEquals(fsnStrBefore.toString(), fsnStrAfter.toString());
+  }
+  
+  /**
+   * 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 getStorageFile(String dir, long imageTxId) {
+    return new File(dir, String.format("%s_%019d", NameNodeFile.IMAGE,
+        imageTxId));
+  }
+}

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

@@ -127,7 +127,6 @@ public class SnapshotTestHelper {
   static class TestDirectoryTree {
     /** Height of the directory tree */
     final int height;
-    final FileSystem fs;
     /** Top node of the directory tree */
     final Node topNode;
     /** A map recording nodes for each tree level */
@@ -138,12 +137,11 @@ public class SnapshotTestHelper {
      */
     TestDirectoryTree(int height, FileSystem fs) throws Exception {
       this.height = height;
-      this.fs = fs;
       this.topNode = new Node(new Path("/TestSnapshot"), 0,
           null, fs);
       this.levelMap = new HashMap<Integer, ArrayList<Node>>();
       addDirNode(topNode, 0);
-      genChildren(topNode, height - 1);
+      genChildren(topNode, height - 1, fs);
     }
 
     /**
@@ -164,9 +162,11 @@ public class SnapshotTestHelper {
      * 
      * @param parent The parent node
      * @param level The remaining levels to generate
+     * @param fs The FileSystem where to generate the files/dirs
      * @throws Exception
      */
-    void genChildren(Node parent, int level) throws Exception {
+    private void genChildren(Node parent, int level, FileSystem fs)
+        throws Exception {
       if (level == 0) {
         return;
       }
@@ -176,8 +176,8 @@ public class SnapshotTestHelper {
           "right" + ++id), height - level, parent, fs);
       addDirNode(parent.leftChild, parent.leftChild.level);
       addDirNode(parent.rightChild, parent.rightChild.level);
-      genChildren(parent.leftChild, level - 1);
-      genChildren(parent.rightChild, level - 1);
+      genChildren(parent.leftChild, level - 1, fs);
+      genChildren(parent.rightChild, level - 1, fs);
     }
 
     /**
@@ -246,7 +246,6 @@ public class SnapshotTestHelper {
        * directory creation/deletion
        */
       final ArrayList<Node> nonSnapshotChildren;
-      final FileSystem fs;
 
       Node(Path path, int level, Node parent,
           FileSystem fs) throws Exception {
@@ -254,7 +253,6 @@ public class SnapshotTestHelper {
         this.level = level;
         this.parent = parent;
         this.nonSnapshotChildren = new ArrayList<Node>();
-        this.fs = fs;
         fs.mkdirs(nodePath);
       }
 
@@ -262,8 +260,8 @@ public class SnapshotTestHelper {
        * Create files and add them in the fileList. Initially the last element
        * in the fileList is set to null (where we start file creation).
        */
-      void initFileList(String namePrefix, long fileLen, short replication,
-          long seed, int numFiles) throws Exception {
+      void initFileList(FileSystem fs, String namePrefix, long fileLen,
+          short replication, long seed, int numFiles) throws Exception {
         fileList = new ArrayList<Path>(numFiles);
         for (int i = 0; i < numFiles; i++) {
           Path file = new Path(nodePath, namePrefix + "-f" + i);

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeDirectoryWithSnapshot.java

@@ -148,7 +148,7 @@ public class TestINodeDirectoryWithSnapshot {
     // combine all diffs
     final Diff combined = diffs[0];
     for(int i = 1; i < diffs.length; i++) {
-      combined.combinePostDiff(diffs[i], null);
+      combined.combinePostDiff(diffs[i], null, false);
     }
 
     {
@@ -284,7 +284,7 @@ public class TestINodeDirectoryWithSnapshot {
         before = toString(diff);
       }
 
-      final Triple<Integer, INode, Integer> undoInfo = diff.delete(inode);
+      final Triple<Integer, INode, Integer> undoInfo = diff.delete(inode, true);
 
       if (testUndo) {
         final String after = toString(diff);
@@ -292,7 +292,7 @@ public class TestINodeDirectoryWithSnapshot {
         diff.undoDelete(inode, undoInfo);
         assertDiff(before, diff);
         //re-do
-        diff.delete(inode);
+        diff.delete(inode, true);
         assertDiff(after, diff);
       }
     }
@@ -314,7 +314,7 @@ public class TestINodeDirectoryWithSnapshot {
         before = toString(diff);
       }
 
-      final Triple<Integer, INode, Integer> undoInfo = diff.modify(oldinode, newinode);
+      final Triple<Integer, INode, Integer> undoInfo = diff.modify(oldinode, newinode, true);
 
       if (testUndo) {
         final String after = toString(diff);
@@ -322,7 +322,7 @@ public class TestINodeDirectoryWithSnapshot {
         diff.undoModify(oldinode, newinode, undoInfo);
         assertDiff(before, diff);
         //re-do
-        diff.modify(oldinode, newinode);
+        diff.modify(oldinode, newinode, true);
         assertDiff(after, diff);
       }
     }

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

@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree;
 import org.apache.hadoop.ipc.RemoteException;
@@ -55,7 +56,7 @@ public class TestSnapshot {
   private static final long seed = Time.now();
   protected static final short REPLICATION = 3;
   protected static final long BLOCKSIZE = 1024;
-  /** The number of times snapshots are created for a snapshottable directory  */
+  /** The number of times snapshots are created for a snapshottable directory */
   public static final int SNAPSHOT_ITERATION_NUMBER = 20;
   /** Height of directory tree used for testing */
   public static final int DIRECTORY_TREE_LEVEL = 5;
@@ -143,6 +144,49 @@ public class TestSnapshot {
     return nodes;
   }
 
+  /**
+   * Restart the cluster to check edit log applying and fsimage saving/loading
+   */
+  private void checkFSImage() throws Exception {
+    String rootDir = "/";
+    StringBuffer fsnStrBefore = fsn.getFSDirectory().getINode(rootDir)
+        .dumpTreeRecursively();
+    
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    // later check fsnStrMiddle to see if the edit log is recorded and applied
+    // correctly 
+    StringBuffer fsnStrMiddle = fsn.getFSDirectory().getINode(rootDir)
+        .dumpTreeRecursively();
+    
+    // save namespace and restart cluster
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    // dump the namespace loaded from fsimage
+    StringBuffer fsnStrAfter = fsn.getFSDirectory().getINode(rootDir)
+        .dumpTreeRecursively();
+    
+    System.out.println("================== Original FSDir ==================");
+    System.out.println(fsnStrBefore.toString());
+    System.out.println("================== FSDir After Applying Edit Logs ==================");
+    System.out.println(fsnStrMiddle.toString());
+    System.out.println("================ FSDir After FSImage Saving/Loading ================");
+    System.out.println(fsnStrAfter.toString());
+    System.out.println("====================================================");
+    assertEquals(fsnStrBefore.toString(), fsnStrMiddle.toString());
+    assertEquals(fsnStrBefore.toString(), fsnStrAfter.toString());
+  }
   
   /**
    * Main test, where we will go in the following loop:
@@ -184,6 +228,9 @@ public class TestSnapshot {
       Modification chown = new FileChown(chownDir.nodePath, hdfs, userGroup[0],
           userGroup[1]);
       modifyCurrentDirAndCheckSnapshots(new Modification[]{chmod, chown});
+      
+      // check fsimage saving/loading
+      checkFSImage();
     }
     System.out.println("XXX done:");
     SnapshotTestHelper.dumpTreeRecursively(fsn.getFSDirectory().getINode("/"));
@@ -244,7 +291,8 @@ public class TestSnapshot {
     for (TestDirectoryTree.Node node : nodes) {
       // If the node does not have files in it, create files
       if (node.fileList == null) {
-        node.initFileList(node.nodePath.getName(), BLOCKSIZE, REPLICATION, seed, 6);
+        node.initFileList(hdfs, node.nodePath.getName(), BLOCKSIZE,
+            REPLICATION, seed, 5);
       }
       
       //
@@ -270,18 +318,21 @@ public class TestSnapshot {
       Modification delete = new FileDeletion(
           node.fileList.get((node.nullFileIndex + 1) % node.fileList.size()),
           hdfs);
-      Modification append = new FileAppend(
-          node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
-          hdfs, (int) BLOCKSIZE);
+
+      // 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(
-          node.fileList.get((node.nullFileIndex + 3) % node.fileList.size()),
+          node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
           hdfs, genRandomPermission());
       String[] userGroup = genRandomOwner();
       Modification chown = new FileChown(
-          node.fileList.get((node.nullFileIndex + 4) % node.fileList.size()),
+          node.fileList.get((node.nullFileIndex + 3) % node.fileList.size()),
           hdfs, userGroup[0], userGroup[1]);
       Modification replication = new FileChangeReplication(
-          node.fileList.get((node.nullFileIndex + 5) % node.fileList.size()),
+          node.fileList.get((node.nullFileIndex + 4) % node.fileList.size()),
           hdfs, (short) (random.nextInt(REPLICATION) + 1));
       node.nullFileIndex = (node.nullFileIndex + 1) % node.fileList.size();
       Modification dirChange = new DirCreationOrDeletion(node.nodePath, hdfs,
@@ -289,7 +340,8 @@ public class TestSnapshot {
       
       mList.add(create);
       mList.add(delete);
-      mList.add(append);
+      //TODO
+      //mList.add(append);
       mList.add(chmod);
       mList.add(chown);
       mList.add(replication);
@@ -606,7 +658,7 @@ public class TestSnapshot {
   /**
    * Directory creation or deletion.
    */
-  static class DirCreationOrDeletion extends Modification {
+  class DirCreationOrDeletion extends Modification {
     private final TestDirectoryTree.Node node;
     private final boolean isCreation;
     private final Path changedPath;
@@ -656,15 +708,16 @@ public class TestSnapshot {
       if (isCreation) {
         // creation
         TestDirectoryTree.Node newChild = new TestDirectoryTree.Node(
-            changedPath, node.level + 1, node, node.fs);
+            changedPath, node.level + 1, node, hdfs);
         // create file under the new non-snapshottable directory
-        newChild.initFileList(node.nodePath.getName(), BLOCKSIZE, REPLICATION, seed, 2);
+        newChild.initFileList(hdfs, node.nodePath.getName(), BLOCKSIZE,
+            REPLICATION, seed, 2);
         node.nonSnapshotChildren.add(newChild);
       } else {
         // deletion
         TestDirectoryTree.Node childToDelete = node.nonSnapshotChildren
             .remove(node.nonSnapshotChildren.size() - 1);
-        node.fs.delete(childToDelete.nodePath, true);
+        hdfs.delete(childToDelete.nodePath, true);
       }
     }
 

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

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.SnapshotDiff;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.ipc.RemoteException;
 import org.junit.After;
 import org.junit.Before;
@@ -85,7 +86,7 @@ public class TestSnapshotRename {
    */
   private void checkSnapshotList(INodeDirectorySnapshottable srcRoot,
       String[] sortedNames, String[] names) {
-    List<Snapshot> listByName = srcRoot.getSnapshotsByNames();
+    ReadOnlyList<Snapshot> listByName = srcRoot.getSnapshotsByNames();
     assertEquals(sortedNames.length, listByName.size());
     for (int i = 0; i < listByName.size(); i++) {
       assertEquals(sortedNames[i], listByName.get(i).getRoot().getLocalName());