Browse Source

HDFS-4760. Update inodeMap after node replacement. Contributed by Jing Zhao

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1477827 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
3a3e0f5731
22 changed files with 411 additions and 258 deletions
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 0 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  3. 80 117
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  4. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  6. 15 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  7. 19 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  8. 55 36
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  9. 17 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  10. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  11. 137 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  12. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  13. 7 4
      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. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  16. 10 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  17. 20 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  18. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java
  19. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java
  20. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  21. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  22. 12 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

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

@@ -324,3 +324,5 @@ Branch-2802 Snapshot (Unreleased)
 
 
   HDFS-4773. Fix bugs in quota usage computation and OfflineImageViewer.
   HDFS-4773. Fix bugs in quota usage computation and OfflineImageViewer.
   (Jing Zhao via szetszwo)
   (Jing Zhao via szetszwo)
+
+  HDFS-4760. Update inodeMap after node replacement.  (Jing Zhao via szetszwo)

+ 0 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -3016,12 +3016,6 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     return blocksMap.addBlockCollection(block, bc);
     return blocksMap.addBlockCollection(block, bc);
   }
   }
 
 
-  public void addBlockCollection(BlockCollection bc) {
-    for(BlockInfo block : bc.getBlocks()) {
-      addBlockCollection(block, bc);
-    }
-  }
-
   public BlockCollection getBlockCollection(Block b) {
   public BlockCollection getBlockCollection(Block b) {
     return blocksMap.getBlockCollection(b);
     return blocksMap.getBlockCollection(b);
   }
   }

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

@@ -61,10 +61,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 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.INodeReference.WithCount;
-import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -73,8 +71,6 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotAccessControlExce
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotException;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
-import org.apache.hadoop.hdfs.util.GSet;
-import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -116,7 +112,7 @@ public class FSDirectory implements Closeable {
   private final int maxComponentLength;
   private final int maxComponentLength;
   private final int maxDirItems;
   private final int maxDirItems;
   private final int lsLimit;  // max list limit
   private final int lsLimit;  // max list limit
-  private GSet<INode, INodeWithAdditionalFields> inodeMap; // Synchronized by dirLock
+  private final INodeMap inodeMap; // Synchronized by dirLock
 
 
   // lock to protect the directory and BlockMap
   // lock to protect the directory and BlockMap
   private ReentrantReadWriteLock dirLock;
   private ReentrantReadWriteLock dirLock;
@@ -157,7 +153,7 @@ public class FSDirectory implements Closeable {
     this.dirLock = new ReentrantReadWriteLock(true); // fair
     this.dirLock = new ReentrantReadWriteLock(true); // fair
     this.cond = dirLock.writeLock().newCondition();
     this.cond = dirLock.writeLock().newCondition();
     rootDir = createRoot(ns);
     rootDir = createRoot(ns);
-    inodeMap = initInodeMap(rootDir);
+    inodeMap = INodeMap.newInstance(rootDir);
     this.fsImage = fsImage;
     this.fsImage = fsImage;
     int configuredLimit = conf.getInt(
     int configuredLimit = conf.getInt(
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
@@ -180,16 +176,6 @@ public class FSDirectory implements Closeable {
     nameCache = new NameCache<ByteArray>(threshold);
     nameCache = new NameCache<ByteArray>(threshold);
     namesystem = ns;
     namesystem = ns;
   }
   }
-  
-  private static GSet<INode, INodeWithAdditionalFields> initInodeMap(
-      INodeDirectory rootDir) {
-    // Compute the map capacity by allocating 1% of total memory
-    int capacity = LightWeightGSet.computeCapacity(1, "INodeMap");
-    GSet<INode, INodeWithAdditionalFields> map
-        = new LightWeightGSet<INode, INodeWithAdditionalFields>(capacity);
-    map.put(rootDir);
-    return map;
-  }
     
     
   private FSNamesystem getFSNamesystem() {
   private FSNamesystem getFSNamesystem() {
     return namesystem;
     return namesystem;
@@ -608,7 +594,8 @@ public class FSDirectory implements Closeable {
     // snapshot is taken on the dst tree, changes will be recorded in the latest
     // snapshot is taken on the dst tree, changes will be recorded in the latest
     // snapshot of the src tree.
     // snapshot of the src tree.
     if (isSrcInSnapshot) {
     if (isSrcInSnapshot) {
-      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot());
+      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot(),
+          inodeMap);
       srcIIP.setLastINode(srcChild);
       srcIIP.setLastINode(srcChild);
     }
     }
     
     
@@ -618,8 +605,9 @@ public class FSDirectory implements Closeable {
     int srcRefDstSnapshot = srcChildIsReference ? srcChild.asReference()
     int srcRefDstSnapshot = srcChildIsReference ? srcChild.asReference()
         .getDstSnapshotId() : Snapshot.INVALID_ID;
         .getDstSnapshotId() : Snapshot.INVALID_ID;
     if (isSrcInSnapshot) {
     if (isSrcInSnapshot) {
-      final INodeReference.WithName withName = srcIIP.getINode(-2).asDirectory()
-          .replaceChild4ReferenceWithName(srcChild, srcIIP.getLatestSnapshot()); 
+      final INodeReference.WithName withName = 
+          srcIIP.getINode(-2).asDirectory().replaceChild4ReferenceWithName(
+              srcChild, srcIIP.getLatestSnapshot()); 
       withCount = (INodeReference.WithCount) withName.getReferredINode();
       withCount = (INodeReference.WithCount) withName.getReferredINode();
       srcChild = withName;
       srcChild = withName;
       srcIIP.setLastINode(srcChild);
       srcIIP.setLastINode(srcChild);
@@ -676,9 +664,11 @@ public class FSDirectory implements Closeable {
         }
         }
         // update modification time of dst and the parent of src
         // update modification time of dst and the parent of src
         final INode srcParent = srcIIP.getINode(-2);
         final INode srcParent = srcIIP.getINode(-2);
-        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
+        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot(),
+            inodeMap);
         dstParent = dstIIP.getINode(-2); // refresh dstParent
         dstParent = dstIIP.getINode(-2); // refresh dstParent
-        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
+        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot(),
+            inodeMap);
         // update moved leases with new filename
         // update moved leases with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);     
         getFSNamesystem().unprotectedChangeLease(src, dst);     
 
 
@@ -859,7 +849,8 @@ public class FSDirectory implements Closeable {
     // snapshot is taken on the dst tree, changes will be recorded in the latest
     // snapshot is taken on the dst tree, changes will be recorded in the latest
     // snapshot of the src tree.
     // snapshot of the src tree.
     if (isSrcInSnapshot) {
     if (isSrcInSnapshot) {
-      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot());
+      srcChild = srcChild.recordModification(srcIIP.getLatestSnapshot(),
+          inodeMap);
       srcIIP.setLastINode(srcChild);
       srcIIP.setLastINode(srcChild);
     }
     }
     
     
@@ -937,9 +928,11 @@ public class FSDirectory implements Closeable {
         }
         }
 
 
         final INode srcParent = srcIIP.getINode(-2);
         final INode srcParent = srcIIP.getINode(-2);
-        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot());
+        srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshot(),
+            inodeMap);
         dstParent = dstIIP.getINode(-2);
         dstParent = dstIIP.getINode(-2);
-        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot());
+        dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshot(),
+            inodeMap);
         // update moved lease with new filename
         // update moved lease with new filename
         getFSNamesystem().unprotectedChangeLease(src, dst);
         getFSNamesystem().unprotectedChangeLease(src, dst);
 
 
@@ -1071,7 +1064,8 @@ public class FSDirectory implements Closeable {
       updateCount(iip, 0, dsDelta, true);
       updateCount(iip, 0, dsDelta, true);
     }
     }
 
 
-    file = file.setFileReplication(replication, iip.getLatestSnapshot());
+    file = file.setFileReplication(replication, iip.getLatestSnapshot(),
+        inodeMap);
     
     
     final short newBR = file.getBlockReplication(); 
     final short newBR = file.getBlockReplication(); 
     // check newBR < oldBR case. 
     // check newBR < oldBR case. 
