Browse Source

HDFS-5632. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature. Contributed by jing9

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1550917 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 years ago
parent
commit
44a6560b5d
31 changed files with 830 additions and 820 deletions
  1. 4 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java
  3. 29 40
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  4. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  5. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  6. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  7. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  8. 21 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  9. 263 99
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  10. 13 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  11. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  12. 33 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  13. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  14. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
  15. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  17. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  18. 321 462
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  19. 1 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  20. 27 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  21. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
  22. 11 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  25. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
  26. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
  27. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java
  28. 45 58
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  29. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
  30. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  31. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java

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

@@ -249,6 +249,10 @@ Trunk (Unreleased)
     HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
     via jing9)
 
+    HDFS-5632. Flatten INodeDirectory hierarchy: Replace
+    INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
+    (jing9 via szetszwo)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java

@@ -153,6 +153,10 @@ public final class DirectoryWithQuotaFeature implements INode.Feature {
     verifyNamespaceQuota(nsDelta);
     verifyDiskspaceQuota(dsDelta);
   }
+  
+  boolean isQuotaSet() {
+    return nsQuota >= 0 || dsQuota >= 0;
+  }
 
   private String namespaceString() {
     return "namespace: " + (nsQuota < 0? "-": namespace + "/" + nsQuota);

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

@@ -67,7 +67,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 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.Snapshot.Root;
 import org.apache.hadoop.hdfs.util.ByteArray;
@@ -622,8 +621,7 @@ public class FSDirectory implements Closeable {
     // snapshot is taken on the dst tree, changes will be recorded in the latest
     // snapshot of the src tree.
     if (isSrcInSnapshot) {
-      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot(),
-          inodeMap);
+      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot());
       srcIIP.setLastINode(srcChild);
     }
     
@@ -692,11 +690,9 @@ public class FSDirectory implements Closeable {
         }
         // update modification time of dst and the parent of src
         final INode srcParent = srcIIP.getINode(-2);
-        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot(),
-            inodeMap);
+        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
         dstParent = dstIIP.getINode(-2); // refresh dstParent
-        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot(),
-            inodeMap);
+        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
         // update moved leases with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);     
 
@@ -734,11 +730,10 @@ public class FSDirectory implements Closeable {
         }
         
         if (isSrcInSnapshot) {
-          // srcParent must be an INodeDirectoryWithSnapshot instance since
-          // isSrcInSnapshot is true and src node has been removed from 
-          // srcParent 
-          ((INodeDirectoryWithSnapshot) srcParent).undoRename4ScrParent(
-              oldSrcChild.asReference(), srcChild, srcIIP.getLatestSnapshot());
+          // srcParent must have snapshot feature since isSrcInSnapshot is true
+          // and src node has been removed from srcParent 
+          srcParent.undoRename4ScrParent(oldSrcChild.asReference(), srcChild,
+              srcIIP.getLatestSnapshot());
         } else {
           // original srcChild is not in latest snapshot, we only need to add
           // the srcChild back
@@ -879,8 +874,7 @@ public class FSDirectory implements Closeable {
     // snapshot is taken on the dst tree, changes will be recorded in the latest
     // snapshot of the src tree.
     if (isSrcInSnapshot) {
-      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot(),
-          inodeMap);
+      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot());
       srcIIP.setLastINode(srcChild);
     }
     
@@ -958,11 +952,9 @@ public class FSDirectory implements Closeable {
         }
 
         final INode srcParent = srcIIP.getINode(-2);
-        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot(),
-            inodeMap);
+        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
         dstParent = dstIIP.getINode(-2);
-        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot(),
-            inodeMap);
+        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
         // update moved lease with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);
 
@@ -1019,9 +1011,9 @@ public class FSDirectory implements Closeable {
           withCount.getReferredINode().setLocalName(srcChildName);
         }
         