@@ -1137,7 +1131,8 @@ public class FSDirectory implements Closeable {
     if (inode == null) {
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
       throw new FileNotFoundException("File does not exist: " + src);
     }
     }
-    inode.setPermission(permissions, inodesInPath.getLatestSnapshot());
+    inode.setPermission(permissions, inodesInPath.getLatestSnapshot(), 
+        inodeMap);
   }
   }
 
 
   void setOwner(String src, String username, String groupname)
   void setOwner(String src, String username, String groupname)
@@ -1162,10 +1157,11 @@ public class FSDirectory implements Closeable {
       throw new FileNotFoundException("File does not exist: " + src);
       throw new FileNotFoundException("File does not exist: " + src);
     }
     }
     if (username != null) {
     if (username != null) {
-      inode = inode.setUser(username, inodesInPath.getLatestSnapshot());
+      inode = inode.setUser(username, inodesInPath.getLatestSnapshot(),
+          inodeMap);
     }
     }
     if (groupname != null) {
     if (groupname != null) {
-      inode.setGroup(groupname, inodesInPath.getLatestSnapshot());
+      inode.setGroup(groupname, inodesInPath.getLatestSnapshot(), inodeMap);
     }
     }
   }
   }
 
 
@@ -1237,12 +1233,15 @@ public class FSDirectory implements Closeable {
       if(nodeToRemove == null) continue;
       if(nodeToRemove == null) continue;
       
       
       nodeToRemove.setBlocks(null);
       nodeToRemove.setBlocks(null);
-      trgParent.removeChild(nodeToRemove, trgLatestSnapshot);
+      trgParent.removeChild(nodeToRemove, trgLatestSnapshot, null);
       count++;
       count++;
     }
     }
     
     
-    trgInode.setModificationTime(timestamp, trgLatestSnapshot);
-    trgParent.updateModificationTime(timestamp, trgLatestSnapshot);
+    // update inodeMap
+    removeFromInodeMap(Arrays.asList(allSrcInodes));
+    
+    trgInode.setModificationTime(timestamp, trgLatestSnapshot, inodeMap);
+    trgParent.updateModificationTime(timestamp, trgLatestSnapshot, inodeMap);
     // update quota on the parent directory ('count' files removed, 0 space)
     // update quota on the parent directory ('count' files removed, 0 space)
     unprotectedUpdateCount(trgIIP, trgINodes.length-1, -count, 0);
     unprotectedUpdateCount(trgIIP, trgINodes.length-1, -count, 0);
   }
   }
@@ -1384,7 +1383,7 @@ public class FSDirectory implements Closeable {
 
 
     // record modification
     // record modification
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
-    targetNode = targetNode.recordModification(latestSnapshot);
+    targetNode = targetNode.recordModification(latestSnapshot, inodeMap);
     iip.setLastINode(targetNode);
     iip.setLastINode(targetNode);
 
 
     // Remove the node from the namespace
     // Remove the node from the namespace
@@ -1395,7 +1394,7 @@ public class FSDirectory implements Closeable {
 
 
     // set the parent's modification time
     // set the parent's modification time
     final INodeDirectory parent = targetNode.getParent();
     final INodeDirectory parent = targetNode.getParent();
-    parent.updateModificationTime(mtime, latestSnapshot);
+    parent.updateModificationTime(mtime, latestSnapshot, inodeMap);
     if (removed == 0) {
     if (removed == 0) {
       return 0;
       return 0;
     }
     }
@@ -1466,8 +1465,7 @@ public class FSDirectory implements Closeable {
       final INodeFile newnode) {
       final INodeFile newnode) {
     Preconditions.checkState(hasWriteLock());
     Preconditions.checkState(hasWriteLock());
 
 
-    oldnode.getParent().replaceChild(oldnode, newnode);
-    addToInodeMapUnprotected(newnode);
+    oldnode.getParent().replaceChild(oldnode, newnode, inodeMap);
     oldnode.clear();
     oldnode.clear();
 
 
     /* Currently oldnode and newnode are assumed to contain the same
     /* Currently oldnode and newnode are assumed to contain the same
@@ -1984,15 +1982,6 @@ public class FSDirectory implements Closeable {
     }
     }
   }
   }
   
   
-  private INode getFromINodeMap(INode inode) {
-    readLock();
-    try {
-      return inodeMap.get(inode);
-    } finally {
-      readUnlock();
-    }
-  }
-  
   /**
   /**
    * Add the given child to the namespace.
    * Add the given child to the namespace.
    * @param src The full path name of the child node.
    * @param src The full path name of the child node.
@@ -2194,14 +2183,14 @@ public class FSDirectory implements Closeable {
     updateCount(iip, pos,
     updateCount(iip, pos,
         counts.get(Quota.NAMESPACE), counts.get(Quota.DISKSPACE), checkQuota);
         counts.get(Quota.NAMESPACE), counts.get(Quota.DISKSPACE), checkQuota);
     final INodeDirectory parent = inodes[pos-1].asDirectory();
     final INodeDirectory parent = inodes[pos-1].asDirectory();
-    final boolean added = parent.addChild(child, true, iip.getLatestSnapshot());
+    final boolean added = parent.addChild(child, true, iip.getLatestSnapshot(),
+        inodeMap);
     if (!added) {
     if (!added) {
       updateCountNoQuotaCheck(iip, pos,
       updateCountNoQuotaCheck(iip, pos,
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
           -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
     } else {
     } else {
-      // update parent node
       iip.setINode(pos - 1, child.getParent());
       iip.setINode(pos - 1, child.getParent());
-      addToInodeMapUnprotected(child);
+      addToInodeMap(child);
     }
     }
     return added;
     return added;
   }
   }
@@ -2228,13 +2217,12 @@ public class FSDirectory implements Closeable {
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final Snapshot latestSnapshot = iip.getLatestSnapshot();
     final INode last = iip.getLastINode();
     final INode last = iip.getLastINode();
     final INodeDirectory parent = iip.getINode(-2).asDirectory();
     final INodeDirectory parent = iip.getINode(-2).asDirectory();
-    if (!parent.removeChild(last, latestSnapshot)) {
+    if (!parent.removeChild(last, latestSnapshot, inodeMap)) {
       return -1;
       return -1;
     }
     }
-    if (parent != last.getParent()) {
-      // parent is changed
-      addToInodeMapUnprotected(last.getParent());
-      iip.setINode(-2, last.getParent());
+    INodeDirectory newParent = last.getParent();
+    if (parent != newParent) {
+      iip.setINode(-2, newParent);
     }
     }
     
     
     if (!last.isInLatestSnapshot(latestSnapshot)) {
     if (!last.isInLatestSnapshot(latestSnapshot)) {
@@ -2277,22 +2265,51 @@ public class FSDirectory implements Closeable {
     }
     }
   }
   }
 
 
-  /** This method is always called with writeLock held */
-  final void addToInodeMapUnprotected(INode inode) {
+  public INodeMap getINodeMap() {
+    return inodeMap;
+  }
+  
+  /**
+   * This method is always called with writeLock of FSDirectory held.
+   */
+  public final void addToInodeMap(INode inode) {
     if (inode instanceof INodeWithAdditionalFields) {
     if (inode instanceof INodeWithAdditionalFields) {
       inodeMap.put((INodeWithAdditionalFields)inode);
       inodeMap.put((INodeWithAdditionalFields)inode);
     }
     }
   }
   }
   
   
-  /* This method is always called with writeLock held */
-  final void removeFromInodeMap(List<INode> inodes) {
+  /**
+   * This method is always called with writeLock of FSDirectory held.
+   */
+  public final void removeFromInodeMap(List<? extends INode> inodes) {
     if (inodes != null) {
     if (inodes != null) {
       for (INode inode : inodes) {
       for (INode inode : inodes) {
-        inodeMap.remove(inode);
+        if (inode != null && inode instanceof INodeWithAdditionalFields) {
+          inodeMap.remove(inode);
+        }
       }
       }
     }
     }
   }
   }
   
   
+  /**
+   * Get the inode from inodeMap based on its inode id.
+   * @param id The given id
+   * @return The inode associated with the given id
+   */
+  public INode getInode(long id) {
+    readLock();
+    try {
+      return inodeMap.get(id);
+    } finally {
+      readUnlock();
+    }
+  }
+  
+  @VisibleForTesting
+  int getInodeMapSize() {
+    return inodeMap.size();
+  }
+  
   /**
   /**
    * See {@link ClientProtocol#setQuota(String, long, long)} for the contract.
    * See {@link ClientProtocol#setQuota(String, long, long)} for the contract.
    * Sets quota for for a directory.
    * Sets quota for for a directory.
@@ -2354,18 +2371,12 @@ public class FSDirectory implements Closeable {
           if (!(quotaNode instanceof INodeDirectoryWithSnapshot)) {
           if (!(quotaNode instanceof INodeDirectoryWithSnapshot)) {
             // will not come here for root because root is snapshottable and
             // will not come here for root because root is snapshottable and
             // root's nsQuota is always set
             // root's nsQuota is always set
-            INodeDirectory newNode = quotaNode.replaceSelf4INodeDirectory();
-            // update the inodeMap
-            inodeMap.put(newNode);
-            return newNode;
-          } 
+            return quotaNode.replaceSelf4INodeDirectory(inodeMap);
+          }
         }
         }
       } else {
       } else {
         // a non-quota directory; so replace it with a directory with quota
         // a non-quota directory; so replace it with a directory with quota
-        INodeDirectory newNode = dirNode.replaceSelf4Quota(latest, nsQuota, dsQuota);
-        // update the inodeMap
-        inodeMap.put(newNode);
-        return newNode;
+        return dirNode.replaceSelf4Quota(latest, nsQuota, dsQuota, inodeMap);
       }
       }
       return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null;
       return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null;
     }
     }
@@ -2431,7 +2442,7 @@ public class FSDirectory implements Closeable {
     assert hasWriteLock();
     assert hasWriteLock();
     boolean status = false;
     boolean status = false;
     if (mtime != -1) {
     if (mtime != -1) {
-      inode = inode.setModificationTime(mtime, latest);
+      inode = inode.setModificationTime(mtime, latest, inodeMap);
       status = true;
       status = true;
     }
     }
     if (atime != -1) {
     if (atime != -1) {
@@ -2442,7 +2453,7 @@ public class FSDirectory implements Closeable {
       if (atime <= inodeTime + getFSNamesystem().getAccessTimePrecision() && !force) {
       if (atime <= inodeTime + getFSNamesystem().getAccessTimePrecision() && !force) {
         status =  false;
         status =  false;
       } else {
       } else {
-        inode.setAccessTime(atime, latest);
+        inode.setAccessTime(atime, latest, inodeMap);
         status = true;
         status = true;
       }
       }
     } 
     } 
@@ -2458,7 +2469,7 @@ public class FSDirectory implements Closeable {
       setReady(false);
       setReady(false);
       rootDir = createRoot(getFSNamesystem());
       rootDir = createRoot(getFSNamesystem());
       inodeMap.clear();
       inodeMap.clear();
-      addToInodeMapUnprotected(rootDir);
+      addToInodeMap(rootDir);
       nameCache.reset();
       nameCache.reset();
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
@@ -2622,49 +2633,6 @@ public class FSDirectory implements Closeable {
   void shutdown() {
   void shutdown() {
     nameCache.reset();
     nameCache.reset();
     inodeMap.clear();
     inodeMap.clear();
-    inodeMap = null;
-  }
-  
-  INode getInode(long id) {
-    INode inode = new INodeWithAdditionalFields(id, null, new PermissionStatus(
-        "", "", new FsPermission((short) 0)), 0, 0) {
-      
-      @Override
-      INode recordModification(Snapshot latest) throws QuotaExceededException {
-        return null;
-      }
-      
-      @Override
-      public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
-          List<INode> removedINodes) {
-        // Nothing to do
-      }
-      
-      @Override
-      public Counts computeQuotaUsage(Counts counts, boolean useCache,
-          int lastSnapshotId) {
-        return null;
-      }
-      
-      @Override
-      public Content.Counts computeContentSummary(Content.Counts counts) {
-        return null;
-      }
-      
-      @Override
-      public CountsMap computeContentSummary(CountsMap countsMap) {
-        return null;
-      }
-      
-      @Override
-      public Counts cleanSubtree(Snapshot snapshot, Snapshot prior,
-          BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes)
-          throws QuotaExceededException {
-        return null;
-      }
-    };
-      
-    return getFromINodeMap(inode);
   }
   }
   
   
   /**
   /**
@@ -2732,11 +2700,6 @@ public class FSDirectory implements Closeable {
     return path.toString();
     return path.toString();
   }
   }
   
   
-  @VisibleForTesting
-  int getInodeMapSize() {
-    return inodeMap.size();
-  }
-  
   /** Check if a given inode name is reserved */
   /** Check if a given inode name is reserved */
   public static boolean isReservedName(INode inode) {
   public static boolean isReservedName(INode inode) {
     return CHECK_RESERVED_FILE_NAMES
     return CHECK_RESERVED_FILE_NAMES

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

@@ -315,8 +315,8 @@ public class FSEditLogLoader {
       // update the block list.
       // update the block list.
       
       
       // Update the salient file attributes.
       // Update the salient file attributes.
-      newFile.setAccessTime(addCloseOp.atime, null);
-      newFile.setModificationTime(addCloseOp.mtime, null);
+      newFile.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
+      newFile.setModificationTime(addCloseOp.mtime, null, fsDir.getINodeMap());
       updateBlocks(fsDir, addCloseOp, newFile);
       updateBlocks(fsDir, addCloseOp, newFile);
       break;
       break;
     }
     }
@@ -334,8 +334,8 @@ public class FSEditLogLoader {
       final INodeFile oldFile = INodeFile.valueOf(iip.getINode(0), addCloseOp.path);
       final INodeFile oldFile = INodeFile.valueOf(iip.getINode(0), addCloseOp.path);
 
 
       // Update the salient file attributes.
       // Update the salient file attributes.
-      oldFile.setAccessTime(addCloseOp.atime, null);
-      oldFile.setModificationTime(addCloseOp.mtime, null);
+      oldFile.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
+      oldFile.setModificationTime(addCloseOp.mtime, null, fsDir.getINodeMap());
       updateBlocks(fsDir, addCloseOp, oldFile);
       updateBlocks(fsDir, addCloseOp, oldFile);
 
 
       // Now close the file
       // Now close the file

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

@@ -560,7 +560,7 @@ public class FSImageFormat {
       final byte[] localName = FSImageSerialization.readLocalName(in);
       final byte[] localName = FSImageSerialization.readLocalName(in);
       INode inode = loadINode(localName, isSnapshotINode, in);
       INode inode = loadINode(localName, isSnapshotINode, in);
       if (LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
       if (LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
-        namesystem.dir.addToInodeMapUnprotected(inode);
+        namesystem.dir.addToInodeMap(inode);
       }
       }
       return inode;
       return inode;
     }
     }

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

@@ -1990,7 +1990,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   LocatedBlock prepareFileForWrite(String src, INodeFile file,
   LocatedBlock prepareFileForWrite(String src, INodeFile file,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       boolean writeToEditLog, Snapshot latestSnapshot) throws IOException {
       boolean writeToEditLog, Snapshot latestSnapshot) throws IOException {
-    file = file.recordModification(latestSnapshot);
+    file = file.recordModification(latestSnapshot, dir.getINodeMap());
     final INodeFileUnderConstruction cons = file.toUnderConstruction(
     final INodeFileUnderConstruction cons = file.toUnderConstruction(
         leaseHolder, clientMachine, clientNode);
         leaseHolder, clientMachine, clientNode);
 
 
@@ -3400,7 +3400,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     assert hasWriteLock();
     assert hasWriteLock();
     leaseManager.removeLease(pendingFile.getClientName(), src);
     leaseManager.removeLease(pendingFile.getClientName(), src);
     
     
-    pendingFile = pendingFile.recordModification(latestSnapshot);
+    pendingFile = pendingFile.recordModification(latestSnapshot,
+        dir.getINodeMap());
 
 
     // The file is no longer pending.
     // The file is no longer pending.
     // Create permanent INode, update blocks
     // Create permanent INode, update blocks
@@ -5823,7 +5824,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       }
       checkSuperuserPrivilege();
       checkSuperuserPrivilege();
 
 
-      snapshotManager.setSnapshottable(path);
+      dir.writeLock();
+      try {
+        snapshotManager.setSnapshottable(path);
+      } finally {
+        dir.writeUnlock();
+      }
       getEditLog().logAllowSnapshot(path);
       getEditLog().logAllowSnapshot(path);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
@@ -5846,7 +5852,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       }
       checkSuperuserPrivilege();
       checkSuperuserPrivilege();
 
 
-      snapshotManager.resetSnapshottable(path);
+      dir.writeLock();
+      try {
+        snapshotManager.resetSnapshottable(path);
+      } finally {
+        dir.writeUnlock();
+      }
       getEditLog().logDisallowSnapshot(path);
       getEditLog().logDisallowSnapshot(path);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();

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

@@ -96,9 +96,9 @@ public abstract class INode implements Diff.Element<byte[]> {
   abstract void setUser(String user);
   abstract void setUser(String user);
 
 
   /** Set user */
   /** Set user */
-  final INode setUser(String user, Snapshot latest)
+  final INode setUser(String user, Snapshot latest, INodeMap inodeMap)
       throws QuotaExceededException {
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest);
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
     nodeToUpdate.setUser(user);
     nodeToUpdate.setUser(user);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }
@@ -119,9 +119,9 @@ public abstract class INode implements Diff.Element<byte[]> {
   abstract void setGroup(String group);
   abstract void setGroup(String group);
 
 
   /** Set group */
   /** Set group */
-  final INode setGroup(String group, Snapshot latest)
+  final INode setGroup(String group, Snapshot latest, INodeMap inodeMap)
       throws QuotaExceededException {
       throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest);
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
     nodeToUpdate.setGroup(group);
     nodeToUpdate.setGroup(group);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }
@@ -143,9 +143,9 @@ public abstract class INode implements Diff.Element<byte[]> {
   abstract void setPermission(FsPermission permission);
   abstract void setPermission(FsPermission permission);
 
 
   /** Set the {@link FsPermission} of this {@link INode} */
   /** Set the {@link FsPermission} of this {@link INode} */
-  INode setPermission(FsPermission permission, Snapshot latest)
-      throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest);
+  INode setPermission(FsPermission permission, Snapshot latest,
+      INodeMap inodeMap) throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
     nodeToUpdate.setPermission(permission);
     nodeToUpdate.setPermission(permission);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }
@@ -219,12 +219,14 @@ public abstract class INode implements Diff.Element<byte[]> {
    *
    *
    * @param latest the latest snapshot that has been taken.
    * @param latest the latest snapshot that has been taken.
    *        Note that it is null if no snapshots have 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.
    * @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
    *         However, in some cases, this inode may be replaced with a new inode
    *         for maintaining snapshots. The current inode is then the new inode.
    *         for maintaining snapshots. The current inode is then the new inode.
    */
    */
-  abstract INode recordModification(final Snapshot latest)
-      throws QuotaExceededException;
+  abstract INode recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException;
 
 
   /** Check whether it's a reference. */
   /** Check whether it's a reference. */
   public boolean isReference() {
   public boolean isReference() {
@@ -564,16 +566,16 @@ public abstract class INode implements Diff.Element<byte[]> {
   }
   }
 
 
   /** Update modification time if it is larger than the current value. */
   /** Update modification time if it is larger than the current value. */
-  public abstract INode updateModificationTime(long mtime, Snapshot latest)
-      throws QuotaExceededException;
+  public abstract INode updateModificationTime(long mtime, Snapshot latest,
+      INodeMap inodeMap) throws QuotaExceededException;
 
 
   /** Set the last modification time of inode. */
   /** Set the last modification time of inode. */
   public abstract void setModificationTime(long modificationTime);
   public abstract void setModificationTime(long modificationTime);
 
 
   /** Set the last modification time of inode. */
   /** Set the last modification time of inode. */
-  public final INode setModificationTime(long modificationTime, Snapshot latest)
-      throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest);
+  public final INode setModificationTime(long modificationTime,
+      Snapshot latest, INodeMap inodeMap) throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
     nodeToUpdate.setModificationTime(modificationTime);
     nodeToUpdate.setModificationTime(modificationTime);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }
@@ -599,9 +601,9 @@ public abstract class INode implements Diff.Element<byte[]> {
   /**
   /**
    * Set last access time of inode.
    * Set last access time of inode.
    */
    */
-  public final INode setAccessTime(long accessTime, Snapshot latest)
-      throws QuotaExceededException {
-    final INode nodeToUpdate = recordModification(latest);
+  public final INode setAccessTime(long accessTime, Snapshot latest,
+      INodeMap inodeMap) throws QuotaExceededException {
+    final INode nodeToUpdate = recordModification(latest, inodeMap);
     nodeToUpdate.setAccessTime(accessTime);
     nodeToUpdate.setAccessTime(accessTime);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }

+ 55 - 36
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -113,11 +113,11 @@ public class INodeDirectory extends INodeWithAdditionalFields {
    * @param child the child inode to be removed
    * @param child the child inode to be removed
    * @param latest See {@link INode#recordModification(Snapshot)}.
    * @param latest See {@link INode#recordModification(Snapshot)}.
    */
    */
-  public boolean removeChild(INode child, Snapshot latest)
-      throws QuotaExceededException {
+  public boolean removeChild(INode child, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
     if (isInLatestSnapshot(latest)) {
     if (isInLatestSnapshot(latest)) {
-      return replaceSelf4INodeDirectoryWithSnapshot()
-          .removeChild(child, latest);
+      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
+          .removeChild(child, latest, inodeMap);
     }
     }
 
 
     return removeChild(child);
     return removeChild(child);
@@ -147,59 +147,66 @@ public class INodeDirectory extends INodeWithAdditionalFields {
    * {@link INodeDirectoryWithSnapshot} depending on the latest snapshot.
    * {@link INodeDirectoryWithSnapshot} depending on the latest snapshot.
    */
    */
   INodeDirectoryWithQuota replaceSelf4Quota(final Snapshot latest,
   INodeDirectoryWithQuota replaceSelf4Quota(final Snapshot latest,
-      final long nsQuota, final long dsQuota) throws QuotaExceededException {
+      final long nsQuota, final long dsQuota, final INodeMap inodeMap)
+      throws QuotaExceededException {
     Preconditions.checkState(!(this instanceof INodeDirectoryWithQuota),
     Preconditions.checkState(!(this instanceof INodeDirectoryWithQuota),
         "this is already an INodeDirectoryWithQuota, this=%s", this);
         "this is already an INodeDirectoryWithQuota, this=%s", this);
 
 
     if (!this.isInLatestSnapshot(latest)) {
     if (!this.isInLatestSnapshot(latest)) {
       final INodeDirectoryWithQuota q = new INodeDirectoryWithQuota(
       final INodeDirectoryWithQuota q = new INodeDirectoryWithQuota(
           this, true, nsQuota, dsQuota);
           this, true, nsQuota, dsQuota);
-      replaceSelf(q);
+      replaceSelf(q, inodeMap);
       return q;
       return q;
     } else {
     } else {
       final INodeDirectoryWithSnapshot s = new INodeDirectoryWithSnapshot(this);
       final INodeDirectoryWithSnapshot s = new INodeDirectoryWithSnapshot(this);
       s.setQuota(nsQuota, dsQuota);
       s.setQuota(nsQuota, dsQuota);
-      return replaceSelf(s).saveSelf2Snapshot(latest, this);
+      return replaceSelf(s, inodeMap).saveSelf2Snapshot(latest, this);
     }
     }
   }
   }
   /** Replace itself with an {@link INodeDirectorySnapshottable}. */
   /** Replace itself with an {@link INodeDirectorySnapshottable}. */
   public INodeDirectorySnapshottable replaceSelf4INodeDirectorySnapshottable(
   public INodeDirectorySnapshottable replaceSelf4INodeDirectorySnapshottable(
-      Snapshot latest) throws QuotaExceededException {
+      Snapshot latest, final INodeMap inodeMap) throws QuotaExceededException {
     Preconditions.checkState(!(this instanceof INodeDirectorySnapshottable),
     Preconditions.checkState(!(this instanceof INodeDirectorySnapshottable),
         "this is already an INodeDirectorySnapshottable, this=%s", this);
         "this is already an INodeDirectorySnapshottable, this=%s", this);
     final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(this);
     final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(this);
-    replaceSelf(s).saveSelf2Snapshot(latest, this);
+    replaceSelf(s, inodeMap).saveSelf2Snapshot(latest, this);
     return s;
     return s;
   }
   }
 
 
   /** Replace itself with an {@link INodeDirectoryWithSnapshot}. */
   /** Replace itself with an {@link INodeDirectoryWithSnapshot}. */
-  public INodeDirectoryWithSnapshot replaceSelf4INodeDirectoryWithSnapshot() {
-    return replaceSelf(new INodeDirectoryWithSnapshot(this));
+  public INodeDirectoryWithSnapshot replaceSelf4INodeDirectoryWithSnapshot(
+      final INodeMap inodeMap) {
+    return replaceSelf(new INodeDirectoryWithSnapshot(this), inodeMap);
   }
   }
 
 
   /** Replace itself with {@link INodeDirectory}. */
   /** Replace itself with {@link INodeDirectory}. */
-  public INodeDirectory replaceSelf4INodeDirectory() {
+  public INodeDirectory replaceSelf4INodeDirectory(final INodeMap inodeMap) {
     Preconditions.checkState(getClass() != INodeDirectory.class,
     Preconditions.checkState(getClass() != INodeDirectory.class,
         "the class is already INodeDirectory, this=%s", this);
         "the class is already INodeDirectory, this=%s", this);
-    return replaceSelf(new INodeDirectory(this, true));
+    return replaceSelf(new INodeDirectory(this, true), inodeMap);
   }
   }
 
 
   /** Replace itself with the given directory. */
   /** Replace itself with the given directory. */
-  private final <N extends INodeDirectory> N replaceSelf(final N newDir) {
+  private final <N extends INodeDirectory> N replaceSelf(final N newDir,
+      final INodeMap inodeMap) {
     final INodeReference ref = getParentReference();
     final INodeReference ref = getParentReference();
     if (ref != null) {
     if (ref != null) {
       ref.setReferredINode(newDir);
       ref.setReferredINode(newDir);
+      if (inodeMap != null) {
+        inodeMap.put(newDir);
+      }
     } else {
     } else {
       final INodeDirectory parent = getParent();
       final INodeDirectory parent = getParent();
       Preconditions.checkArgument(parent != null, "parent is null, this=%s", this);
       Preconditions.checkArgument(parent != null, "parent is null, this=%s", this);
-      parent.replaceChild(this, newDir);
+      parent.replaceChild(this, newDir, inodeMap);
     }
     }
     clear();
     clear();
     return newDir;
     return newDir;
   }
   }
 
 
   /** Replace the given child with a new child. */
   /** Replace the given child with a new child. */
-  public void replaceChild(INode oldChild, final INode newChild) {
+  public void replaceChild(INode oldChild, final INode newChild,
+      final INodeMap inodeMap) {
     Preconditions.checkNotNull(children);
     Preconditions.checkNotNull(children);
     final int i = searchChildren(newChild.getLocalNameBytes());
     final int i = searchChildren(newChild.getLocalNameBytes());
     Preconditions.checkState(i >= 0);
     Preconditions.checkState(i >= 0);
@@ -220,9 +227,12 @@ public class INodeDirectory extends INodeWithAdditionalFields {
             (WithCount) oldChild.asReference().getReferredINode();
             (WithCount) oldChild.asReference().getReferredINode();
         withCount.removeReference(oldChild.asReference());
         withCount.removeReference(oldChild.asReference());
       }
       }
-      // do the replacement
       children.set(i, newChild);
       children.set(i, newChild);
     }
     }
+    // update the inodeMap
+    if (inodeMap != null) {
+      inodeMap.put(newChild);
+    }
   }
   }
 
 
   INodeReference.WithName replaceChild4ReferenceWithName(INode oldChild,
   INodeReference.WithName replaceChild4ReferenceWithName(INode oldChild,
@@ -241,43 +251,47 @@ public class INodeDirectory extends INodeWithAdditionalFields {
     }
     }
     final INodeReference.WithName ref = new INodeReference.WithName(this,
     final INodeReference.WithName ref = new INodeReference.WithName(this,
         withCount, oldChild.getLocalNameBytes(), latest.getId());
         withCount, oldChild.getLocalNameBytes(), latest.getId());
-    replaceChild(oldChild, ref);
+    replaceChild(oldChild, ref, null);
     return ref;
     return ref;
   }
   }
   
   
-  private void replaceChildFile(final INodeFile oldChild, final INodeFile newChild) {
-    replaceChild(oldChild, newChild);
+  private void replaceChildFile(final INodeFile oldChild,
+      final INodeFile newChild, final INodeMap inodeMap) {
+    replaceChild(oldChild, newChild, inodeMap);
     oldChild.clear();
     oldChild.clear();
     newChild.updateBlockCollection();
     newChild.updateBlockCollection();
   }
   }
 
 
   /** Replace a child {@link INodeFile} with an {@link INodeFileWithSnapshot}. */
   /** Replace a child {@link INodeFile} with an {@link INodeFileWithSnapshot}. */
   INodeFileWithSnapshot replaceChild4INodeFileWithSnapshot(
   INodeFileWithSnapshot replaceChild4INodeFileWithSnapshot(
-      final INodeFile child) {
+      final INodeFile child, final INodeMap inodeMap) {
     Preconditions.checkArgument(!(child instanceof INodeFileWithSnapshot),
     Preconditions.checkArgument(!(child instanceof INodeFileWithSnapshot),
         "Child file is already an INodeFileWithSnapshot, child=" + child);
         "Child file is already an INodeFileWithSnapshot, child=" + child);
     final INodeFileWithSnapshot newChild = new INodeFileWithSnapshot(child);
     final INodeFileWithSnapshot newChild = new INodeFileWithSnapshot(child);
-    replaceChildFile(child, newChild);
+    replaceChildFile(child, newChild, inodeMap);
     return newChild;
     return newChild;
   }
   }
 
 
   /** Replace a child {@link INodeFile} with an {@link INodeFileUnderConstructionWithSnapshot}. */
   /** Replace a child {@link INodeFile} with an {@link INodeFileUnderConstructionWithSnapshot}. */
   INodeFileUnderConstructionWithSnapshot replaceChild4INodeFileUcWithSnapshot(
   INodeFileUnderConstructionWithSnapshot replaceChild4INodeFileUcWithSnapshot(
-      final INodeFileUnderConstruction child) {
+      final INodeFileUnderConstruction child, final INodeMap inodeMap) {
     Preconditions.checkArgument(!(child instanceof INodeFileUnderConstructionWithSnapshot),
     Preconditions.checkArgument(!(child instanceof INodeFileUnderConstructionWithSnapshot),
         "Child file is already an INodeFileUnderConstructionWithSnapshot, child=" + child);
         "Child file is already an INodeFileUnderConstructionWithSnapshot, child=" + child);
     final INodeFileUnderConstructionWithSnapshot newChild
     final INodeFileUnderConstructionWithSnapshot newChild
         = new INodeFileUnderConstructionWithSnapshot(child, null);
         = new INodeFileUnderConstructionWithSnapshot(child, null);
-    replaceChildFile(child, newChild);
+    replaceChildFile(child, newChild, inodeMap);
     return newChild;
     return newChild;
   }
   }
 
 
   @Override
   @Override
-  public INodeDirectory recordModification(Snapshot latest)
-      throws QuotaExceededException {
-    return isInLatestSnapshot(latest)?
-        replaceSelf4INodeDirectoryWithSnapshot().recordModification(latest)
-        : this;
+  public INodeDirectory recordModification(Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
+          .recordModification(latest, inodeMap);
+    } else {
+      return this;
+    }
   }
   }
 
 
   /**
   /**
@@ -286,12 +300,13 @@ public class INodeDirectory extends INodeWithAdditionalFields {
    * @return the child inode, which may be replaced.
    * @return the child inode, which may be replaced.
    */
    */
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
-      final INode snapshotCopy) throws QuotaExceededException {
+      final INode snapshotCopy, final INodeMap inodeMap)
+      throws QuotaExceededException {
     if (latest == null) {
     if (latest == null) {
       return child;
       return child;
     }
     }
-    return replaceSelf4INodeDirectoryWithSnapshot()
-        .saveChild2Snapshot(child, latest, snapshotCopy);
+    return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
+        .saveChild2Snapshot(child, latest, snapshotCopy, inodeMap);
   }
   }
 
 
   /**
   /**
@@ -378,24 +393,28 @@ public class INodeDirectory extends INodeWithAdditionalFields {
    * @param setModTime set modification time for the parent node
    * @param setModTime set modification time for the parent node
    *                   not needed when replaying the addition and 
    *                   not needed when replaying the addition and 
    *                   the parent already has the proper mod time
    *                   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; 
    * @return false if the child with this name already exists; 
    *         otherwise, return true;
    *         otherwise, return true;
    */
    */
   public boolean addChild(INode node, final boolean setModTime,
   public boolean addChild(INode node, final boolean setModTime,
-      final Snapshot latest) throws QuotaExceededException {
+      final Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
     final int low = searchChildren(node.getLocalNameBytes());
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
     if (low >= 0) {
       return false;
       return false;
     }
     }
 
 
     if (isInLatestSnapshot(latest)) {
     if (isInLatestSnapshot(latest)) {
-      return replaceSelf4INodeDirectoryWithSnapshot()
-          .addChild(node, setModTime, latest);
+      INodeDirectoryWithSnapshot sdir = 
+          replaceSelf4INodeDirectoryWithSnapshot(inodeMap);
+      boolean added = sdir.addChild(node, setModTime, latest, inodeMap);
+      return added;
     }
     }
     addChild(node, low);
     addChild(node, low);
     if (setModTime) {
     if (setModTime) {
       // update modification time of the parent directory
       // update modification time of the parent directory
-      updateModificationTime(node.getModificationTime(), latest);
+      updateModificationTime(node.getModificationTime(), latest, inodeMap);
     }
     }
     return true;
     return true;
   }
   }

+ 17 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.Util;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.Util;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
@@ -155,12 +156,16 @@ public class INodeFile extends INodeWithAdditionalFields implements BlockCollect
   }
   }
 
 
   @Override
   @Override
-  public INodeFile recordModification(final Snapshot latest)
-      throws QuotaExceededException {
-    return isInLatestSnapshot(latest)?
-        getParent().replaceChild4INodeFileWithSnapshot(this)
-            .recordModification(latest)
-        : this;
+  public INodeFile recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      INodeFileWithSnapshot newFile = getParent()
+          .replaceChild4INodeFileWithSnapshot(this, inodeMap)
+          .recordModification(latest, inodeMap);
+      return newFile;
+    } else {
+      return this;
+    }
   }
   }
 
 
   /**
   /**
@@ -179,9 +184,9 @@ public class INodeFile extends INodeWithAdditionalFields implements BlockCollect
    * the {@link FsAction#EXECUTE} action, if any, is ignored.
    * the {@link FsAction#EXECUTE} action, if any, is ignored.
    */
    */
   @Override
   @Override