-        if (srcParent instanceof INodeDirectoryWithSnapshot) {
-          ((INodeDirectoryWithSnapshot) srcParent).undoRename4ScrParent(
-              oldSrcChild.asReference(), srcChild, srcIIP.getLatestSnapshot());
+        if (srcParent.isWithSnapshot()) {
+          srcParent.undoRename4ScrParent(oldSrcChild.asReference(), srcChild,
+              srcIIP.getLatestSnapshot());
         } else {
           // srcParent is not an INodeDirectoryWithSnapshot, we only need to add
           // the srcChild back
@@ -1030,9 +1022,9 @@ public class FSDirectory implements Closeable {
       }
       if (undoRemoveDst) {
         // Rename failed - restore dst
-        if (dstParent instanceof INodeDirectoryWithSnapshot) {
-          ((INodeDirectoryWithSnapshot) dstParent).undoRename4DstParent(
-              removedDst, dstIIP.getLatestSnapshot());
+        if (dstParent.isDirectory() && dstParent.asDirectory().isWithSnapshot()) {
+          dstParent.asDirectory().undoRename4DstParent(removedDst,
+              dstIIP.getLatestSnapshot());
         } else {
           addLastINodeNoQuotaCheck(dstIIP, removedDst);
         }
@@ -1163,8 +1155,7 @@ public class FSDirectory implements Closeable {
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
     }
-    inode.setPermission(permissions, inodesInPath.getLatestSnapshot(), 
-        inodeMap);
+    inode.setPermission(permissions, inodesInPath.getLatestSnapshot());
   }
 
   void setOwner(String src, String username, String groupname)
@@ -1189,11 +1180,10 @@ public class FSDirectory implements Closeable {
       throw new FileNotFoundException("File does not exist: " + src);
     }
     if (username != null) {
-      inode = inode.setUser(username, inodesInPath.getLatestSnapshot(),
-          inodeMap);
+      inode = inode.setUser(username, inodesInPath.getLatestSnapshot());
     }
     if (groupname != null) {
-      inode.setGroup(groupname, inodesInPath.getLatestSnapshot(), inodeMap);
+      inode.setGroup(groupname, inodesInPath.getLatestSnapshot());
     }
   }
 
@@ -1266,7 +1256,7 @@ public class FSDirectory implements Closeable {
       if(nodeToRemove == null) continue;
       
       nodeToRemove.setBlocks(null);
-      trgParent.removeChild(nodeToRemove, trgLatestSnapshot, null);
+      trgParent.removeChild(nodeToRemove, trgLatestSnapshot);
       inodeMap.remove(nodeToRemove);
       count++;
     }
@@ -1274,8 +1264,8 @@ public class FSDirectory implements Closeable {
     // update inodeMap
     removeFromInodeMap(Arrays.asList(allSrcInodes));
     
-    trgInode.setModificationTime(timestamp, trgLatestSnapshot, inodeMap);
-    trgParent.updateModificationTime(timestamp, trgLatestSnapshot, inodeMap);
+    trgInode.setModificationTime(timestamp, trgLatestSnapshot);
+    trgParent.updateModificationTime(timestamp, trgLatestSnapshot);
     // update quota on the parent directory ('count' files removed, 0 space)
     unprotectedUpdateCount(trgIIP, trgINodes.length-1, -count, 0);
   }
@@ -1419,7 +1409,7 @@ public class FSDirectory implements Closeable {
 
     // record modification
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
-    targetNode = targetNode.recordModification(latestSnapshot, inodeMap);
+    targetNode = targetNode.recordModification(latestSnapshot);
     iip.setLastINode(targetNode);
 
     // Remove the node from the namespace
@@ -1430,7 +1420,7 @@ public class FSDirectory implements Closeable {
 
     // set the parent's modification time
     final INodeDirectory parent = targetNode.getParent();
-    parent.updateModificationTime(mtime, latestSnapshot, inodeMap);
+    parent.updateModificationTime(mtime, latestSnapshot);
     if (removed == 0) {
       return 0;
     }
@@ -2203,8 +2193,7 @@ public class FSDirectory implements Closeable {
     final INodeDirectory parent = inodes[pos-1].asDirectory();
     boolean added = false;
     try {
-      added = parent.addChild(child, true, iip.getLatestSnapshot(),
-          inodeMap);
+      added = parent.addChild(child, true, iip.getLatestSnapshot());
     } catch (QuotaExceededException e) {
       updateCountNoQuotaCheck(iip, pos,
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
@@ -2242,7 +2231,7 @@ public class FSDirectory implements Closeable {
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final INode last = iip.getLastINode();
     final INodeDirectory parent = iip.getINode(-2).asDirectory();
-    if (!parent.removeChild(last, latestSnapshot, inodeMap)) {
+    if (!parent.removeChild(last, latestSnapshot)) {
       return -1;
     }
     INodeDirectory newParent = last.getParent();
@@ -2394,7 +2383,7 @@ public class FSDirectory implements Closeable {
       }
 
       final Snapshot latest = iip.getLatestSnapshot();
-      dirNode = dirNode.recordModification(latest, inodeMap);
+      dirNode = dirNode.recordModification(latest);
       dirNode.setQuota(nsQuota, dsQuota);
       return dirNode;
     }
@@ -2462,7 +2451,7 @@ public class FSDirectory implements Closeable {
     assert hasWriteLock();
     boolean status = false;
     if (mtime != -1) {
-      inode = inode.setModificationTime(mtime, latest, inodeMap);
+      inode = inode.setModificationTime(mtime, latest);
       status = true;
     }
     if (atime != -1) {
@@ -2473,7 +2462,7 @@ public class FSDirectory implements Closeable {
       if (atime <= inodeTime + getFSNamesystem().getAccessTimePrecision() && !force) {
         status =  false;
       } else {
-        inode.setAccessTime(atime, latest, inodeMap);
+        inode.setAccessTime(atime, latest);
         status = true;
       }
     } 

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

@@ -31,18 +31,18 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 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.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
@@ -55,11 +55,11 @@ 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.ModifyCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCacheDirectiveInfoOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
 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;
@@ -354,8 +354,8 @@ public class FSEditLogLoader {
       // update the block list.
       
       // Update the salient file attributes.
-      newFile.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
-      newFile.setModificationTime(addCloseOp.mtime, null, fsDir.getINodeMap());
+      newFile.setAccessTime(addCloseOp.atime, null);
+      newFile.setModificationTime(addCloseOp.mtime, null);
       updateBlocks(fsDir, addCloseOp, newFile);
       break;
     }
@@ -373,8 +373,8 @@ public class FSEditLogLoader {
       final INodeFile file = INodeFile.valueOf(iip.getINode(0), addCloseOp.path);
 
       // Update the salient file attributes.
-      file.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
-      file.setModificationTime(addCloseOp.mtime, null, fsDir.getINodeMap());
+      file.setAccessTime(addCloseOp.atime, null);
+      file.setModificationTime(addCloseOp.mtime, null);
       updateBlocks(fsDir, addCloseOp, file);
 
       // Now close the file

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -52,9 +52,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 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.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -731,9 +731,10 @@ public class FSImageFormat {
       if (nsQuota >= 0 || dsQuota >= 0) {
         dir.addDirectoryWithQuotaFeature(nsQuota, dsQuota);
       }
-      return snapshottable ? new INodeDirectorySnapshottable(dir)
-          : withSnapshot ? new INodeDirectoryWithSnapshot(dir)
-          : dir;
+      if (withSnapshot) {
+        dir.addSnapshotFeature(null);
+      }
+      return snapshottable ? new INodeDirectorySnapshottable(dir) : dir;
     } else if (numBlocks == -2) {
       //symlink
 
@@ -1113,10 +1114,10 @@ public class FSImageFormat {
       final ReadOnlyList<INode> children = current.getChildrenList(null);
       int dirNum = 0;
       List<INodeDirectory> snapshotDirs = null;
-      if (current instanceof INodeDirectoryWithSnapshot) {
+      DirectoryWithSnapshotFeature sf = current.getDirectoryWithSnapshotFeature();
+      if (sf != null) {
         snapshotDirs = new ArrayList<INodeDirectory>();
-        ((INodeDirectoryWithSnapshot) current).getSnapshotDirectory(
-            snapshotDirs);
+        sf.getSnapshotDirectory(snapshotDirs);
         dirNum += snapshotDirs.size();
       }
       

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

@@ -36,7 +36,6 @@ 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.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 import org.apache.hadoop.io.IntWritable;
@@ -239,7 +238,7 @@ public class FSImageSerialization {
       out.writeBoolean(true);
     } else {
       out.writeBoolean(false);
-      out.writeBoolean(node instanceof INodeDirectoryWithSnapshot);
+      out.writeBoolean(node.isWithSnapshot());
     }
     
     writePermissionStatus(node, out);

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2295,7 +2295,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       boolean writeToEditLog, Snapshot latestSnapshot, boolean logRetryCache)
       throws IOException {
-    file = file.recordModification(latestSnapshot, dir.getINodeMap());
+    file = file.recordModification(latestSnapshot);
     final INodeFile cons = file.toUnderConstruction(leaseHolder, clientMachine,
         clientNode);
 
@@ -3783,8 +3783,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     Preconditions.checkArgument(uc != null);
     leaseManager.removeLease(uc.getClientName(), src);
     
-    pendingFile = pendingFile.recordModification(latestSnapshot,
-        dir.getINodeMap());
+    pendingFile = pendingFile.recordModification(latestSnapshot);
 
     // The file is no longer pending.
     // Create permanent INode, update blocks. No need to replace the inode here

+ 21 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.Diff;
@@ -96,9 +95,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setUser(String user);
 
   /** Set user */
-  final INode setUser(String user, Snapshot latest, INodeMap inodeMap)
+  final INode setUser(String user, Snapshot latest)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setUser(user);
     return nodeToUpdate;
   }
@@ -120,9 +119,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setGroup(String group);
 
   /** Set group */
-  final INode setGroup(String group, Snapshot latest, INodeMap inodeMap)
+  final INode setGroup(String group, Snapshot latest)
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setGroup(group);
     return nodeToUpdate;
   }
@@ -145,9 +144,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   abstract void setPermission(FsPermission permission);
 
   /** Set the {@link FsPermission} of this {@link INode} */
-  INode setPermission(FsPermission permission, Snapshot latest,
-      INodeMap inodeMap) throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+  INode setPermission(FsPermission permission, Snapshot latest) 
+      throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setPermission(permission);
     return nodeToUpdate;
   }
@@ -231,14 +230,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *
    * @param latest the latest snapshot that has been taken.
    *        Note that it is null if no snapshots have been taken.
-   * @param inodeMap while recording modification, the inode or its parent may 
-   *                 get replaced, and the inodeMap needs to be updated.
    * @return The current inode, which usually is the same object of this inode.
    *         However, in some cases, this inode may be replaced with a new inode
    *         for maintaining snapshots. The current inode is then the new inode.
    */
-  abstract INode recordModification(final Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException;
+  abstract INode recordModification(final Snapshot latest)
+      throws QuotaExceededException;
 
   /** Check whether it's a reference. */
   public boolean isReference() {
@@ -318,7 +315,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * Call recordModification(..) to capture the current states.
    * Mark the INode as deleted.
    * 
-   * 1.4 The current inode is a {@link INodeDirectoryWithSnapshot}.
+   * 1.4 The current inode is an {@link INodeDirectory} with snapshot feature.
    * Call recordModification(..) to capture the current states. 
    * Destroy files/directories created after the latest snapshot 
    * (i.e., the inodes stored in the created list of the latest snapshot).
@@ -329,7 +326,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * 2.2 To clean {@link INodeDirectory}: recursively clean its children.
    * 2.3 To clean INodeFile with snapshot: delete the corresponding snapshot in
    * its diff list.
-   * 2.4 To clean {@link INodeDirectoryWithSnapshot}: delete the corresponding 
+   * 2.4 To clean {@link INodeDirectory} with snapshot: delete the corresponding 
    * snapshot in its diff list. Recursively clean its children.
    * </pre>
    * 
@@ -575,16 +572,16 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   }
 
   /** Update modification time if it is larger than the current value. */
-  public abstract INode updateModificationTime(long mtime, Snapshot latest,
-      INodeMap inodeMap) throws QuotaExceededException;
+  public abstract INode updateModificationTime(long mtime, Snapshot latest) 
+      throws QuotaExceededException;
 
   /** Set the last modification time of inode. */
   public abstract void setModificationTime(long modificationTime);
 
   /** Set the last modification time of inode. */
   public final INode setModificationTime(long modificationTime,
-      Snapshot latest, INodeMap inodeMap) throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+      Snapshot latest) throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setModificationTime(modificationTime);
     return nodeToUpdate;
   }
@@ -611,9 +608,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   /**
    * Set last access time of inode.
    */
-  public final INode setAccessTime(long accessTime, Snapshot latest,
-      INodeMap inodeMap) throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest, inodeMap);
+  public final INode setAccessTime(long accessTime, Snapshot latest)
+      throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest);
     nodeToUpdate.setAccessTime(accessTime);
     return nodeToUpdate;
   }
@@ -753,8 +750,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     }
   }
 
-  /** INode feature such as {@link FileUnderConstructionFeature}
-   *  and {@link DirectoryWithQuotaFeature}.
+  /** 
+   * INode feature such as {@link FileUnderConstructionFeature}
+   * and {@link DirectoryWithQuotaFeature}.
    */
   public interface Feature {
   }

+ 263 - 99
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -32,9 +32,11 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 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.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -168,40 +170,49 @@ public class INodeDirectory extends INodeWithAdditionalFields
   private int searchChildren(byte[] name) {
     return children == null? -1: Collections.binarySearch(children, name);
   }
-
+  
+  protected DirectoryWithSnapshotFeature addSnapshotFeature(
+      DirectoryDiffList diffs) {
+    Preconditions.checkState(!isWithSnapshot(), 
+        "Directory is already with snapshot");
+    DirectoryWithSnapshotFeature sf = new DirectoryWithSnapshotFeature(diffs);
+    addFeature(sf);
+    return sf;
+  }
+  
   /**
-   * Remove the specified child from this directory.
-   * 
-   * @param child the child inode to be removed
-   * @param latest See {@link INode#recordModification(Snapshot, INodeMap)}.
+   * If feature list contains a {@link DirectoryWithSnapshotFeature}, return it;
+   * otherwise, return null.
    */
-  public boolean removeChild(INode child, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest)) {
-      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
-          .removeChild(child, latest, inodeMap);
+  public final DirectoryWithSnapshotFeature getDirectoryWithSnapshotFeature() {
+    for (Feature f : features) {
+      if (f instanceof DirectoryWithSnapshotFeature) {
+        return (DirectoryWithSnapshotFeature) f;
+      }
     }
-
-    return removeChild(child);
+    return null;
   }
 
-  /** 
-   * Remove the specified child from this directory.
-   * The basic remove method which actually calls children.remove(..).
-   *
-   * @param child the child inode to be removed
-   * 
-   * @return true if the child is removed; false if the child is not found.
-   */
-  protected final boolean removeChild(final INode child) {
-    final int i = searchChildren(child.getLocalNameBytes());
-    if (i < 0) {
-      return false;
-    }
-
-    final INode removed = children.remove(i);
-    Preconditions.checkState(removed == child);
-    return true;
+  /** Is this file has the snapshot feature? */
+  public final boolean isWithSnapshot() {
+    return getDirectoryWithSnapshotFeature() != null;
+  }
+  
+  public DirectoryDiffList getDiffs() {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    return sf != null ? sf.getDiffs() : null;
+  }
+  
+  @Override
+  public INodeDirectoryAttributes getSnapshotINode(Snapshot snapshot) {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    return sf == null ? this : sf.getDiffs().getSnapshotINode(snapshot, this);
+  }
+  
+  @Override
+  public String toDetailString() {
+    DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
+    return super.toDetailString() + (sf == null ? "" : ", " + sf.getDiffs()); 
   }
 
   /** Replace itself with an {@link INodeDirectorySnapshottable}. */
@@ -210,16 +221,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
     Preconditions.checkState(!(this instanceof INodeDirectorySnapshottable),
         "this is already an INodeDirectorySnapshottable, this=%s", this);
     final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(this);
-    replaceSelf(s, inodeMap).saveSelf2Snapshot(latest, this);
+    replaceSelf(s, inodeMap).getDirectoryWithSnapshotFeature().getDiffs()
+        .saveSelf2Snapshot(latest, s, this);
     return s;
   }
 
-  /** Replace itself with an {@link INodeDirectoryWithSnapshot}. */
-  public INodeDirectoryWithSnapshot replaceSelf4INodeDirectoryWithSnapshot(
-      final INodeMap inodeMap) {
-    return replaceSelf(new INodeDirectoryWithSnapshot(this), inodeMap);
-  }
-
   /** Replace itself with {@link INodeDirectory}. */
   public INodeDirectory replaceSelf4INodeDirectory(final INodeMap inodeMap) {
     Preconditions.checkState(getClass() != INodeDirectory.class,
@@ -245,7 +251,13 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return newDir;
   }
   
-  /** Replace the given child with a new child. */
+  /** 
+   * Replace the given child with a new child. Note that we no longer need to
+   * replace an normal INodeDirectory or INodeFile into an
+   * INodeDirectoryWithSnapshot or INodeFileUnderConstruction. The only cases
+   * for child replacement is for {@link INodeDirectorySnapshottable} and 
+   * reference nodes.
+   */
   public void replaceChild(INode oldChild, final INode newChild,
       final INodeMap inodeMap) {
     Preconditions.checkNotNull(children);
@@ -256,24 +268,24 @@ public class INodeDirectory extends INodeWithAdditionalFields
             .asReference().getReferredINode());
     oldChild = children.get(i);
     
-    if (oldChild.isReference() && !newChild.isReference()) {
-      // replace the referred inode, e.g., 
-      // INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
-      final INode withCount = oldChild.asReference().getReferredINode();
-      withCount.asReference().setReferredINode(newChild);
-    } else {
-      if (oldChild.isReference()) {
-        // both are reference nodes, e.g., DstReference -> WithName
-        final INodeReference.WithCount withCount = 
-            (WithCount) oldChild.asReference().getReferredINode();
-        withCount.removeReference(oldChild.asReference());
-      }
-      children.set(i, newChild);
+    if (oldChild.isReference() && newChild.isReference()) {
+      // both are reference nodes, e.g., DstReference -> WithName
+      final INodeReference.WithCount withCount = 
+          (WithCount) oldChild.asReference().getReferredINode();
+      withCount.removeReference(oldChild.asReference());
     }
+    children.set(i, newChild);
+    
+    // replace the instance in the created list of the diff list
+    DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
+    }
+    
     // update the inodeMap
     if (inodeMap != null) {
       inodeMap.put(newChild);
-    }
+    }    
   }
 
   INodeReference.WithName replaceChild4ReferenceWithName(INode oldChild,
@@ -298,14 +310,18 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public INodeDirectory recordModification(Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest)) {
-      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
-          .recordModification(latest, inodeMap);
-    } else {
-      return this;
+  public INodeDirectory recordModification(Snapshot latest) 
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
+      // add snapshot feature if necessary
+      DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+      if (sf == null) {
+        sf = addSnapshotFeature(null);
+      }
+      // record self in the diff list if necessary
+      sf.getDiffs().saveSelf2Snapshot(latest, this, null);
     }
+    return this;
   }
 
   /**
@@ -314,13 +330,17 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * @return the child inode, which may be replaced.
    */
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
-      final INode snapshotCopy, final INodeMap inodeMap)
-      throws QuotaExceededException {
+      final INode snapshotCopy) throws QuotaExceededException {
     if (latest == null) {
       return child;
     }
-    return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
-        .saveChild2Snapshot(child, latest, snapshotCopy, inodeMap);
+    
+    // add snapshot feature if necessary
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf == null) {
+      sf = this.addSnapshotFeature(null);
+    }
+    return sf.saveChild2Snapshot(this, child, latest, snapshotCopy);
   }
 
   /**
@@ -331,9 +351,36 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * @return the child inode.
    */
   public INode getChild(byte[] name, Snapshot snapshot) {
-    final ReadOnlyList<INode> c = getChildrenList(snapshot);
-    final int i = ReadOnlyList.Util.binarySearch(c, name);
-    return i < 0? null: c.get(i);
+    DirectoryWithSnapshotFeature sf;
+    if (snapshot == null || (sf = getDirectoryWithSnapshotFeature()) == null) {
+      ReadOnlyList<INode> c = getCurrentChildrenList();
+      final int i = ReadOnlyList.Util.binarySearch(c, name);
+      return i < 0 ? null : c.get(i);
+    }
+    
+    return sf.getChild(this, name, snapshot);
+  }
+  
+  /**
+   * @param snapshot
+   *          if it is not null, get the result from the given snapshot;
+   *          otherwise, get the result from the current directory.
+   * @return the current children list if the specified snapshot is null;
+   *         otherwise, return the children list corresponding to the snapshot.
+   *         Note that the returned list is never null.
+   */
+  public ReadOnlyList<INode> getChildrenList(final Snapshot snapshot) {
+    DirectoryWithSnapshotFeature sf;
+    if (snapshot == null
+        || (sf = this.getDirectoryWithSnapshotFeature()) == null) {
+      return getCurrentChildrenList();
+    }
+    return sf.getChildrenList(this, snapshot);
+  }
+  
+  private ReadOnlyList<INode> getCurrentChildrenList() {
+    return children == null ? ReadOnlyList.Util.<INode> emptyList()
+        : ReadOnlyList.Util.asReadOnlyList(children);
   }
 
   /** @return the {@link INodesInPath} containing only the last inode. */
@@ -399,6 +446,41 @@ public class INodeDirectory extends INodeWithAdditionalFields
     }
     return -nextPos;
   }
+  
+  /**
+   * Remove the specified child from this directory.
+   */
+  public boolean removeChild(INode child, Snapshot latest)
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      // create snapshot feature if necessary
+      DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
+      if (sf == null) {
+        sf = this.addSnapshotFeature(null);
+      }
+      return sf.removeChild(this, child, latest);
+    }
+    return removeChild(child);
+  }
+  
+  /** 
+   * Remove the specified child from this directory.
+   * The basic remove method which actually calls children.remove(..).
+   *
+   * @param child the child inode to be removed
+   * 
+   * @return true if the child is removed; false if the child is not found.
+   */
+  public boolean removeChild(final INode child) {
+    final int i = searchChildren(child.getLocalNameBytes());
+    if (i < 0) {
+      return false;
+    }
+
+    final INode removed = children.remove(i);
+    Preconditions.checkState(removed == child);
+    return true;
+  }
 
   /**
    * Add a child inode to the directory.
@@ -407,34 +489,32 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * @param setModTime set modification time for the parent node
    *                   not needed when replaying the addition and 
    *                   the parent already has the proper mod time
-   * @param inodeMap update the inodeMap if the directory node gets replaced
    * @return false if the child with this name already exists; 
    *         otherwise, return true;
    */
   public boolean addChild(INode node, final boolean setModTime,
-      final Snapshot latest, final INodeMap inodeMap)
-      throws QuotaExceededException {
+      final Snapshot latest) throws QuotaExceededException {
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
       return false;
     }
 
     if (isInLatestSnapshot(latest)) {
-      INodeDirectoryWithSnapshot sdir = 
-          replaceSelf4INodeDirectoryWithSnapshot(inodeMap);
-      boolean added = sdir.addChild(node, setModTime, latest, inodeMap);
-      return added;
+      // create snapshot feature if necessary
+      DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
+      if (sf == null) {
+        sf = this.addSnapshotFeature(null);
+      }
+      return sf.addChild(this, node, setModTime, latest);
     }
     addChild(node, low);
     if (setModTime) {
       // update modification time of the parent directory
-      updateModificationTime(node.getModificationTime(), latest, inodeMap);
+      updateModificationTime(node.getModificationTime(), latest);
     }
     return true;
   }
 