-  final INode setPermission(FsPermission permission, Snapshot latest)
-      throws QuotaExceededException {
-    return super.setPermission(permission.applyUMask(UMASK), latest);
+  final INode setPermission(FsPermission permission, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    return super.setPermission(permission.applyUMask(UMASK), latest, inodeMap);
   }
   }
 
 
   /** @return the replication factor of the file. */
   /** @return the replication factor of the file. */
@@ -211,9 +216,9 @@ public class INodeFile extends INodeWithAdditionalFields implements BlockCollect
   }
   }
 
 
   /** Set the replication factor of this file. */
   /** Set the replication factor of this file. */
-  public final INodeFile setFileReplication(short replication, Snapshot latest)
-      throws QuotaExceededException {
-    final INodeFile nodeToUpdate = recordModification(latest);
+  public final INodeFile setFileReplication(short replication, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    final INodeFile nodeToUpdate = recordModification(latest, inodeMap);
     nodeToUpdate.setFileReplication(replication);
     nodeToUpdate.setFileReplication(replication);
     return nodeToUpdate;
     return nodeToUpdate;
   }
   }

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -130,12 +131,16 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
   }
   }
   
   
   @Override
   @Override
-  public INodeFileUnderConstruction recordModification(final Snapshot latest)
-      throws QuotaExceededException {
-    return isInLatestSnapshot(latest)?
-        getParent().replaceChild4INodeFileUcWithSnapshot(this)
-            .recordModification(latest)
-        : this;
+  public INodeFileUnderConstruction recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      INodeFileUnderConstructionWithSnapshot newFile = getParent()
+          .replaceChild4INodeFileUcWithSnapshot(this, inodeMap)
+          .recordModification(latest, inodeMap);
+      return newFile;
+    } else {
+      return this;
+    }
   }
   }
 
 
   /** Assert all blocks are complete. */
   /** Assert all blocks are complete. */

+ 137 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java

@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import 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.Content.CountsMap;
+import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.hdfs.util.GSet;
+import org.apache.hadoop.hdfs.util.LightWeightGSet;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Storing all the {@link INode}s and maintaining the mapping between INode ID
+ * and INode.  
+ */
+public class INodeMap {
+  
+  static INodeMap newInstance(INodeDirectory rootDir) {
+    // Compute the map capacity by allocating 1% of total memory
+    int capacity = LightWeightGSet.computeCapacity(1, "INodeMap");
+    GSet<INode, INodeWithAdditionalFields> map
+        = new LightWeightGSet<INode, INodeWithAdditionalFields>(capacity);
+    map.put(rootDir);
+    return new INodeMap(map);
+  }
+  
+  /** Synchronized by external lock. */
+  private GSet<INode, INodeWithAdditionalFields> map;
+  
+  private INodeMap(GSet<INode, INodeWithAdditionalFields> map) {
+    Preconditions.checkArgument(map != null);
+    this.map = map;
+  }
+  
+  /**
+   * Add an {@link INode} into the {@link INode} map. Replace the old value if 
+   * necessary. 
+   * @param inode The {@link INode} to be added to the map.
+   */
+  public final void put(INode inode) {
+    if (inode instanceof INodeWithAdditionalFields) {
+      map.put((INodeWithAdditionalFields)inode);
+    }
+  }
+  
+  /**
+   * Remove a {@link INode} from the map.
+   * @param inode The {@link INode} to be removed.
+   */
+  public final void remove(INode inode) {
+    map.remove(inode);
+  }
+  
+  /**
+   * @return The size of the map.
+   */
+  public int size() {
+    return map.size();
+  }
+  
+  /**
+   * Get the {@link INode} with the given id from the map.
+   * @param id ID of the {@link INode}.
+   * @return The {@link INode} in the map with the given id. Return null if no 
+   *         such {@link INode} in the map.
+   */
+  public INode get(long id) {
+    INode inode = new INodeWithAdditionalFields(id, null, new PermissionStatus(
+        "", "", new FsPermission((short) 0)), 0, 0) {
+      
+      @Override
+      INode recordModification(Snapshot latest, INodeMap inodeMap)
+          throws QuotaExceededException {
+        return null;
+      }
+      
+      @Override
+      public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
+          List<INode> removedINodes) {
+        // Nothing to do
+      }
+      
+      @Override
+      public Counts computeQuotaUsage(Counts counts, boolean useCache,
+          int lastSnapshotId) {
+        return null;
+      }
+      
+      @Override
+      public Content.Counts computeContentSummary(Content.Counts counts) {
+        return null;
+      }
+      
+      @Override
+      public CountsMap computeContentSummary(CountsMap countsMap) {
+        return null;
+      }
+      
+      @Override
+      public Counts cleanSubtree(Snapshot snapshot, Snapshot prior,
+          BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes)
+          throws QuotaExceededException {
+        return null;
+      }
+    };
+      
+    return map.get(inode);
+  }
+  
+  /**
+   * Clear the {@link #map}
+   */
+  public void clear() {
+    map.clear();
+  }
+}

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -197,9 +197,9 @@ public abstract class INodeReference extends INode {
   }
   }
   
   
   @Override
   @Override