-
-  /** The same as addChild(node, false, null, false) */
   public boolean addChild(INode node) {
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
@@ -463,21 +543,34 @@ public class INodeDirectory extends INodeWithAdditionalFields
   @Override
   public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
       int lastSnapshotId) {
-    final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
-    if (q != null) {
-      if (useCache && isQuotaSet()) {
-        q.addNamespaceDiskspace(counts);
-      } else {
-        computeDirectoryQuotaUsage(counts, false, lastSnapshotId);
+    final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    
+    // we are computing the quota usage for a specific snapshot here, i.e., the
+    // computation only includes files/directories that exist at the time of the
+    // given snapshot
+    if (sf != null && lastSnapshotId != Snapshot.INVALID_ID
+        && !(useCache && isQuotaSet())) {
+      Snapshot lastSnapshot = sf.getDiffs().getSnapshotById(lastSnapshotId);
+      ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshot);
+      for (INode child : childrenList) {
+        child.computeQuotaUsage(counts, useCache, lastSnapshotId);
       }
+      counts.add(Quota.NAMESPACE, 1);
       return counts;
+    }
+    
+    // compute the quota usage in the scope of the current directory tree
+    final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
+    if (useCache && q != null && q.isQuotaSet()) { // use the cached quota
+      return q.addNamespaceDiskspace(counts);
     } else {
+      useCache = q != null && !q.isQuotaSet() ? false : useCache;
       return computeDirectoryQuotaUsage(counts, useCache, lastSnapshotId);
     }
   }
 
-  Quota.Counts computeDirectoryQuotaUsage(Quota.Counts counts, boolean useCache,
-      int lastSnapshotId) {
+  private Quota.Counts computeDirectoryQuotaUsage(Quota.Counts counts,
+      boolean useCache, int lastSnapshotId) {
     if (children != null) {
       for (INode child : children) {
         child.computeQuotaUsage(counts, useCache, lastSnapshotId);
@@ -489,12 +582,21 @@ public class INodeDirectory extends INodeWithAdditionalFields
   /** Add quota usage for this inode excluding children. */
   public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
     counts.add(Quota.NAMESPACE, 1);
+    // include the diff list
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.computeQuotaUsage4CurrentDirectory(counts);
+    }
     return counts;
   }
 
   @Override
   public ContentSummaryComputationContext computeContentSummary(
       ContentSummaryComputationContext summary) {
+    final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.computeContentSummary4Snapshot(summary.getCounts());
+    }
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     if (q != null) {
       return q.computeContentSummary(this, summary);
@@ -521,13 +623,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
       if (lastYieldCount == summary.getYieldCount()) {
         continue;
       }
-
       // The locks were released and reacquired. Check parent first.
       if (getParent() == null) {
         // Stop further counting and return whatever we have so far.
         break;
       }
-
       // Obtain the children list again since it may have been modified.
       childrenList = getChildrenList(null);
       // Reposition in case the children list is changed. Decrement by 1
@@ -537,24 +637,77 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
     // Increment the directory count for this directory.
     summary.getCounts().add(Content.DIRECTORY, 1);
-
     // Relinquish and reacquire locks if necessary.
     summary.yield();
-
     return summary;
   }
-
+  
   /**
-   * @param snapshot
-   *          if it is not null, get the result from the given snapshot;
-   *          otherwise, get the result from the current directory.
-   * @return the current children list if the specified snapshot is null;
-   *         otherwise, return the children list corresponding to the snapshot.
-   *         Note that the returned list is never null.
+   * This method is usually called by the undo section of rename.
+   * 
+   * Before calling this function, in the rename operation, we replace the
+   * original src node (of the rename operation) with a reference node (WithName
+   * instance) in both the children list and a created list, delete the
+   * reference node from the children list, and add it to the corresponding
+   * deleted list.
+   * 
+   * To undo the above operations, we have the following steps in particular:
+   * 
+   * <pre>
+   * 1) remove the WithName node from the deleted list (if it exists) 
+   * 2) replace the WithName node in the created list with srcChild 
+   * 3) add srcChild back as a child of srcParent. Note that we already add 
+   * the node into the created list of a snapshot diff in step 2, we do not need
+   * to add srcChild to the created list of the latest snapshot.
+   * </pre>
+   * 
+   * We do not need to update quota usage because the old child is in the 
+   * deleted list before. 
+   * 
+   * @param oldChild
+   *          The reference node to be removed/replaced
+   * @param newChild
+   *          The node to be added back
+   * @param latestSnapshot
+   *          The latest snapshot. Note this may not be the last snapshot in the
+   *          diff list, since the src tree of the current rename operation
+   *          may be the dst tree of a previous rename.
+   * @throws QuotaExceededException should not throw this exception
    */
-  public ReadOnlyList<INode> getChildrenList(final Snapshot snapshot) {
-    return children == null ? ReadOnlyList.Util.<INode>emptyList()
-        : ReadOnlyList.Util.asReadOnlyList(children);
+  public void undoRename4ScrParent(final INodeReference oldChild,
+      final INode newChild, Snapshot latestSnapshot)
+      throws QuotaExceededException {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    Preconditions.checkState(sf != null,
+        "Directory does not have snapshot feature");
+    sf.getDiffs().removeChild(ListType.DELETED, oldChild);
+    sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
+    addChild(newChild, true, null);
+  }
+  
+  /**
+   * Undo the rename operation for the dst tree, i.e., if the rename operation
+   * (with OVERWRITE option) removes a file/dir from the dst tree, add it back
+   * and delete possible record in the deleted list.  
+   */
+  public void undoRename4DstParent(final INode deletedChild,
+      Snapshot latestSnapshot) throws QuotaExceededException {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    Preconditions.checkState(sf != null,
+        "Directory does not have snapshot feature");
+    boolean removeDeletedChild = sf.getDiffs().removeChild(ListType.DELETED,
+        deletedChild);
+    // pass null for inodeMap since the parent node will not get replaced when
+    // undoing rename
+    final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
+        : latestSnapshot);
+    // update quota usage if adding is successfully and the old child has not
+    // been stored in deleted list before
+    if (added && !removeDeletedChild) {
+      final Quota.Counts counts = deletedChild.computeQuotaUsage();
+      addSpaceConsumed(counts.get(Quota.NAMESPACE),
+          counts.get(Quota.DISKSPACE), false);
+    }
   }
 
   /** Set the children list to null. */
@@ -578,7 +731,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
     // the diff list, the snapshot to be deleted has been combined or renamed
     // to its latest previous snapshot. (besides, we also need to consider nodes
     // created after prior but before snapshot. this will be done in 
-    // INodeDirectoryWithSnapshot#cleanSubtree)
+    // DirectoryWithSnapshotFeature)
     Snapshot s = snapshot != null && prior != null ? prior : snapshot;
     for (INode child : getChildrenList(s)) {
       if (snapshot != null && excludedNodes != null
@@ -596,6 +749,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
   @Override
   public void destroyAndCollectBlocks(final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
+    final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    if (sf != null) {
+      sf.clear(this, collectedBlocks, removedINodes);
+    }
     for (INode child : getChildrenList(null)) {
       child.destroyAndCollectBlocks(collectedBlocks, removedINodes);
     }
@@ -608,6 +765,13 @@ public class INodeDirectory extends INodeWithAdditionalFields
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes, final boolean countDiffChange)
       throws QuotaExceededException {
+    DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
+    // there is snapshot data
+    if (sf != null) {
+      return sf.cleanDirectory(this, snapshot, prior, collectedBlocks,
+          removedINodes, countDiffChange);
+    }
+    // there is no snapshot data
     if (prior == null && snapshot == null) {
       // destroy the whole subtree and collect blocks that should be deleted
       Quota.Counts counts = Quota.Counts.newInstance();

+ 13 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -27,7 +27,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.server.blockmanagement.*;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@@ -246,6 +250,8 @@ public class INodeFile extends INodeWithAdditionalFields
   /* Start of Snapshot Feature */
 
   private FileWithSnapshotFeature addSnapshotFeature(FileDiffList diffs) {
+    Preconditions.checkState(!isWithSnapshot(), 
+        "File is already with snapshot");
     FileWithSnapshotFeature sf = new FileWithSnapshotFeature(diffs);
     this.addFeature(sf);
     return sf;
@@ -279,25 +285,23 @@ public class INodeFile extends INodeWithAdditionalFields
   public INodeFileAttributes getSnapshotINode(final Snapshot snapshot) {
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
     if (sf != null) {
-      return sf.getSnapshotINode(this, snapshot);
+      return sf.getDiffs().getSnapshotINode(snapshot, this);
     } else {
       return this;
     }
   }
 
   @Override
-  public INodeFile recordModification(final Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest)) {
+  public INodeFile recordModification(final Snapshot latest) 
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
       // the file is in snapshot, create a snapshot feature if it does not have
       FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
       if (sf == null) {
         sf = addSnapshotFeature(null);
       }
       // record self in the diff list if necessary
-      if (!shouldRecordInSrcSnapshot(latest)) {
-        sf.getDiffs().saveSelf2Snapshot(latest, this, null);
-      }
+      sf.getDiffs().saveSelf2Snapshot(latest, this, null);
     }
     return this;
   }
@@ -349,7 +353,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication, Snapshot latest,
       final INodeMap inodeMap) throws QuotaExceededException {
-    final INodeFile nodeToUpdate = recordModification(latest, inodeMap);
+    final INodeFile nodeToUpdate = recordModification(latest);
     nodeToUpdate.setFileReplication(replication);
     return nodeToUpdate;
   }

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

@@ -89,8 +89,7 @@ public class INodeMap {
         "", "", new FsPermission((short) 0)), 0, 0) {
       
       @Override
-      INode recordModification(Snapshot latest, INodeMap inodeMap)
-          throws QuotaExceededException {
+      INode recordModification(Snapshot latest) throws QuotaExceededException {
         return null;
       }
       

+ 33 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.base.Preconditions;
@@ -103,9 +103,12 @@ public abstract class INodeReference extends INode {
       INode referred = wc.getReferredINode();
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         return referred.asFile().getDiffs().getPrior(wn.lastSnapshotId);
-      } else if (referred instanceof INodeDirectoryWithSnapshot) { 
-        return ((INodeDirectoryWithSnapshot) referred).getDiffs().getPrior(
-            wn.lastSnapshotId);
+      } else if (referred.isDirectory()) {
+        DirectoryWithSnapshotFeature sf = referred.asDirectory()
+            .getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          return sf.getDiffs().getPrior(wn.lastSnapshotId);
+        }
       }
     }
     return null;
@@ -231,9 +234,9 @@ public abstract class INodeReference extends INode {
   }
   
   @Override
-  public final INode updateModificationTime(long mtime, Snapshot latest,
-      INodeMap inodeMap) throws QuotaExceededException {
-    return referred.updateModificationTime(mtime, latest, inodeMap);
+  public final INode updateModificationTime(long mtime, Snapshot latest) 
+      throws QuotaExceededException {
+    return referred.updateModificationTime(mtime, latest);
   }
   
   @Override
@@ -252,9 +255,9 @@ public abstract class INodeReference extends INode {
   }
 
   @Override
-  final INode recordModification(Snapshot latest, final INodeMap inodeMap)
+  final INode recordModification(Snapshot latest)
       throws QuotaExceededException {
-    referred.recordModification(latest, inodeMap);
+    referred.recordModification(latest);
     // reference is never replaced 
     return this;
   }
@@ -547,9 +550,12 @@ public abstract class INodeReference extends INode {
       Snapshot snapshot = null;
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         snapshot = referred.asFile().getDiffs().getPrior(lastSnapshotId);
-      } else if (referred instanceof INodeDirectoryWithSnapshot) {
-        snapshot = ((INodeDirectoryWithSnapshot) referred).getDiffs().getPrior(
-            lastSnapshotId);
+      } else if (referred.isDirectory()) {
+        DirectoryWithSnapshotFeature sf = referred.asDirectory()
+            .getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          snapshot = sf.getDiffs().getPrior(lastSnapshotId);
+        }
       }
       return snapshot;
     }
@@ -634,10 +640,11 @@ public abstract class INodeReference extends INode {
         Snapshot snapshot = getSelfSnapshot(prior);
         
         INode referred = getReferredINode().asReference().getReferredINode();
-        if (referred.isFile() && referred.asFile().isWithSnapshot()) {
-          // if referred is a file, it must be a file with Snapshot since we did
+        if (referred.isFile()) {
+          // if referred is a file, it must be a file with snapshot since we did
           // recordModification before the rename
           INodeFile file = referred.asFile();
+          Preconditions.checkState(file.isWithSnapshot());
           // make sure we mark the file as deleted
           file.getFileWithSnapshotFeature().deleteCurrentFile();
           try {
@@ -649,14 +656,14 @@ public abstract class INodeReference extends INode {
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", e);
           }
-        } else if (referred instanceof INodeDirectoryWithSnapshot) {
+        } else if (referred.isDirectory()) {
           // similarly, if referred is a directory, it must be an
-          // INodeDirectoryWithSnapshot
-          INodeDirectoryWithSnapshot sdir = 
-              (INodeDirectoryWithSnapshot) referred;
+          // INodeDirectory with snapshot
+          INodeDirectory dir = referred.asDirectory();
+          Preconditions.checkState(dir.isWithSnapshot());
           try {
-            INodeDirectoryWithSnapshot.destroyDstSubtree(sdir, snapshot, prior,
-                collectedBlocks, removedINodes);
+            DirectoryWithSnapshotFeature.destroyDstSubtree(dir, snapshot,
+                prior, collectedBlocks, removedINodes);
           } catch (QuotaExceededException e) {
             LOG.error("should not exceed quota while snapshot deletion", e);
           }
@@ -670,9 +677,12 @@ public abstract class INodeReference extends INode {
       Snapshot lastSnapshot = null;
       if (referred.isFile() && referred.asFile().isWithSnapshot()) {
         lastSnapshot = referred.asFile().getDiffs().getLastSnapshot();
-      } else if (referred instanceof INodeDirectoryWithSnapshot) {
-        lastSnapshot = ((INodeDirectoryWithSnapshot) referred)
-            .getLastSnapshot();
+      } else if (referred.isDirectory()) {
+        DirectoryWithSnapshotFeature sf = referred.asDirectory()
+            .getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          lastSnapshot = sf.getLastSnapshot();
+        }
       }
       if (lastSnapshot != null && !lastSnapshot.equals(prior)) {
         return lastSnapshot;

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java

@@ -45,11 +45,10 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
 
   @Override
-  INode recordModification(Snapshot latest, final INodeMap inodeMap)
-      throws QuotaExceededException {
+  INode recordModification(Snapshot latest) throws QuotaExceededException {
     if (isInLatestSnapshot(latest)) {
       INodeDirectory parent = getParent();
-      parent.saveChild2Snapshot(this, latest, new INodeSymlink(this), inodeMap);
+      parent.saveChild2Snapshot(this, latest, new INodeSymlink(this));
     }
     return this;
   }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java

@@ -231,13 +231,13 @@ public abstract class INodeWithAdditionalFields extends INode
 
   /** Update modification time if it is larger than the current value. */
   @Override
-  public final INode updateModificationTime(long mtime, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
+  public final INode updateModificationTime(long mtime, Snapshot latest) 
+      throws QuotaExceededException {
     Preconditions.checkState(isDirectory());
     if (mtime <= modificationTime) {
       return this;
     }
-    return setModificationTime(mtime, latest, inodeMap);
+    return setModificationTime(mtime, latest);
   }
 
   final void cloneModificationTime(INodeWithAdditionalFields that) {

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java

@@ -26,8 +26,8 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 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 com.google.common.base.Preconditions;
@@ -132,11 +132,11 @@ public class INodesInPath {
       final boolean isRef = curNode.isReference();
       final boolean isDir = curNode.isDirectory();
       final INodeDirectory dir = isDir? curNode.asDirectory(): null;  
-      if (!isRef && isDir && dir instanceof INodeDirectoryWithSnapshot) {
+      if (!isRef && isDir && dir.isWithSnapshot()) {
         //if the path is a non-snapshot path, update the latest snapshot.
         if (!existing.isSnapshot()) {
-          existing.updateLatestSnapshot(
-              ((INodeDirectoryWithSnapshot)dir).getLastSnapshot());
+          existing.updateLatestSnapshot(dir.getDirectoryWithSnapshotFeature()
+              .getLastSnapshot());
         }
       } else if (isRef && isDir && !lastComp) {
         // If the curNode is a reference node, need to check its dstSnapshot:
@@ -155,10 +155,10 @@ public class INodesInPath {
           if (latest == null ||  // no snapshot in dst tree of rename
               dstSnapshotId >= latest.getId()) { // the above scenario 
             Snapshot lastSnapshot = null;
-            if (curNode.isDirectory()
-                && curNode.asDirectory() instanceof INodeDirectoryWithSnapshot) {
-              lastSnapshot = ((INodeDirectoryWithSnapshot) curNode
-                  .asDirectory()).getLastSnapshot();
+            DirectoryWithSnapshotFeature sf = null;
+            if (curNode.isDirectory() && 
+                (sf = curNode.asDirectory().getDirectoryWithSnapshotFeature()) != null) {
+              lastSnapshot = sf.getLastSnapshot();
             }
             existing.setSnapshot(lastSnapshot);
           }

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

@@ -98,7 +98,7 @@ abstract class AbstractINodeDiff<N extends INode,
   }
 
   /** Save the INode state to the snapshot if it is not done already. */
-  void saveSnapshotCopy(A snapshotCopy, N currentINode) {
+  void saveSnapshotCopy(A snapshotCopy) {
     Preconditions.checkState(snapshotINode == null, "Expected snapshotINode to be null");
     snapshotINode = snapshotCopy;
   }

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

@@ -25,8 +25,8 @@ import java.util.List;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
@@ -271,7 +271,7 @@ abstract class AbstractINodeDiffList<N extends INode,
    *         Note that the current inode is returned if there is no change
    *         between the given snapshot and the current state. 
    */
-  A getSnapshotINode(final Snapshot snapshot, final A currentINode) {
+  public A getSnapshotINode(final Snapshot snapshot, final A currentINode) {
     final D diff = getDiff(snapshot);
     final A inode = diff == null? null: diff.getSnapshotINode();
     return inode == null? currentINode: inode;
@@ -306,7 +306,7 @@ abstract class AbstractINodeDiffList<N extends INode,
         if (snapshotCopy == null) {
           snapshotCopy = createSnapshotCopy(currentINode);
         }
-        diff.saveSnapshotCopy(snapshotCopy, currentINode);
+        diff.saveSnapshotCopy(snapshotCopy);
       }
     }
   }

+ 321 - 462
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java

@@ -28,6 +28,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
@@ -35,10 +36,10 @@ import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -51,18 +52,17 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import com.google.common.base.Preconditions;
 
 /**
- * The directory with snapshots. It maintains a list of snapshot diffs for
- * storing snapshot data. When there are modifications to the directory, the old
- * data is stored in the latest snapshot, if there is any.
+ * Feature for directory with snapshot-related information.
  */
-public class INodeDirectoryWithSnapshot extends INodeDirectory {
+@InterfaceAudience.Private
+public class DirectoryWithSnapshotFeature implements INode.Feature {
   /**
    * The difference between the current state and a previous snapshot
    * of the children list of an INodeDirectory.
    */
   static class ChildrenDiff extends Diff<byte[], INode> {
     ChildrenDiff() {}
-    
+
     private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
       super(created, deleted);
     }
@@ -73,7 +73,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
      */
     private final boolean replace(final ListType type,
         final INode oldChild, final INode newChild) {
-      final List<INode> list = getList(type); 
+      final List<INode> list = getList(type);
       final int i = search(list, oldChild.getLocalNameBytes());
       if (i < 0 || list.get(i).getId() != oldChild.getId()) {
         return false;
@@ -93,10 +93,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       }
       return false;
     }
-    
+
     /** clear the created list */
-    private Quota.Counts destroyCreatedList(
-        final INodeDirectoryWithSnapshot currentINode,
+    private Quota.Counts destroyCreatedList(final INodeDirectory currentINode,
         final BlocksMapUpdateInfo collectedBlocks,
         final List<INode> removedINodes) {
       Quota.Counts counts = Quota.Counts.newInstance();
@@ -110,7 +109,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       createdList.clear();
       return counts;
     }
-    
+
     /** clear the deleted list */
     private Quota.Counts destroyDeletedList(
         final BlocksMapUpdateInfo collectedBlocks,
@@ -124,19 +123,19 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       deletedList.clear();
       return counts;
     }
-    
+
     /** Serialize {@link #created} */
     private void writeCreated(DataOutput out) throws IOException {
       final List<INode> created = getList(ListType.CREATED);
       out.writeInt(created.size());
       for (INode node : created) {
-        // For INode in created list, we only need to record its local name 
+        // For INode in created list, we only need to record its local name
         byte[] name = node.getLocalNameBytes();
         out.writeShort(name.length);
         out.write(name);
       }
     }
-    
+
     /** Serialize {@link #deleted} */
     private void writeDeleted(DataOutput out,
         ReferenceMap referenceMap) throws IOException {
@@ -146,12 +145,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
       }
     }
-    
+
     /** Serialize to out */
     private void write(DataOutput out, ReferenceMap referenceMap
         ) throws IOException {
       writeCreated(out);
-      writeDeleted(out, referenceMap);    
+      writeDeleted(out, referenceMap);
     }
 
     /** Get the list of INodeDirectory contained in the deleted list */
@@ -162,17 +161,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
       }
     }
-    
+
     /**
      * Interpret the diff and generate a list of {@link DiffReportEntry}.
      * @param parentPath The relative path of the parent.
-     * @param parent The directory that the diff belongs to.
-     * @param fromEarlier True indicates {@code diff=later-earlier}, 
+     * @param fromEarlier True indicates {@code diff=later-earlier},
      *                    False indicates {@code diff=earlier-later}
      * @return A list of {@link DiffReportEntry} as the diff report.
      */
     public List<DiffReportEntry> generateReport(byte[][] parentPath,
-        INodeDirectoryWithSnapshot parent, boolean fromEarlier) {
+        boolean fromEarlier) {
       List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
       List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
       int c = 0, d = 0;
@@ -217,7 +215,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       return dList;
     }
   }
-  
+
   /**
    * The difference of an {@link INodeDirectory} between two snapshots.
    */
@@ -243,16 +241,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       this.childrenSize = childrenSize;
       this.diff = new ChildrenDiff(createdList, deletedList);
     }
-    
+
     ChildrenDiff getChildrenDiff() {
       return diff;
     }
-    
+
     /** Is the inode the root of the snapshot? */
     boolean isSnapshotRoot() {
       return snapshotINode == snapshot.getRoot();
     }
-    
+
     @Override
     Quota.Counts combinePosteriorAndCollectBlocks(
         final INodeDirectory currentDir, final DirectoryDiff posterior,
@@ -277,14 +275,15 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
      *         Since the snapshot is read-only, the logical view of the list is
      *         never changed although the internal data structure may mutate.
      */
-    ReadOnlyList<INode> getChildrenList(final INodeDirectory currentDir) {
+    private ReadOnlyList<INode> getChildrenList(final INodeDirectory currentDir) {
       return new ReadOnlyList<INode>() {
         private List<INode> children = null;
 
         private List<INode> initChildren() {
           if (children == null) {
             final ChildrenDiff combined = new ChildrenDiff();
-            for(DirectoryDiff d = DirectoryDiff.this; d != null; d = d.getPosterior()) {
+            for (DirectoryDiff d = DirectoryDiff.this; d != null; 
+                d = d.getPosterior()) {
               combined.combinePosterior(d.diff, null);
             }
             children = combined.apply2Current(ReadOnlyList.Util.asList(
@@ -297,17 +296,17 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         public Iterator<INode> iterator() {
           return initChildren().iterator();
         }
-    
+
         @Override
         public boolean isEmpty() {
           return childrenSize == 0;
         }
-    
+
         @Override
         public int size() {
           return childrenSize;
         }
-    
+
         @Override
         public INode get(int i) {
           return initChildren().get(i);
@@ -322,9 +321,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         final Container<INode> returned = d.diff.accessPrevious(name);
         if (returned != null) {
           // the diff is able to determine the inode
-          return returned.getElement(); 
+          return returned.getElement();
         } else if (!checkPosterior) {
-          // Since checkPosterior is false, return null, i.e. not found.   
+          // Since checkPosterior is false, return null, i.e. not found.
           return null;
         } else if (d.getPosterior() == null) {
           // no more posterior diff, get from current inode.
@@ -332,12 +331,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
       }
     }
-    
+
     @Override
     public String toString() {
       return super.toString() + " childrenSize=" + childrenSize + ", " + diff;
     }
-    
+
     @Override
     void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
       writeSnapshot(out);
@@ -386,7 +385,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
     }
 
     /** Replace the given child in the created/deleted list, if there is any. */
-    private boolean replaceChild(final ListType type, final INode oldChild,
+    public boolean replaceChild(final ListType type, final INode oldChild,
         final INode newChild) {
       final List<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
@@ -397,9 +396,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       }
       return false;
     }
-    
+
     /** Remove the given child in the created/deleted list, if there is any. */
-    private boolean removeChild(final ListType type, final INode child) {
+    public boolean removeChild(final ListType type, final INode child) {
       final List<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
         final ChildrenDiff diff = diffList.get(i).diff;
@@ -410,84 +409,134 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
       return false;
     }
   }
-
-  /**
-   * Compute the difference between Snapshots.
-   * 
-   * @param fromSnapshot Start point of the diff computation. Null indicates
-   *          current tree.
-   * @param toSnapshot End point of the diff computation. Null indicates current
-   *          tree.
-   * @param diff Used to capture the changes happening to the children. Note
-   *          that the diff still represents (later_snapshot - earlier_snapshot)
-   *          although toSnapshot can be before fromSnapshot.
-   * @return Whether changes happened between the startSnapshot and endSnaphsot.
-   */
-  boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
-      Snapshot toSnapshot, ChildrenDiff diff) {
-    Snapshot earlier = fromSnapshot;
-    Snapshot later = toSnapshot;
-    if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
-      earlier = toSnapshot;
-      later = fromSnapshot;
+  
+  private static Map<INode, INode> cloneDiffList(List<INode> diffList) {
+    if (diffList == null || diffList.size() == 0) {
+      return null;
     }
-    
-    boolean modified = diffs.changedBetweenSnapshots(earlier,
-        later);
-    if (!modified) {
-      return false;
+    Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
+    for (INode node : diffList) {
+      map.put(node, node);
     }
-    
-    final List<DirectoryDiff> difflist = diffs.asList();
-    final int size = difflist.size();
-    int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
-    int laterDiffIndex = later == null ? size : Collections
-        .binarySearch(difflist, later.getId());
-    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
-        : earlierDiffIndex;
-    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
-        : laterDiffIndex;
-    
-    boolean dirMetadataChanged = false;
-    INodeDirectoryAttributes dirCopy = null;
-    for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
-      DirectoryDiff sdiff = difflist.get(i);
-      diff.combinePosterior(sdiff.diff, null);
-      if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
-        if (dirCopy == null) {
-          dirCopy = sdiff.snapshotINode;
-        } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
-          dirMetadataChanged = true;
+    return map;
+  }
+  
+  /**
+   * Destroy a subtree under a DstReference node.
+   */
+  public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
+      final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) throws QuotaExceededException {
+    Preconditions.checkArgument(prior != null);
+    if (inode.isReference()) {
+      if (inode instanceof INodeReference.WithName && snapshot != null) {
+        // this inode has been renamed before the deletion of the DstReference
+        // subtree
+        inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes,
+            true);
+      } else { 
+        // for DstReference node, continue this process to its subtree
+        destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
+            prior, collectedBlocks, removedINodes);
+      }
+    } else if (inode.isFile()) {
+      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes, true);
+    } else if (inode.isDirectory()) {
+      Map<INode, INode> excludedNodes = null;
+      INodeDirectory dir = inode.asDirectory();
+      DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
+      if (sf != null) {
+        DirectoryDiffList diffList = sf.getDiffs();
+        DirectoryDiff priorDiff = diffList.getDiff(prior);
+        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+          List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
+          excludedNodes = cloneDiffList(dList);
+        }
+        
+        if (snapshot != null) {
+          diffList.deleteSnapshotDiff(snapshot, prior, dir, collectedBlocks,
+              removedINodes, true);
+        }
+        priorDiff = diffList.getDiff(prior);
+        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+          priorDiff.diff.destroyCreatedList(dir, collectedBlocks,
+              removedINodes);
         }
       }
+      for (INode child : inode.asDirectory().getChildrenList(prior)) {
+        if (excludedNodes != null && excludedNodes.containsKey(child)) {
+          continue;
+        }
+        destroyDstSubtree(child, snapshot, prior, collectedBlocks,
+            removedINodes);
+      }
     }
-
-    if (!diff.isEmpty() || dirMetadataChanged) {
-      return true;
-    } else if (dirCopy != null) {
-      for (int i = laterDiffIndex; i < size; i++) {
-        if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
-          return true;
+  }
+  
+  /**
+   * Clean an inode while we move it from the deleted list of post to the
+   * deleted list of prior.
+   * @param inode The inode to clean.
+   * @param post The post snapshot.
+   * @param prior The prior snapshot.
+   * @param collectedBlocks Used to collect blocks for later deletion.
+   * @return Quota usage update.
+   */
+  private static Quota.Counts cleanDeletedINode(INode inode,
+      final Snapshot post, final Snapshot prior,
+      final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes, final boolean countDiffChange) 
+      throws QuotaExceededException {
+    Quota.Counts counts = Quota.Counts.newInstance();
+    Deque<INode> queue = new ArrayDeque<INode>();
+    queue.addLast(inode);
+    while (!queue.isEmpty()) {
+      INode topNode = queue.pollFirst();
+      if (topNode instanceof INodeReference.WithName) {
+        INodeReference.WithName wn = (INodeReference.WithName) topNode;
+        if (wn.getLastSnapshotId() >= post.getId()) {
+          wn.cleanSubtree(post, prior, collectedBlocks, removedINodes,
+              countDiffChange);
+        }
+        // For DstReference node, since the node is not in the created list of
+        // prior, we should treat it as regular file/dir
+      } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
+        INodeFile file = topNode.asFile();
+        counts.add(file.getDiffs().deleteSnapshotDiff(post, prior, file,
+            collectedBlocks, removedINodes, countDiffChange));
+      } else if (topNode.isDirectory()) {
+        INodeDirectory dir = topNode.asDirectory();
+        ChildrenDiff priorChildrenDiff = null;
+        DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          // delete files/dirs created after prior. Note that these
+          // files/dirs, along with inode, were deleted right after post.
+          DirectoryDiff priorDiff = sf.getDiffs().getDiff(prior);
+          if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
+            priorChildrenDiff = priorDiff.getChildrenDiff();
+            counts.add(priorChildrenDiff.destroyCreatedList(dir,
+                collectedBlocks, removedINodes));
+          }
+        }
+        
+        for (INode child : dir.getChildrenList(prior)) {
+          if (priorChildrenDiff != null
+              && priorChildrenDiff.search(ListType.DELETED,
+                  child.getLocalNameBytes()) != null) {
+            continue;
+          }
+          queue.addLast(child);
         }
       }
-      return !dirCopy.metadataEquals(this);
-    } else {
-      return false;
     }
+    return counts;
   }
 
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
   private final DirectoryDiffList diffs;
 
-  public INodeDirectoryWithSnapshot(INodeDirectory that) {
-    this(that, true, that instanceof INodeDirectoryWithSnapshot?
-        ((INodeDirectoryWithSnapshot)that).getDiffs(): null);
-  }
-
-  INodeDirectoryWithSnapshot(INodeDirectory that, boolean adopt,
-      DirectoryDiffList diffs) {
-    super(that, adopt, true);
-    this.diffs = diffs != null? diffs: new DirectoryDiffList();
+  public DirectoryWithSnapshotFeature(DirectoryDiffList diffs) {
+    this.diffs = diffs != null ? diffs : new DirectoryDiffList();
   }
 
   /** @return the last snapshot. */
@@ -499,204 +548,203 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
   public DirectoryDiffList getDiffs() {
     return diffs;
   }
-
-  @Override
-  public INodeDirectoryAttributes getSnapshotINode(Snapshot snapshot) {
-    return diffs.getSnapshotINode(snapshot, this);
-  }
-
-  @Override
-  public INodeDirectoryWithSnapshot recordModification(final Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
-      return saveSelf2Snapshot(latest, null);
+  
+  /**
+   * Get all the directories that are stored in some snapshot but not in the
+   * current children list. These directories are equivalent to the directories
+   * stored in the deletes lists.
+   */
+  public void getSnapshotDirectory(List<INodeDirectory> snapshotDir) {
+    for (DirectoryDiff sdiff : diffs) {
+      sdiff.getChildrenDiff().getDirsInDeleted(snapshotDir);
     }
-    return this;
-  }
-
-  /** Save the snapshot copy to the latest snapshot. */
-  public INodeDirectoryWithSnapshot saveSelf2Snapshot(
-      final Snapshot latest, final INodeDirectory snapshotCopy)
-          throws QuotaExceededException {
-    diffs.saveSelf2Snapshot(latest, this, snapshotCopy);
-    return this;
   }
 
-  @Override
-  public INode saveChild2Snapshot(final INode child, final Snapshot latest,
-      final INode snapshotCopy, final INodeMap inodeMap)
-      throws QuotaExceededException {
-    Preconditions.checkArgument(!child.isDirectory(),
-        "child is a directory, child=%s", child);
-    if (latest == null) {
-      return child;
-    }
-
-    final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, this);
-    if (diff.getChild(child.getLocalNameBytes(), false, this) != null) {
-      // it was already saved in the latest snapshot earlier.  
-      return child;
-    }
-
-    diff.diff.modify(snapshotCopy, child);
-    return child;
-  }
+  /**
+   * Add an inode into parent's children list. The caller of this method needs
+   * to make sure that parent is in the given snapshot "latest".
+   */
+  public boolean addChild(INodeDirectory parent, INode inode,
+      boolean setModTime, Snapshot latest) throws QuotaExceededException {
+    ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, parent).diff;
+    int undoInfo = diff.create(inode);
 
-  @Override
-  public boolean addChild(INode inode, boolean setModTime, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    ChildrenDiff diff = null;
-    Integer undoInfo = null;
-    if (isInLatestSnapshot(latest)) {
-      diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
-      undoInfo = diff.create(inode);
-    }
-    final boolean added = super.addChild(inode, setModTime, null, inodeMap);
-    if (!added && undoInfo != null) {
+    final boolean added = parent.addChild(inode, setModTime, null);
+    if (!added) {
       diff.undoCreate(inode, undoInfo);
     }
-    return added; 
+    return added;
   }
 
-  @Override
-  public boolean removeChild(INode child, Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    ChildrenDiff diff = null;
-    UndoInfo<INode> undoInfo = null;
+  /**
+   * Remove an inode from parent's children list. The caller of this method
+   * needs to make sure that parent is in the given snapshot "latest".
+   */
+  public boolean removeChild(INodeDirectory parent, INode child,
+      Snapshot latest) throws QuotaExceededException {
     // For a directory that is not a renamed node, if isInLatestSnapshot returns
     // false, the directory is not in the latest snapshot, thus we do not need
     // to record the removed child in any snapshot.
     // For a directory that was moved/renamed, note that if the directory is in
-    // any of the previous snapshots, we will create a reference node for the 
+    // any of the previous snapshots, we will create a reference node for the
     // directory while rename, and isInLatestSnapshot will return true in that
     // scenario (if all previous snapshots have been deleted, isInLatestSnapshot
-    // still returns false). Thus if isInLatestSnapshot returns false, the 
-    // directory node cannot be in any snapshot (not in current tree, nor in 
-    // previous src tree). Thus we do not need to record the removed child in 
+    // still returns false). Thus if isInLatestSnapshot returns false, the
+    // directory node cannot be in any snapshot (not in current tree, nor in
+    // previous src tree). Thus we do not need to record the removed child in
     // any snapshot.
-    if (isInLatestSnapshot(latest)) {
-      diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
-      undoInfo = diff.delete(child);
-    }
-    final boolean removed = removeChild(child);
-    if (undoInfo != null) {
-      if (!removed) {
-        //remove failed, undo
-        diff.undoDelete(child, undoInfo);
-      }
+    ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, parent).diff;
+    UndoInfo<INode> undoInfo = diff.delete(child);
+
+    final boolean removed = parent.removeChild(child);
+    if (!removed && undoInfo != null) {
+      // remove failed, undo
+      diff.undoDelete(child, undoInfo);
     }
     return removed;
   }
   
-  @Override
-  public void replaceChild(final INode oldChild, final INode newChild,
-      final INodeMap inodeMap) {
-    super.replaceChild(oldChild, newChild, inodeMap);
-    if (oldChild.getParentReference() != null && !newChild.isReference()) {
-      // oldChild is referred by a Reference node. Thus we are replacing the 
-      // referred inode, e.g., 
-      // INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
-      // in this case, we do not need to update the diff list
-      return;
-    } else {
-      diffs.replaceChild(ListType.CREATED, oldChild, newChild);
-    }
-  }
-  
   /**
-   * This method is usually called by the undo section of rename.
-   * 
-   * Before calling this function, in the rename operation, we replace the
-   * original src node (of the rename operation) with a reference node (WithName
-   * instance) in both the children list and a created list, delete the
-   * reference node from the children list, and add it to the corresponding
-   * deleted list.
-   * 
-   * To undo the above operations, we have the following steps in particular:
-   * 
-   * <pre>
-   * 1) remove the WithName node from the deleted list (if it exists) 
-   * 2) replace the WithName node in the created list with srcChild 
-   * 3) add srcChild back as a child of srcParent. Note that we already add 
-   * the node into the created list of a snapshot diff in step 2, we do not need
-   * to add srcChild to the created list of the latest snapshot.
-   * </pre>
-   * 
-   * We do not need to update quota usage because the old child is in the 
-   * deleted list before. 
-   * 
-   * @param oldChild
-   *          The reference node to be removed/replaced
-   * @param newChild
-   *          The node to be added back
-   * @param latestSnapshot
-   *          The latest snapshot. Note this may not be the last snapshot in the
-   *          {@link #diffs}, since the src tree of the current rename operation
-   *          may be the dst tree of a previous rename.
-   * @throws QuotaExceededException should not throw this exception
+   * @return If there is no corresponding directory diff for the given
+   *         snapshot, this means that the current children list should be
+   *         returned for the snapshot. Otherwise we calculate the children list
+   *         for the snapshot and return it. 
    */
-  public void undoRename4ScrParent(final INodeReference oldChild,
-      final INode newChild, Snapshot latestSnapshot)
-      throws QuotaExceededException {
-    diffs.removeChild(ListType.DELETED, oldChild);
-    diffs.replaceChild(ListType.CREATED, oldChild, newChild);
-    // pass null for inodeMap since the parent node will not get replaced when
-    // undoing rename
-    addChild(newChild, true, null, null);
+  public ReadOnlyList<INode> getChildrenList(INodeDirectory currentINode,
+      final Snapshot snapshot) {
+    final DirectoryDiff diff = diffs.getDiff(snapshot);
+    return diff != null ? diff.getChildrenList(currentINode) : currentINode
+        .getChildrenList(null);
   }
   
-  /**
-   * Undo the rename operation for the dst tree, i.e., if the rename operation
-   * (with OVERWRITE option) removes a file/dir from the dst tree, add it back
-   * and delete possible record in the deleted list.  
-   */
-  public void undoRename4DstParent(final INode deletedChild,
-      Snapshot latestSnapshot) throws QuotaExceededException {
-    boolean removeDeletedChild = diffs.removeChild(ListType.DELETED,
-        deletedChild);
-    // pass null for inodeMap since the parent node will not get replaced when
-    // undoing rename
-    final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
-        : latestSnapshot, null);
-    // update quota usage if adding is successfully and the old child has not
-    // been stored in deleted list before
-    if (added && !removeDeletedChild) {
-      final Quota.Counts counts = deletedChild.computeQuotaUsage();
-      addSpaceConsumed(counts.get(Quota.NAMESPACE),
-          counts.get(Quota.DISKSPACE), false);
-    }
-  }
-
-  @Override
-  public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
+  public INode getChild(INodeDirectory currentINode, byte[] name,
+      Snapshot snapshot) {
     final DirectoryDiff diff = diffs.getDiff(snapshot);
-    return diff != null? diff.getChildrenList(this): super.getChildrenList(null);
+    return diff != null ? diff.getChild(name, true, currentINode)
+        : currentINode.getChild(name, null);
   }
+  
+  /** Used to record the modification of a symlink node */
+  public INode saveChild2Snapshot(INodeDirectory currentINode,
+      final INode child, final Snapshot latest, final INode snapshotCopy)
+      throws QuotaExceededException {
+    Preconditions.checkArgument(!child.isDirectory(),
+        "child is a directory, child=%s", child);
+    Preconditions.checkArgument(latest != null);
+    
+    final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest,
+        currentINode);
+    if (diff.getChild(child.getLocalNameBytes(), false, currentINode) != null) {
+      // it was already saved in the latest snapshot earlier.  
+      return child;
+    }
 
-  @Override
-  public INode getChild(byte[] name, Snapshot snapshot) {
-    final DirectoryDiff diff = diffs.getDiff(snapshot);
-    return diff != null? diff.getChild(name, true, this): super.getChild(name, null);
+    diff.diff.modify(snapshotCopy, child);
+    return child;
   }
-
-  @Override
-  public String toDetailString() {
-    return super.toDetailString() + ", " + diffs;
+  
+  public void clear(INodeDirectory currentINode,
+      final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
+    // destroy its diff list
+    for (DirectoryDiff diff : diffs) {
+      diff.destroyDiffAndCollectBlocks(currentINode, collectedBlocks,
+          removedINodes);
+    }
+    diffs.clear();
+  }
+  
+  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
+    for(DirectoryDiff d : diffs) {
+      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
+        deleted.computeQuotaUsage(counts, false, Snapshot.INVALID_ID);
+      }
+    }
+    counts.add(Quota.NAMESPACE, diffs.asList().size());
+    return counts;
+  }
+  
+  public void computeContentSummary4Snapshot(final Content.Counts counts) {
+    // Create a new blank summary context for blocking processing of subtree.
+    ContentSummaryComputationContext summary = 
+        new ContentSummaryComputationContext();
+    for(DirectoryDiff d : diffs) {
+      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
+        deleted.computeContentSummary(summary);
+      }
+    }
+    // Add the counts from deleted trees.
+    counts.add(summary.getCounts());
+    // Add the deleted directory count.
+    counts.add(Content.DIRECTORY, diffs.asList().size());
   }
   
   /**
-   * Get all the directories that are stored in some snapshot but not in the
-   * current children list. These directories are equivalent to the directories
-   * stored in the deletes lists.
+   * Compute the difference between Snapshots.
+   *
+   * @param fromSnapshot Start point of the diff computation. Null indicates
+   *          current tree.
+   * @param toSnapshot End point of the diff computation. Null indicates current
+   *          tree.
+   * @param diff Used to capture the changes happening to the children. Note
+   *          that the diff still represents (later_snapshot - earlier_snapshot)
+   *          although toSnapshot can be before fromSnapshot.
+   * @param currentINode The {@link INodeDirectory} this feature belongs to.
+   * @return Whether changes happened between the startSnapshot and endSnaphsot.
    */
-  public void getSnapshotDirectory(List<INodeDirectory> snapshotDir) {
-    for (DirectoryDiff sdiff : diffs) {
-      sdiff.getChildrenDiff().getDirsInDeleted(snapshotDir);
+  boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
+      Snapshot toSnapshot, ChildrenDiff diff, INodeDirectory currentINode) {
+    Snapshot earlier = fromSnapshot;
+    Snapshot later = toSnapshot;
+    if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
+      earlier = toSnapshot;
+      later = fromSnapshot;
+    }
+
+    boolean modified = diffs.changedBetweenSnapshots(earlier, later);
+    if (!modified) {
+      return false;
+    }
+
+    final List<DirectoryDiff> difflist = diffs.asList();
+    final int size = difflist.size();
+    int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
+    int laterDiffIndex = later == null ? size : Collections
+        .binarySearch(difflist, later.getId());
+    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
+        : earlierDiffIndex;
+    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
+        : laterDiffIndex;
+
+    boolean dirMetadataChanged = false;
+    INodeDirectoryAttributes dirCopy = null;
+    for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
+      DirectoryDiff sdiff = difflist.get(i);
+      diff.combinePosterior(sdiff.diff, null);
+      if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
+        if (dirCopy == null) {
+          dirCopy = sdiff.snapshotINode;
+        } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
+          dirMetadataChanged = true;
+        }
+      }
+    }
+
+    if (!diff.isEmpty() || dirMetadataChanged) {
+      return true;
+    } else if (dirCopy != null) {
+      for (int i = laterDiffIndex; i < size; i++) {
+        if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
+          return true;
+        }
+      }
+      return !dirCopy.metadataEquals(currentINode);
+    } else {
+      return false;
     }
   }
 
-  @Override
-  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
+  public Quota.Counts cleanDirectory(final INodeDirectory currentINode,
+      final Snapshot snapshot, Snapshot prior,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes, final boolean countDiffChange)
       throws QuotaExceededException {
@@ -704,12 +752,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorDeleted = null;
     if (snapshot == null) { // delete the current directory
-      recordModification(prior, null);
+      currentINode.recordModification(prior);
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
-        counts.add(lastDiff.diff.destroyCreatedList(this, collectedBlocks,
-            removedINodes));
+        counts.add(lastDiff.diff.destroyCreatedList(currentINode,
+            collectedBlocks, removedINodes));
       }
     } else {
       // update prior
@@ -726,7 +774,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
       }
       
-      counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, this, 
+      counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, currentINode, 
           collectedBlocks, removedINodes, countDiffChange));
       
       // check priorDiff again since it may be created during the diff deletion
@@ -767,202 +815,13 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
       }
     }
-    counts.add(cleanSubtreeRecursively(snapshot, prior, collectedBlocks,
-        removedINodes, priorDeleted, countDiffChange));
+    counts.add(currentINode.cleanSubtreeRecursively(snapshot, prior,
+        collectedBlocks, removedINodes, priorDeleted, countDiffChange));
     
-    if (isQuotaSet()) {
-      getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
+    if (currentINode.isQuotaSet()) {
+      currentINode.getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
     }
     return counts;
   }
-  
-  /**
-   * Clean an inode while we move it from the deleted list of post to the
-   * deleted list of prior.
-   * @param inode The inode to clean.
-   * @param post The post snapshot.
-   * @param prior The prior snapshot.
-   * @param collectedBlocks Used to collect blocks for later deletion.
-   * @return Quota usage update.
-   */
-  private static Quota.Counts cleanDeletedINode(INode inode,
-      final Snapshot post, final Snapshot prior,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange) 
-      throws QuotaExceededException {
-    Quota.Counts counts = Quota.Counts.newInstance();
-    Deque<INode> queue = new ArrayDeque<INode>();
-    queue.addLast(inode);
-    while (!queue.isEmpty()) {
-      INode topNode = queue.pollFirst();
-      if (topNode instanceof INodeReference.WithName) {
-        INodeReference.WithName wn = (INodeReference.WithName) topNode;
-        if (wn.getLastSnapshotId() >= post.getId()) {
-          wn.cleanSubtree(post, prior, collectedBlocks, removedINodes,
-              countDiffChange);
-        }
-        // For DstReference node, since the node is not in the created list of
-        // prior, we should treat it as regular file/dir
-      } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
-        INodeFile file = topNode.asFile();
-        counts.add(file.getDiffs().deleteSnapshotDiff(post, prior, file,
-            collectedBlocks, removedINodes, countDiffChange));
-      } else if (topNode.isDirectory()) {
-        INodeDirectory dir = topNode.asDirectory();
-        ChildrenDiff priorChildrenDiff = null;
-        if (dir instanceof INodeDirectoryWithSnapshot) {
-          // delete files/dirs created after prior. Note that these
-          // files/dirs, along with inode, were deleted right after post.
-          INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
-          DirectoryDiff priorDiff = sdir.getDiffs().getDiff(prior);
-          if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
-            priorChildrenDiff = priorDiff.getChildrenDiff();
-            counts.add(priorChildrenDiff.destroyCreatedList(sdir,
-                collectedBlocks, removedINodes));
-          }
-        }
-        
-        for (INode child : dir.getChildrenList(prior)) {
-          if (priorChildrenDiff != null
-              && priorChildrenDiff.search(ListType.DELETED,
-                  child.getLocalNameBytes()) != null) {
-            continue;
-          }
-          queue.addLast(child);
-        }
-      }
-    }
-    return counts;
-  }
-
-  @Override
-  public void destroyAndCollectBlocks(
-      final BlocksMapUpdateInfo collectedBlocks, 
-      final List<INode> removedINodes) {
-    // destroy its diff list
-    for (DirectoryDiff diff : diffs) {
-      diff.destroyDiffAndCollectBlocks(this, collectedBlocks, removedINodes);
-    }
-    diffs.clear();
-    super.destroyAndCollectBlocks(collectedBlocks, removedINodes);
-  }
-
-  @Override
-  public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
-      boolean useCache, int lastSnapshotId) {
-    if ((useCache && isQuotaSet()) || lastSnapshotId == Snapshot.INVALID_ID) {
-      return super.computeQuotaUsage(counts, useCache, lastSnapshotId);
-    }
-    
-    Snapshot lastSnapshot = diffs.getSnapshotById(lastSnapshotId);
-    
-    ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshot);
-    for (INode child : childrenList) {
-      child.computeQuotaUsage(counts, useCache, lastSnapshotId);
-    }
-    
-    counts.add(Quota.NAMESPACE, 1);
-    return counts;
-  }
-  
-  @Override
-  public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
-    super.computeQuotaUsage4CurrentDirectory(counts);
-    for(DirectoryDiff d : diffs) {
-      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        deleted.computeQuotaUsage(counts, false, Snapshot.INVALID_ID);
-      }
-    }
-    counts.add(Quota.NAMESPACE, diffs.asList().size());
-    return counts;
-  }
-
-  @Override
-  public ContentSummaryComputationContext computeContentSummary(
-      final ContentSummaryComputationContext summary) {
-    // Snapshot summary calc won't be relinquishing locks in the middle.
-    // Do this first and handover to parent.
-    computeContentSummary4Snapshot(summary.getCounts());
-    super.computeContentSummary(summary);
-    return summary;
-  }
-
-  private void computeContentSummary4Snapshot(final Content.Counts counts) {
-    // Create a new blank summary context for blocking processing of subtree.
-    ContentSummaryComputationContext summary = 
-        new ContentSummaryComputationContext();
-    for(DirectoryDiff d : diffs) {
-      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        deleted.computeContentSummary(summary);
-      }
-    }
-    // Add the counts from deleted trees.
-    counts.add(summary.getCounts());
-    // Add the deleted directory count.
-    counts.add(Content.DIRECTORY, diffs.asList().size());
-  }
-  
-  private static Map<INode, INode> cloneDiffList(List<INode> diffList) {
-    if (diffList == null || diffList.size() == 0) {
-      return null;
-    }
-    Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
-    for (INode node : diffList) {
-      map.put(node, node);
-    }
-    return map;
-  }
-  
-  /**
-   * Destroy a subtree under a DstReference node.
-   */
-  public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
-      final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes) throws QuotaExceededException {
-    Preconditions.checkArgument(prior != null);
-    if (inode.isReference()) {
-      if (inode instanceof INodeReference.WithName && snapshot != null) {
-        // this inode has been renamed before the deletion of the DstReference
-        // subtree
-        inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes,
-            true);
-      } else { 
-        // for DstReference node, continue this process to its subtree
-        destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
-            prior, collectedBlocks, removedINodes);
-      }
-    } else if (inode.isFile()) {
-      inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes, true);
-    } else if (inode.isDirectory()) {
-      Map<INode, INode> excludedNodes = null;
-      if (inode instanceof INodeDirectoryWithSnapshot) {
-        INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) inode;
-        
-        DirectoryDiffList diffList = sdir.getDiffs();
-        DirectoryDiff priorDiff = diffList.getDiff(prior);
-        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
-          List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
-          excludedNodes = cloneDiffList(dList);
-        }
-        
-        if (snapshot != null) {
-          diffList.deleteSnapshotDiff(snapshot, prior, sdir, collectedBlocks,
-              removedINodes, true);
-        }
-        priorDiff = diffList.getDiff(prior);
-        if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
-          priorDiff.diff.destroyCreatedList(sdir, collectedBlocks,
-              removedINodes);
-        }
-      }
-      for (INode child : inode.asDirectory().getChildrenList(prior)) {
-        if (excludedNodes != null && excludedNodes.containsKey(child)) {
-          continue;
-        }
-        destroyDstSubtree(child, snapshot, prior, collectedBlocks,
-            removedINodes);
-      }
-    }
-  }
 }

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

@@ -25,7 +25,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
@@ -57,10 +56,6 @@ public class FileWithSnapshotFeature implements INode.Feature {
     isCurrentFileDeleted = true;
   }
 
-  public INodeFileAttributes getSnapshotINode(INodeFile f, Snapshot snapshot) {
-    return diffs.getSnapshotINode(snapshot, f);
-  }
-
   public FileDiffList getDiffs() {
     return diffs;
   }
@@ -90,7 +85,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     if (snapshot == null) {
       // delete the current file while the file has snapshot feature
       if (!isCurrentFileDeleted()) {
-        file.recordModification(prior, null);
+        file.recordModification(prior);
         deleteCurrentFile();
       }
       collectBlocksAndClear(file, collectedBlocks, removedINodes);

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

@@ -44,6 +44,8 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
@@ -58,7 +60,7 @@ import com.google.common.primitives.SignedBytes;
  * by the namesystem and FSDirectory locks.
  */
 @InterfaceAudience.Private
-public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
+public class INodeDirectorySnapshottable extends INodeDirectory {
   /** Limit the number of snapshot per snapshottable directory. */
   static final int SNAPSHOT_LIMIT = 1 << 16;
 
@@ -115,8 +117,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
      * the two snapshots, while its associated value is a {@link ChildrenDiff}
      * storing the changes (creation/deletion) happened to the children (files).
      */
-    private final Map<INodeDirectoryWithSnapshot, ChildrenDiff> dirDiffMap = 
-        new HashMap<INodeDirectoryWithSnapshot, ChildrenDiff>();
+    private final Map<INodeDirectory, ChildrenDiff> dirDiffMap = 
+        new HashMap<INodeDirectory, ChildrenDiff>();
     
     SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
         Snapshot end) {
@@ -126,8 +128,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     }
     
     /** Add a dir-diff pair */
-    private void addDirDiff(INodeDirectoryWithSnapshot dir,
-        byte[][] relativePath, ChildrenDiff diff) {
+    private void addDirDiff(INodeDirectory dir, byte[][] relativePath,
+        ChildrenDiff diff) {
       dirDiffMap.put(dir, diff);
       diffMap.put(dir, relativePath);
     }
@@ -154,8 +156,7 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
         if (node.isDirectory()) {
           ChildrenDiff dirDiff = dirDiffMap.get(node);
           List<DiffReportEntry> subList = dirDiff.generateReport(
-              diffMap.get(node), (INodeDirectoryWithSnapshot) node,
-              isFromEarlier());
+              diffMap.get(node), isFromEarlier());
           diffReportList.addAll(subList);
         }
       }