-  public final INode updateModificationTime(long mtime, Snapshot latest)
-      throws QuotaExceededException {
-    return referred.updateModificationTime(mtime, latest);
+  public final INode updateModificationTime(long mtime, Snapshot latest,
+      INodeMap inodeMap) throws QuotaExceededException {
+    return referred.updateModificationTime(mtime, latest, inodeMap);
   }
   }
   
   
   @Override
   @Override
@@ -218,8 +218,9 @@ public abstract class INodeReference extends INode {
   }
   }
 
 
   @Override
   @Override
-  final INode recordModification(Snapshot latest) throws QuotaExceededException {
-    referred.recordModification(latest);
+  final INode recordModification(Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    referred.recordModification(latest, inodeMap);
     // reference is never replaced 
     // reference is never replaced 
     return this;
     return this;
   }
   }

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java

@@ -46,10 +46,13 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
   }
 
 
   @Override
   @Override
-  INode recordModification(Snapshot latest) throws QuotaExceededException {
-    return isInLatestSnapshot(latest)?
-        getParent().saveChild2Snapshot(this, latest, new INodeSymlink(this))
-        : this;
+  INode recordModification(Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
+    if (isInLatestSnapshot(latest)) {
+      INodeDirectory parent = getParent();
+      parent.saveChild2Snapshot(this, latest, new INodeSymlink(this), inodeMap);
+    }
+    return this;
   }
   }
 
 
   /** @return true unconditionally. */
   /** @return true unconditionally. */

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

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

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