@@ -183,8 +184,11 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   private int snapshotQuota = SNAPSHOT_LIMIT;
 
   public INodeDirectorySnapshottable(INodeDirectory dir) {
-    super(dir, true, dir instanceof INodeDirectoryWithSnapshot ? 
-        ((INodeDirectoryWithSnapshot) dir).getDiffs(): null);
+    super(dir, true, true);
+    // add snapshot feature if the original directory does not have it
+    if (!isWithSnapshot()) {
+      addSnapshotFeature(null);
+    }
   }
   
   /** @return the number of existing snapshots. */
@@ -298,8 +302,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     snapshotsByNames.add(-i - 1, s);
 
     //set modification time
-    updateModificationTime(Time.now(), null, null);
-    s.getRoot().setModificationTime(getModificationTime(), null, null);
+    updateModificationTime(Time.now(), null);
+    s.getRoot().setModificationTime(getModificationTime(), null);
     return s;
   }
   
@@ -413,12 +417,12 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
     if (node.isDirectory()) {
       INodeDirectory dir = node.asDirectory();
-      if (dir instanceof INodeDirectoryWithSnapshot) {
-        INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
-        boolean change = sdir.computeDiffBetweenSnapshots(
-            diffReport.from, diffReport.to, diff);
+      DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
+      if (sf != null) {
+        boolean change = sf.computeDiffBetweenSnapshots(diffReport.from,
+            diffReport.to, diff, dir);
         if (change) {
-          diffReport.addDirDiff(sdir, relativePath, diff);
+          diffReport.addDirDiff(dir, relativePath, diff);
         }
       }
       ReadOnlyList<INode> children = dir.getChildrenList(diffReport
@@ -453,13 +457,15 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   INodeDirectory replaceSelf(final Snapshot latest, final INodeMap inodeMap)
       throws QuotaExceededException {
     if (latest == null) {
-      Preconditions.checkState(getLastSnapshot() == null,
+      Preconditions.checkState(
+          getDirectoryWithSnapshotFeature().getLastSnapshot() == null,
           "latest == null but getLastSnapshot() != null, this=%s", this);
-      return replaceSelf4INodeDirectory(inodeMap);
-    } else {
-      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
-          .recordModification(latest, null);
     }
+    INodeDirectory dir = replaceSelf4INodeDirectory(inodeMap);
+    if (latest != null) {
+      dir.recordModification(latest);
+    }
+    return dir;
   }
 
   @Override

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

@@ -117,9 +117,8 @@ public class Snapshot implements Comparable<byte[]> {
     for(; inode != null; inode = inode.getParent()) {
       if (inode.isDirectory()) {
         final INodeDirectory dir = inode.asDirectory();
-        if (dir instanceof INodeDirectoryWithSnapshot) {
-          latest = ((INodeDirectoryWithSnapshot) dir).getDiffs().updatePrior(
-              anchor, latest);
+        if (dir.isWithSnapshot()) {
+          latest = dir.getDiffs().updatePrior(anchor, latest);
         }
       }
     }

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

@@ -36,8 +36,8 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -91,8 +91,7 @@ public class SnapshotFSImageFormat {
   public static void saveDirectoryDiffList(final INodeDirectory dir,
       final DataOutput out, final ReferenceMap referenceMap
       ) throws IOException {
-    saveINodeDiffs(dir instanceof INodeDirectoryWithSnapshot?
-        ((INodeDirectoryWithSnapshot)dir).getDiffs(): null, out, referenceMap);
+    saveINodeDiffs(dir.getDiffs(), out, referenceMap);
   }
   
   public static void saveFileDiffList(final INodeFile file,
@@ -139,7 +138,7 @@ public class SnapshotFSImageFormat {
    * @return The created node.
    */
   private static INode loadCreated(byte[] createdNodeName,
-      INodeDirectoryWithSnapshot parent) throws IOException {
+      INodeDirectory 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 (DirectoryDiff postDiff : parent.getDiffs()) {
@@ -165,7 +164,7 @@ public class SnapshotFSImageFormat {
    * @param in The {@link DataInput} to read.
    * @return The created list.
    */
-  private static List<INode> loadCreatedList(INodeDirectoryWithSnapshot parent,
+  private static List<INode> loadCreatedList(INodeDirectory parent,
       DataInput in) throws IOException {
     // read the size of the created list
     int createdSize = in.readInt();
@@ -188,7 +187,7 @@ public class SnapshotFSImageFormat {
    * @param loader The {@link Loader} instance.
    * @return The deleted list.
    */
-  private static List<INode> loadDeletedList(INodeDirectoryWithSnapshot parent,
+  private static List<INode> loadDeletedList(INodeDirectory parent,
       List<INode> createdList, DataInput in, FSImageFormat.Loader loader)
       throws IOException {
     int deletedSize = in.readInt();
@@ -239,11 +238,10 @@ public class SnapshotFSImageFormat {
   public static void loadDirectoryDiffList(INodeDirectory dir,
       DataInput in, FSImageFormat.Loader loader) throws IOException {
     final int size = in.readInt();
-    if (dir instanceof INodeDirectoryWithSnapshot) {
-      INodeDirectoryWithSnapshot withSnapshot = (INodeDirectoryWithSnapshot)dir;
-      DirectoryDiffList diffs = withSnapshot.getDiffs();
+    if (dir.isWithSnapshot()) {
+      DirectoryDiffList diffs = dir.getDiffs();
       for (int i = 0; i < size; i++) {
-        diffs.addFirst(loadDirectoryDiff(withSnapshot, in, loader));
+        diffs.addFirst(loadDirectoryDiff(dir, in, loader));
       }
     }
   }
@@ -277,9 +275,8 @@ public class SnapshotFSImageFormat {
    *               using.
    * @return A {@link DirectoryDiff}.
    */
-  private static DirectoryDiff loadDirectoryDiff(
-      INodeDirectoryWithSnapshot parent, DataInput in,
-      FSImageFormat.Loader loader) throws IOException {
+  private static DirectoryDiff loadDirectoryDiff(INodeDirectory parent,
+      DataInput in, FSImageFormat.Loader loader) throws IOException {
     // 1. Read the full path of the Snapshot root to identify the Snapshot
     final Snapshot snapshot = loader.getSnapshot(in);
 

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

@@ -41,8 +41,8 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.log4j.Level;

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

@@ -781,7 +781,7 @@ public class TestINodeFile {
       }
       System.out.println("Adding component " + DFSUtil.bytes2String(component));
       dir = new INodeDirectory(++id, component, permstatus, 0);
-      prev.addChild(dir, false, null, null);
+      prev.addChild(dir, false, null);
       prev = dir;
     }
     return dir; // Last Inode in the chain

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java

@@ -31,7 +31,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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.junit.AfterClass;
 import org.junit.Assert;
@@ -206,8 +205,7 @@ public class TestSnapshotPathINodes {
     // Check the INode for file1 (snapshot file)
     INode snapshotFileNode = inodes[inodes.length - 1]; 
     assertINodeFile(snapshotFileNode, file1);
-    assertTrue(snapshotFileNode.getParent() instanceof 
-        INodeDirectoryWithSnapshot);
+    assertTrue(snapshotFileNode.getParent().isWithSnapshot());
     
     // Call getExistingPathINodes and request only one INode.
     nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, 1, false);

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

@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Before;

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

@@ -358,7 +358,7 @@ public class TestNestedSnapshots {
     
     FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
     INode subNode = fsdir.getINode(sub.toString());
-    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
+    assertTrue(subNode.asDirectory().isWithSnapshot());
     
     hdfs.allowSnapshot(sub);
     subNode = fsdir.getINode(sub.toString());
@@ -366,6 +366,6 @@ public class TestNestedSnapshots {
     
     hdfs.disallowSnapshot(sub);
     subNode = fsdir.getINode(sub.toString());
-    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
+    assertTrue(subNode.asDirectory().isWithSnapshot());
   }
 }

+ 45 - 58
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -59,12 +59,11 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 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.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.ChildrenDiff;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -757,10 +756,10 @@ public class TestRenameWithSnapshots {
     // only 2 references: one in deleted list of sdir1, one in created list of
     // sdir1
     assertEquals(2, fooWithCount.getReferenceCount());
-    INodeDirectoryWithSnapshot foo = (INodeDirectoryWithSnapshot) fooWithCount
-        .asDirectory();
+    INodeDirectory foo = fooWithCount.asDirectory();
     assertEquals(1, foo.getDiffs().asList().size());
-    assertEquals("s1", foo.getLastSnapshot().getRoot().getLocalName());
+    assertEquals("s1", foo.getDirectoryWithSnapshotFeature().getLastSnapshot()
+        .getRoot().getLocalName());
     INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
     assertEquals(1, bar1.getDiffs().asList().size());
     assertEquals("s1", bar1.getDiffs().getLastSnapshot().getRoot()
@@ -973,8 +972,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount fooWithCount = (WithCount) fooRef.getReferredINode();
     // 5 references: s1, s22, s333, s2222, current tree of sdir1
     assertEquals(5, fooWithCount.getReferenceCount());
-    INodeDirectoryWithSnapshot foo = (INodeDirectoryWithSnapshot) fooWithCount
-        .asDirectory();
+    INodeDirectory foo = fooWithCount.asDirectory();
     List<DirectoryDiff> fooDiffs = foo.getDiffs().asList();
     assertEquals(4, fooDiffs.size());
     assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1032,7 +1030,7 @@ public class TestRenameWithSnapshots {
     fooRef = fsdir.getINode(foo_s2222.toString()).asReference();
     fooWithCount = (WithCount) fooRef.getReferredINode();
     assertEquals(4, fooWithCount.getReferenceCount());
-    foo = (INodeDirectoryWithSnapshot) fooWithCount.asDirectory();
+    foo = fooWithCount.asDirectory();
     fooDiffs = foo.getDiffs().asList();
     assertEquals(4, fooDiffs.size());
     assertEquals("s2222", fooDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1171,8 +1169,7 @@ public class TestRenameWithSnapshots {
     assertTrue(fooRef instanceof INodeReference.WithName);
     INodeReference.WithCount fooWC = (WithCount) fooRef.getReferredINode();
     assertEquals(1, fooWC.getReferenceCount());
-    INodeDirectoryWithSnapshot fooDir = (INodeDirectoryWithSnapshot) fooWC
-        .getReferredINode().asDirectory();
+    INodeDirectory fooDir = fooWC.getReferredINode().asDirectory();
     List<DirectoryDiff> diffs = fooDir.getDiffs().asList();
     assertEquals(1, diffs.size());
     assertEquals("s2", diffs.get(0).snapshot.getRoot().getLocalName());
@@ -1263,7 +1260,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory mockDir2 = spy(dir2);
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     
@@ -1288,9 +1285,8 @@ public class TestRenameWithSnapshots {
     assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
     
     INode fooNode = fsdir.getINode4Write(foo.toString());
-    assertTrue(fooNode instanceof INodeDirectoryWithSnapshot);
-    List<DirectoryDiff> fooDiffs = ((INodeDirectoryWithSnapshot) fooNode)
-        .getDiffs().asList();
+    assertTrue(fooNode.isDirectory() && fooNode.asDirectory().isWithSnapshot());
+    List<DirectoryDiff> fooDiffs = fooNode.asDirectory().getDiffs().asList();
     assertEquals(1, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     
@@ -1302,7 +1298,7 @@ public class TestRenameWithSnapshots {
     assertFalse(hdfs.exists(newfoo));
     INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
-    assertFalse(dir2Node instanceof INodeDirectoryWithSnapshot);
+    assertFalse(dir2Node.isWithSnapshot());
     ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
     assertEquals(1, dir2Children.size());
     assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
@@ -1331,7 +1327,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory mockDir2 = spy(dir2);
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     
@@ -1366,7 +1362,7 @@ public class TestRenameWithSnapshots {
     assertFalse(hdfs.exists(newfoo));
     INodeDirectory dir2Node = fsdir.getINode4Write(sdir2.toString())
         .asDirectory();
-    assertFalse(dir2Node instanceof INodeDirectoryWithSnapshot);
+    assertFalse(dir2Node.isWithSnapshot());
     ReadOnlyList<INode> dir2Children = dir2Node.getChildrenList(null);
     assertEquals(1, dir2Children.size());
     assertEquals(dir2file.getName(), dir2Children.get(0).getLocalName());
@@ -1393,7 +1389,7 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
     INodeDirectory mockDir3 = spy(dir3);
     doReturn(false).when(mockDir3).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject(), (INodeMap) anyObject());
+            (Snapshot) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     
@@ -1420,8 +1416,7 @@ public class TestRenameWithSnapshots {
     INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
     assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
     assertTrue(fooNode instanceof INodeReference.DstReference);
-    List<DirectoryDiff> fooDiffs = ((INodeDirectoryWithSnapshot) fooNode
-        .asDirectory()).getDiffs().asList();
+    List<DirectoryDiff> fooDiffs = fooNode.asDirectory().getDiffs().asList();
     assertEquals(1, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     
@@ -1455,8 +1450,7 @@ public class TestRenameWithSnapshots {
     assertTrue(hdfs.exists(foo_s3));
     
     assertTrue(fooNode instanceof INodeReference.DstReference);
-    fooDiffs = ((INodeDirectoryWithSnapshot) fooNode.asDirectory()).getDiffs()
-        .asList();
+    fooDiffs = fooNode.asDirectory().getDiffs().asList();
     assertEquals(2, fooDiffs.size());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
     assertEquals("s3", fooDiffs.get(1).snapshot.getRoot().getLocalName());
@@ -1495,10 +1489,9 @@ public class TestRenameWithSnapshots {
     INodeDirectory mockDir3 = spy(dir3);
     // fail the rename but succeed in undo
     doReturn(false).when(mockDir3).addChild((INode) Mockito.isNull(),
-        anyBoolean(), (Snapshot) anyObject(), (INodeMap) anyObject());
-    Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), 
-        anyBoolean(), (Snapshot) anyObject(), 
-        (INodeMap) anyObject())).thenReturn(false).thenCallRealMethod();
+        anyBoolean(), (Snapshot) anyObject());
+    Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), anyBoolean(), 
+        (Snapshot) anyObject())).thenReturn(false).thenCallRealMethod();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     foo3Node.setParent(mockDir3);
@@ -1561,7 +1554,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     INode fooNode = childrenList.get(0);
-    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(fooNode.asDirectory().isWithSnapshot());
     INode barNode = fsdir.getINode4Write(bar.toString());
     assertTrue(barNode.getClass() == INodeFile.class);
     assertSame(fooNode, barNode.getParent());
@@ -1637,7 +1630,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     INode fooNode = childrenList.get(0);
-    assertTrue(fooNode.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(fooNode.asDirectory().isWithSnapshot());
     assertSame(dir1Node, fooNode.getParent());
     List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
         .getDiffs().asList();
@@ -1656,7 +1649,7 @@ public class TestRenameWithSnapshots {
         .getChildrenList(null));
     assertEquals(1, childrenList.size());
     INode subdir2Node = childrenList.get(0);
-    assertTrue(subdir2Node.getClass() == INodeDirectoryWithSnapshot.class);
+    assertTrue(subdir2Node.asDirectory().isWithSnapshot());
     assertSame(dir2Node, subdir2Node.getParent());
     assertSame(subdir2Node, fsdir.getINode4Write(sub_dir2.toString()));
     INode subsubdir2Node = fsdir.getINode4Write(subsub_dir2.toString());
@@ -1669,7 +1662,7 @@ public class TestRenameWithSnapshots {
     assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
     assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
     
-    diffList = ((INodeDirectoryWithSnapshot) subdir2Node).getDiffs().asList();
+    diffList = subdir2Node.asDirectory().getDiffs().asList();
     assertEquals(0, diffList.size());
   }
   
@@ -1697,8 +1690,7 @@ public class TestRenameWithSnapshots {
     }
     
     // check
-    INodeDirectoryWithSnapshot fooNode = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(foo.toString());
+    INodeDirectory fooNode = fsdir.getINode4Write(foo.toString()).asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(1, children.size());
     List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
@@ -1948,8 +1940,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount wc = 
         (WithCount) fooRef.asReference().getReferredINode();
     assertEquals(1, wc.getReferenceCount());
-    INodeDirectoryWithSnapshot fooNode = 
-        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    INodeDirectory fooNode = wc.getReferredINode().asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(1, children.size());
     assertEquals(bar.getName(), children.get(0).getLocalName());
@@ -2017,8 +2008,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount wc = 
         (WithCount) fooRef.asReference().getReferredINode();
     assertEquals(2, wc.getReferenceCount());
-    INodeDirectoryWithSnapshot fooNode = 
-        (INodeDirectoryWithSnapshot) wc.getReferredINode().asDirectory();
+    INodeDirectory fooNode = wc.getReferredINode().asDirectory();
     ReadOnlyList<INode> children = fooNode.getChildrenList(null);
     assertEquals(3, children.size());
     assertEquals(bar.getName(), children.get(0).getLocalName());
@@ -2044,9 +2034,9 @@ public class TestRenameWithSnapshots {
   
   /**
    * This test demonstrates that 
-   * {@link INodeDirectoryWithSnapshot#removeChild(INode, Snapshot, INodeMap)}
+   * {@link INodeDirectory#removeChild(INode, Snapshot)}
    * and 
-   * {@link INodeDirectoryWithSnapshot#addChild(INode, boolean, Snapshot, INodeMap)}
+   * {@link INodeDirectory#addChild(INode, boolean, Snapshot)}
    * should use {@link INode#isInLatestSnapshot(Snapshot)} to check if the 
    * added/removed child should be recorded in snapshots.
    */
@@ -2063,7 +2053,7 @@ public class TestRenameWithSnapshots {
     hdfs.mkdirs(foo);
     SnapshotTestHelper.createSnapshot(hdfs, dir1, "s1");
     final Path bar = new Path(foo, "bar");
-    // create file bar, and foo will become an INodeDirectoryWithSnapshot
+    // create file bar, and foo will become an INodeDirectory with snapshot
     DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
     // delete snapshot s1. now foo is not in any snapshot
     hdfs.deleteSnapshot(dir1, "s1");
@@ -2079,7 +2069,7 @@ public class TestRenameWithSnapshots {
     
     // delete /dir2/foo. Since it is not in any snapshot, we will call its 
     // destroy function. If we do not use isInLatestSnapshot in removeChild and
-    // addChild methods in INodeDirectoryWithSnapshot, the file bar will be 
+    // addChild methods in INodeDirectory (with snapshot), the file bar will be 
     // stored in the deleted list of foo, and will be destroyed.
     hdfs.delete(foo2, true);
     
@@ -2130,8 +2120,8 @@ public class TestRenameWithSnapshots {
     // check the internal
     assertFalse("after deleting s0, " + foo_s0 + " should not exist",
         hdfs.exists(foo_s0));
-    INodeDirectoryWithSnapshot dir2Node = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(dir2.toString());
+    INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString())
+        .asDirectory();
     assertTrue("the diff list of " + dir2
         + " should be empty after deleting s0", dir2Node.getDiffs().asList()
         .isEmpty());
@@ -2140,16 +2130,14 @@ public class TestRenameWithSnapshots {
     INode fooRefNode = fsdir.getINode4Write(newfoo.toString());
     assertTrue(fooRefNode instanceof INodeReference.DstReference);
     INodeDirectory fooNode = fooRefNode.asDirectory();
-    // fooNode should be still INodeDirectoryWithSnapshot since we call
+    // fooNode should be still INodeDirectory (With Snapshot) since we call
     // recordModification before the rename
-    assertTrue(fooNode instanceof INodeDirectoryWithSnapshot);
-    assertTrue(((INodeDirectoryWithSnapshot) fooNode).getDiffs().asList()
-        .isEmpty());
+    assertTrue(fooNode.isWithSnapshot());
+    assertTrue(fooNode.getDiffs().asList().isEmpty());
     INodeDirectory barNode = fooNode.getChildrenList(null).get(0).asDirectory();
-    // bar should also be an INodeDirectoryWithSnapshot, and both of its diff 
+    // bar should also be INodeDirectory (With Snapshot), and both of its diff 
     // list and children list are empty 
-    assertTrue(((INodeDirectoryWithSnapshot) barNode).getDiffs().asList()
-        .isEmpty());
+    assertTrue(barNode.getDiffs().asList().isEmpty());
     assertTrue(barNode.getChildrenList(null).isEmpty());
     
     restartClusterAndCheckImage(true);
@@ -2199,8 +2187,8 @@ public class TestRenameWithSnapshots {
     assertTrue(hdfs.exists(file_s0));
     
     // check dir1: foo should be in the created list of s0
-    INodeDirectoryWithSnapshot dir1Node = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(dir1.toString());
+    INodeDirectory dir1Node = fsdir.getINode4Write(dir1.toString())
+        .asDirectory();
     List<DirectoryDiff> dir1DiffList = dir1Node.getDiffs().asList();
     assertEquals(1, dir1DiffList.size());
     List<INode> dList = dir1DiffList.get(0).getChildrenDiff()
@@ -2215,8 +2203,8 @@ public class TestRenameWithSnapshots {
     
     // check foo and its subtree
     final Path newbar = new Path(newfoo, bar.getName());
-    INodeDirectoryWithSnapshot barNode = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(newbar.toString());
+    INodeDirectory barNode = fsdir.getINode4Write(newbar.toString())
+        .asDirectory();
     assertSame(fooNode.asDirectory(), barNode.getParent());
     // bar should only have a snapshot diff for s0
     List<DirectoryDiff> barDiffList = barNode.getDiffs().asList();
@@ -2229,8 +2217,8 @@ public class TestRenameWithSnapshots {
     
     // check dir2: a WithName instance for foo should be in the deleted list
     // of the snapshot diff for s2
-    INodeDirectoryWithSnapshot dir2Node = (INodeDirectoryWithSnapshot) fsdir
-        .getINode4Write(dir2.toString());
+    INodeDirectory dir2Node = fsdir.getINode4Write(dir2.toString())
+        .asDirectory();
     List<DirectoryDiff> dir2DiffList = dir2Node.getDiffs().asList();
     // dir2Node should contain 2 snapshot diffs, one for s2, and the other was
     // originally s1 (created when dir2 was transformed to a snapshottable dir),
@@ -2287,8 +2275,7 @@ public class TestRenameWithSnapshots {
     // make sure the file under bar is deleted 
     final Path barInS0 = SnapshotTestHelper.getSnapshotPath(test, "s0",
         "foo/bar");
-    INodeDirectoryWithSnapshot barNode = (INodeDirectoryWithSnapshot) fsdir
-        .getINode(barInS0.toString());
+    INodeDirectory barNode = fsdir.getINode(barInS0.toString()).asDirectory();
     assertEquals(0, barNode.getChildrenList(null).size());
     List<DirectoryDiff> diffList = barNode.getDiffs().asList();
     assertEquals(1, diffList.size());

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

@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.junit.After;
 import org.junit.Before;
@@ -92,12 +92,12 @@ public class TestSetQuotaWithSnapshot {
     INodeDirectory subNode = INodeDirectory.valueOf(
         fsdir.getINode(sub.toString()), sub);
     // subNode should be a INodeDirectory, but not an INodeDirectoryWithSnapshot
-    assertFalse(subNode instanceof INodeDirectoryWithSnapshot);
+    assertFalse(subNode.isWithSnapshot());
     
     hdfs.setQuota(sub, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
     subNode = INodeDirectory.valueOf(fsdir.getINode(sub.toString()), sub);
     assertTrue(subNode.isQuotaSet());
-    assertFalse(subNode instanceof INodeDirectoryWithSnapshot);
+    assertFalse(subNode.isWithSnapshot());
   }
   
   /**
@@ -150,8 +150,8 @@ public class TestSetQuotaWithSnapshot {
     DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
     hdfs.setQuota(dir, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_RESET);
     INode subNode = fsdir.getINode4Write(subDir.toString());
-    assertTrue(subNode instanceof INodeDirectoryWithSnapshot);
-    List<DirectoryDiff> diffList = ((INodeDirectoryWithSnapshot) subNode).getDiffs().asList();
+    assertTrue(subNode.asDirectory().isWithSnapshot());
+    List<DirectoryDiff> diffList = subNode.asDirectory().getDiffs().asList();
     assertEquals(1, diffList.size());
     assertEquals("s2", Snapshot.getSnapshotName(diffList.get(0).snapshot));
     List<INode> createdList = diffList.get(0).getChildrenDiff().getList(ListType.CREATED);

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

@@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
@@ -311,9 +311,9 @@ public class TestSnapshotDeletion {
     // make sure the whole subtree of sub is stored correctly in snapshot
     Path snapshotSub = SnapshotTestHelper.getSnapshotPath(dir, "s1",
         sub.getName());
-    INodeDirectoryWithSnapshot snapshotNode4Sub = 
-        (INodeDirectoryWithSnapshot) fsdir.getINode(snapshotSub.toString());
-    assertEquals(INodeDirectoryWithSnapshot.class, snapshotNode4Sub.getClass());
+    INodeDirectory snapshotNode4Sub = fsdir.getINode(snapshotSub.toString())
+        .asDirectory();
+    assertTrue(snapshotNode4Sub.isWithSnapshot());
     // the snapshot copy of sub has only one child subsub.
     // newFile should have been destroyed
     assertEquals(1, snapshotNode4Sub.getChildrenList(null).size());
@@ -323,8 +323,7 @@ public class TestSnapshotDeletion {
     // check the snapshot copy of subsub, which is contained in the subtree of
     // sub's snapshot copy
     INode snapshotNode4Subsub = snapshotNode4Sub.getChildrenList(null).get(0);
-    assertEquals(INodeDirectoryWithSnapshot.class,
-        snapshotNode4Subsub.getClass());
+    assertTrue(snapshotNode4Subsub.asDirectory().isWithSnapshot());
     assertTrue(snapshotNode4Sub == snapshotNode4Subsub.getParent());
     // check the children of subsub
     INodeDirectory snapshotSubsubDir = (INodeDirectory) snapshotNode4Subsub;
@@ -478,8 +477,8 @@ public class TestSnapshotDeletion {
     DirectoryDiffList diffList = dirNode.getDiffs();
     assertEquals(1, diffList.asList().size());
     assertEquals("s1", diffList.getLast().snapshot.getRoot().getLocalName());
-    diffList = ((INodeDirectoryWithSnapshot) fsdir.getINode(
-        metaChangeDir.toString())).getDiffs();
+    diffList = fsdir.getINode(metaChangeDir.toString()).asDirectory()
+        .getDiffs();
     assertEquals(0, diffList.asList().size());
     
     // check 2. noChangeDir and noChangeFile are still there

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

@@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 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.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;