@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -160,10 +159,6 @@ public class INodesInPath {
                 && curNode.asDirectory() instanceof INodeDirectoryWithSnapshot) {
                 && curNode.asDirectory() instanceof INodeDirectoryWithSnapshot) {
               lastSnapshot = ((INodeDirectoryWithSnapshot) curNode
               lastSnapshot = ((INodeDirectoryWithSnapshot) curNode
                   .asDirectory()).getLastSnapshot();
                   .asDirectory()).getLastSnapshot();
-            } else if (curNode.isFile()
-                && curNode.asFile() instanceof INodeFileWithSnapshot) {
-              lastSnapshot = ((INodeFileWithSnapshot) curNode
-                  .asFile()).getDiffs().getLastSnapshot();
             }
             }
             existing.setSnapshot(lastSnapshot);
             existing.setSnapshot(lastSnapshot);
           }
           }

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

@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 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.Quota;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -274,8 +275,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   }
   }
 
 
   /** Add a snapshot. */
   /** Add a snapshot. */
-  Snapshot addSnapshot(int id, String name)
-      throws SnapshotException, QuotaExceededException {
+  Snapshot addSnapshot(int id, String name) throws SnapshotException,
+      QuotaExceededException {
     //check snapshot quota
     //check snapshot quota
     final int n = getNumSnapshots();
     final int n = getNumSnapshots();
     if (n + 1 > snapshotQuota) {
     if (n + 1 > snapshotQuota) {
@@ -296,8 +297,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
     snapshotsByNames.add(-i - 1, s);
     snapshotsByNames.add(-i - 1, s);
 
 
     //set modification time
     //set modification time
-    updateModificationTime(Time.now(), null);
-    s.getRoot().setModificationTime(getModificationTime(), null);
+    updateModificationTime(Time.now(), null, null);
+    s.getRoot().setModificationTime(getModificationTime(), null, null);
     return s;
     return s;
   }
   }
   
   
@@ -455,13 +456,15 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
    * Replace itself with {@link INodeDirectoryWithSnapshot} or
    * Replace itself with {@link INodeDirectoryWithSnapshot} or
    * {@link INodeDirectory} depending on the latest snapshot.
    * {@link INodeDirectory} depending on the latest snapshot.
    */
    */
-  void replaceSelf(final Snapshot latest) throws QuotaExceededException {
+  INodeDirectory replaceSelf(final Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
     if (latest == null) {
     if (latest == null) {
       Preconditions.checkState(getLastSnapshot() == null,
       Preconditions.checkState(getLastSnapshot() == null,
           "latest == null but getLastSnapshot() != null, this=%s", this);
           "latest == null but getLastSnapshot() != null, this=%s", this);
-      replaceSelf4INodeDirectory();
+      return replaceSelf4INodeDirectory(inodeMap);
     } else {
     } else {
-      replaceSelf4INodeDirectoryWithSnapshot().recordModification(latest);
+      return replaceSelf4INodeDirectoryWithSnapshot(inodeMap)
+          .recordModification(latest, null);
     }
     }
   }
   }
 
 

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

@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
+import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
@@ -634,8 +635,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
   }
 
 
   @Override
   @Override
-  public INodeDirectoryWithSnapshot recordModification(final Snapshot latest)
-      throws QuotaExceededException {
+  public INodeDirectoryWithSnapshot recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
     if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
     if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
       return saveSelf2Snapshot(latest, null);
       return saveSelf2Snapshot(latest, null);
     }
     }
@@ -652,7 +653,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
 
 
   @Override
   @Override
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
   public INode saveChild2Snapshot(final INode child, final Snapshot latest,
-      final INode snapshotCopy) throws QuotaExceededException {
+      final INode snapshotCopy, final INodeMap inodeMap)
+      throws QuotaExceededException {
     Preconditions.checkArgument(!child.isDirectory(),
     Preconditions.checkArgument(!child.isDirectory(),
         "child is a directory, child=%s", child);
         "child is a directory, child=%s", child);
     if (latest == null) {
     if (latest == null) {
@@ -670,15 +672,15 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
   }
 
 
   @Override
   @Override
-  public boolean addChild(INode inode, boolean setModTime, Snapshot latest)
-      throws QuotaExceededException {
+  public boolean addChild(INode inode, boolean setModTime, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
     ChildrenDiff diff = null;
     ChildrenDiff diff = null;
     Integer undoInfo = null;
     Integer undoInfo = null;
     if (latest != null) {
     if (latest != null) {
       diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
       diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
       undoInfo = diff.create(inode);
       undoInfo = diff.create(inode);
     }
     }
-    final boolean added = super.addChild(inode, setModTime, null);
+    final boolean added = super.addChild(inode, setModTime, null, inodeMap);
     if (!added && undoInfo != null) {
     if (!added && undoInfo != null) {
       diff.undoCreate(inode, undoInfo);
       diff.undoCreate(inode, undoInfo);
     }
     }
@@ -686,8 +688,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
   }
 
 
   @Override
   @Override
-  public boolean removeChild(INode child, Snapshot latest)
-      throws QuotaExceededException {
+  public boolean removeChild(INode child, Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
     ChildrenDiff diff = null;
     ChildrenDiff diff = null;
     UndoInfo<INode> undoInfo = null;
     UndoInfo<INode> undoInfo = null;
     if (latest != null) {
     if (latest != null) {
@@ -705,8 +707,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
   }
   
   
   @Override
   @Override
-  public void replaceChild(final INode oldChild, final INode newChild) {
-    super.replaceChild(oldChild, newChild);
+  public void replaceChild(final INode oldChild, final INode newChild,
+      final INodeMap inodeMap) {
+    super.replaceChild(oldChild, newChild, inodeMap);
     diffs.replaceChild(ListType.CREATED, oldChild, newChild);
     diffs.replaceChild(ListType.CREATED, oldChild, newChild);
   }
   }
   
   
@@ -747,7 +750,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       throws QuotaExceededException {
       throws QuotaExceededException {
     diffs.removeChild(ListType.DELETED, oldChild);
     diffs.removeChild(ListType.DELETED, oldChild);
     diffs.replaceChild(ListType.CREATED, oldChild, newChild);
     diffs.replaceChild(ListType.CREATED, oldChild, newChild);
-    addChild(newChild, true, null);
+    // pass null for inodeMap since the parent node will not get replaced when
+    // undoing rename
+    addChild(newChild, true, null, null);
   }
   }
   
   
   /**
   /**
@@ -759,8 +764,10 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       Snapshot latestSnapshot) throws QuotaExceededException {
       Snapshot latestSnapshot) throws QuotaExceededException {
     boolean removeDeletedChild = diffs.removeChild(ListType.DELETED,
     boolean removeDeletedChild = diffs.removeChild(ListType.DELETED,
         deletedChild);
         deletedChild);
+    // pass null for inodeMap since the parent node will not get replaced when
+    // undoing rename
     final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
     final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
-        : latestSnapshot);
+        : latestSnapshot, null);
     // update quota usage if adding is successfully and the old child has not
     // update quota usage if adding is successfully and the old child has not
     // been stored in deleted list before
     // been stored in deleted list before
     if (added && !removeDeletedChild) {
     if (added && !removeDeletedChild) {
@@ -804,7 +811,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       throws QuotaExceededException {
       throws QuotaExceededException {
     Quota.Counts counts = Quota.Counts.newInstance();
     Quota.Counts counts = Quota.Counts.newInstance();
     if (snapshot == null) { // delete the current directory
     if (snapshot == null) { // delete the current directory
-      recordModification(prior);
+      recordModification(prior, null);
       // delete everything in created list
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
       if (lastDiff != null) {

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

@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 
 /**
 /**
@@ -78,7 +79,8 @@ public class INodeFileUnderConstructionWithSnapshot
 
 
   @Override
   @Override
   public INodeFileUnderConstructionWithSnapshot recordModification(
   public INodeFileUnderConstructionWithSnapshot recordModification(
-      final Snapshot latest) throws QuotaExceededException {
+      final Snapshot latest, final INodeMap inodeMap)
+      throws QuotaExceededException {
     if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
     if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
       diffs.saveSelf2Snapshot(latest, this, null);
       diffs.saveSelf2Snapshot(latest, this, null);
     }
     }
@@ -100,7 +102,7 @@ public class INodeFileUnderConstructionWithSnapshot
       final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
       final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
       throws QuotaExceededException {
       throws QuotaExceededException {
     if (snapshot == null) { // delete the current file
     if (snapshot == null) { // delete the current file
-      recordModification(prior);
+      recordModification(prior, null);
       isCurrentFileDeleted = true;
       isCurrentFileDeleted = true;
       Util.collectBlocksAndClear(this, collectedBlocks, removedINodes);
       Util.collectBlocksAndClear(this, collectedBlocks, removedINodes);
       return Quota.Counts.newInstance();
       return Quota.Counts.newInstance();

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

@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 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.Quota;
 
 
 /**
 /**
@@ -65,8 +66,8 @@ public class INodeFileWithSnapshot extends INodeFile
   }
   }
 
 
   @Override
   @Override
-  public INodeFileWithSnapshot recordModification(final Snapshot latest)
-      throws QuotaExceededException {
+  public INodeFileWithSnapshot recordModification(final Snapshot latest,
+      final INodeMap inodeMap) throws QuotaExceededException {
     if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
     if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
       diffs.saveSelf2Snapshot(latest, this, null);
       diffs.saveSelf2Snapshot(latest, this, null);
     }
     }
@@ -88,7 +89,7 @@ public class INodeFileWithSnapshot extends INodeFile
       final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
       final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
       throws QuotaExceededException {
       throws QuotaExceededException {
     if (snapshot == null) { // delete the current file
     if (snapshot == null) { // delete the current file
-      recordModification(prior);
+      recordModification(prior, null);
       isCurrentFileDeleted = true;
       isCurrentFileDeleted = true;
       Util.collectBlocksAndClear(this, collectedBlocks, removedINodes);
       Util.collectBlocksAndClear(this, collectedBlocks, removedINodes);
       return Quota.Counts.newInstance();
       return Quota.Counts.newInstance();

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

@@ -82,7 +82,8 @@ public class SnapshotManager implements SnapshotStats {
       s = (INodeDirectorySnapshottable)d; 
       s = (INodeDirectorySnapshottable)d; 
       s.setSnapshotQuota(INodeDirectorySnapshottable.SNAPSHOT_LIMIT);
       s.setSnapshotQuota(INodeDirectorySnapshottable.SNAPSHOT_LIMIT);
     } else {
     } else {
-      s = d.replaceSelf4INodeDirectorySnapshottable(iip.getLatestSnapshot());
+      s = d.replaceSelf4INodeDirectorySnapshottable(iip.getLatestSnapshot(),
+          fsdir.getINodeMap());
     }
     }
     addSnapshottable(s);
     addSnapshottable(s);
   }
   }
@@ -124,7 +125,7 @@ public class SnapshotManager implements SnapshotStats {
     if (s == fsdir.getRoot()) {
     if (s == fsdir.getRoot()) {
       s.setSnapshotQuota(0); 
       s.setSnapshotQuota(0); 
     } else {
     } else {
-      s.replaceSelf(iip.getLatestSnapshot());
+      s.replaceSelf(iip.getLatestSnapshot(), fsdir.getINodeMap());
     }
     }
     removeSnapshottable(s);
     removeSnapshottable(s);
   }
   }

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

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

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

@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiff;
@@ -1251,9 +1252,9 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory mockDir2 = spy(dir2);
     INodeDirectory mockDir2 = spy(dir2);
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject());
+            (Snapshot) anyObject(), (INodeMap) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
-    root.replaceChild(dir2, mockDir2);
+    root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     
     
     final Path newfoo = new Path(sdir2, "foo");
     final Path newfoo = new Path(sdir2, "foo");
     boolean result = hdfs.rename(foo, newfoo);
     boolean result = hdfs.rename(foo, newfoo);
@@ -1319,9 +1320,9 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory dir2 = fsdir.getINode4Write(sdir2.toString()).asDirectory();
     INodeDirectory mockDir2 = spy(dir2);
     INodeDirectory mockDir2 = spy(dir2);
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
     doReturn(false).when(mockDir2).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject());
+            (Snapshot) anyObject(), (INodeMap) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
-    root.replaceChild(dir2, mockDir2);
+    root.replaceChild(dir2, mockDir2, fsdir.getINodeMap());
     
     
     final Path newfoo = new Path(sdir2, "foo");
     final Path newfoo = new Path(sdir2, "foo");
     boolean result = hdfs.rename(foo, newfoo);
     boolean result = hdfs.rename(foo, newfoo);
@@ -1381,9 +1382,9 @@ public class TestRenameWithSnapshots {
     INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
     INodeDirectory dir3 = fsdir.getINode4Write(sdir3.toString()).asDirectory();
     INodeDirectory mockDir3 = spy(dir3);
     INodeDirectory mockDir3 = spy(dir3);
     doReturn(false).when(mockDir3).addChild((INode) anyObject(), anyBoolean(),
     doReturn(false).when(mockDir3).addChild((INode) anyObject(), anyBoolean(),
-            (Snapshot) anyObject());
+            (Snapshot) anyObject(), (INodeMap) anyObject());
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
-    root.replaceChild(dir3, mockDir3);
+    root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     
     
     final Path foo_dir2 = new Path(sdir2, "foo");
     final Path foo_dir2 = new Path(sdir2, "foo");
     final Path foo_dir3 = new Path(sdir3, "foo");
     final Path foo_dir3 = new Path(sdir3, "foo");
@@ -1483,11 +1484,12 @@ public class TestRenameWithSnapshots {
     INodeDirectory mockDir3 = spy(dir3);
     INodeDirectory mockDir3 = spy(dir3);
     // fail the rename but succeed in undo
     // fail the rename but succeed in undo
     doReturn(false).when(mockDir3).addChild((INode) Mockito.isNull(),
     doReturn(false).when(mockDir3).addChild((INode) Mockito.isNull(),
-        anyBoolean(), (Snapshot) anyObject());
-    Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), anyBoolean(),
-        (Snapshot) anyObject())).thenReturn(false).thenCallRealMethod();
+        anyBoolean(), (Snapshot) anyObject(), (INodeMap) anyObject());
+    Mockito.when(mockDir3.addChild((INode) Mockito.isNotNull(), 
+        anyBoolean(), (Snapshot) anyObject(), 
+        (INodeMap) anyObject())).thenReturn(false).thenCallRealMethod();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
-    root.replaceChild(dir3, mockDir3);
+    root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     foo3Node.setParent(mockDir3);
     foo3Node.setParent(mockDir3);
     
     
     try {
     try {