You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2013/01/31 22:13:04 UTC

svn commit: r1441193 - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/

Author: szetszwo
Date: Thu Jan 31 21:13:04 2013
New Revision: 1441193

URL: http://svn.apache.org/viewvc?rev=1441193&view=rev
Log:
HDFS-4189. Renames the getMutableXxx methods to getXxx4Write and fix a bug that some getExistingPathINodes calls should be getINodesInPath4Write.

Modified:
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt?rev=1441193&r1=1441192&r2=1441193&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt Thu Jan 31 21:13:04 2013
@@ -134,5 +134,9 @@ Branch-2802 Snapshot (Unreleased)
   HDFS-4131. Add capability to namenode to get snapshot diff. (Jing Zhao via
   suresh)
 
-  HDFS-4447. Refactor INodeDirectoryWithSnapshot for support general INode diff
-  lists.  (szetszwo)
+  HDFS-4447. Refactor INodeDirectoryWithSnapshot for supporting general INode
+  diff lists.  (szetszwo)
+
+  HDFS-4189. Renames the getMutableXxx methods to getXxx4Write and fix a bug
+  that some getExistingPathINodes calls should be getINodesInPath4Write.
+  (szetszwo)

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=1441193&r1=1441192&r2=1441193&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Thu Jan 31 21:13:04 2013
@@ -330,12 +330,11 @@ public class FSDirectory implements Clos
 
     writeLock();
     try {
-      final INode[] inodes = inodesInPath.getINodes();
       final INodeFileUnderConstruction fileINode = 
-          INodeFileUnderConstruction.valueOf(inodes[inodes.length-1], path);
+          INodeFileUnderConstruction.valueOf(inodesInPath.getLastINode(), path);
 
       // check quota limits and updated space consumed
-      updateCount(inodesInPath, inodes.length-1, 0,
+      updateCount(inodesInPath, 0,
           fileINode.getPreferredBlockSize()*fileINode.getFileReplication(), true);
 
       // associate new last block for the file
@@ -426,9 +425,8 @@ public class FSDirectory implements Clos
     }
 
     // update space consumed
-    final INodesInPath inodesInPath = rootDir.getExistingPathINodes(path, true);
-    final INode[] inodes = inodesInPath.getINodes();
-    updateCount(inodesInPath, inodes.length-1, 0,
+    final INodesInPath iip = rootDir.getINodesInPath4Write(path, true);
+    updateCount(iip, 0,
         -fileNode.getPreferredBlockSize()*fileNode.getFileReplication(), true);
   }
 
@@ -498,7 +496,7 @@ public class FSDirectory implements Clos
     throws QuotaExceededException, UnresolvedLinkException, 
     FileAlreadyExistsException, SnapshotAccessControlException {
     assert hasWriteLock();
-    INodesInPath srcInodesInPath = rootDir.getMutableINodesInPath(src, false);
+    INodesInPath srcInodesInPath = rootDir.getINodesInPath4Write(src, false);
     INode[] srcInodes = srcInodesInPath.getINodes();
     INode srcInode = srcInodes[srcInodes.length-1];
     
@@ -629,7 +627,7 @@ public class FSDirectory implements Clos
       }
     }
     String error = null;
-    final INodesInPath srcInodesInPath = rootDir.getMutableINodesInPath(src,
+    final INodesInPath srcInodesInPath = rootDir.getINodesInPath4Write(src,
         false);
     final INode[] srcInodes = srcInodesInPath.getINodes();
     final INode srcInode = srcInodes[srcInodes.length - 1];
@@ -666,9 +664,8 @@ public class FSDirectory implements Clos
           + error);
       throw new IOException(error);
     }
-    final byte[][] dstComponents = INode.getPathComponents(dst);
-    final INodesInPath dstInodesInPath = rootDir.getMutableINodesInPath(
-        dstComponents, false);
+    final INodesInPath dstInodesInPath = rootDir.getINodesInPath4Write(
+        dst, false);
     final INode[] dstInodes = dstInodesInPath.getINodes();
     INode dstInode = dstInodes[dstInodes.length - 1];
     if (dstInodes.length == 1) {
@@ -747,7 +744,7 @@ public class FSDirectory implements Clos
         dstChildName = removedDst.getLocalNameBytes();
       }
 
-      removedSrc.setLocalName(dstComponents[dstInodes.length - 1]);
+      removedSrc.setLocalName(dstInodesInPath.getLastLocalName());
       // add src as dst to complete rename
       if (addLastINodeNoQuotaCheck(dstInodesInPath, removedSrc)) {
         removedSrc = null;
@@ -828,7 +825,7 @@ public class FSDirectory implements Clos
       UnresolvedLinkException, SnapshotAccessControlException {
     assert hasWriteLock();
 
-    final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(src, true);
+    final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
     final INode[] inodes = inodesInPath.getINodes();
     INode inode = inodes[inodes.length - 1];
     if (inode == null || !inode.isFile()) {
@@ -839,7 +836,7 @@ public class FSDirectory implements Clos
 
     // check disk quota
     long dsDelta = (replication - oldRepl) * (fileNode.diskspaceConsumed()/oldRepl);
-    updateCount(inodesInPath, inodes.length-1, 0, dsDelta, true);
+    updateCount(inodesInPath, 0, dsDelta, true);
 
     fileNode.setFileReplication(replication, inodesInPath.getLatestSnapshot());
 
@@ -877,23 +874,6 @@ public class FSDirectory implements Clos
       readUnlock();
     }
   }
-
-  boolean existsMutable(String src) throws UnresolvedLinkException,
-      SnapshotAccessControlException {
-    src = normalizePath(src);
-    readLock();
-    try {
-      INode inode = rootDir.getMutableNode(src, false);
-      if (inode == null) {
-         return false;
-      }
-      return inode.isDirectory() || inode.isSymlink() 
-        ? true 
-        : ((INodeFile)inode).getBlocks() != null;
-    } finally {
-      readUnlock();
-    }
-  }
   
   void setPermission(String src, FsPermission permission)
       throws FileNotFoundException, UnresolvedLinkException,
@@ -911,7 +891,7 @@ public class FSDirectory implements Clos
       throws FileNotFoundException, UnresolvedLinkException,
       SnapshotAccessControlException {
     assert hasWriteLock();
-    final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(src, true);
+    final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
     final INode inode = inodesInPath.getLastINode();
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
@@ -935,7 +915,7 @@ public class FSDirectory implements Clos
       throws FileNotFoundException, UnresolvedLinkException,
       SnapshotAccessControlException {
     assert hasWriteLock();
-    final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(src, true);
+    final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
     INode inode = inodesInPath.getLastINode();
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
@@ -952,7 +932,7 @@ public class FSDirectory implements Clos
    * Concat all the blocks from srcs to trg and delete the srcs files
    */
   public void concat(String target, String [] srcs) 
-      throws UnresolvedLinkException {
+      throws UnresolvedLinkException, SnapshotAccessControlException {
     writeLock();
     try {
       // actual move
@@ -976,14 +956,14 @@ public class FSDirectory implements Clos
    * NOTE: - it does not update quota (not needed for concat)
    */
   public void unprotectedConcat(String target, String [] srcs, long timestamp) 
-      throws UnresolvedLinkException {
+      throws UnresolvedLinkException, SnapshotAccessControlException {
     assert hasWriteLock();
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSNamesystem.concat to "+target);
     }
     // do the move
     
-    final INodesInPath trgINodesInPath = rootDir.getExistingPathINodes(target, true);
+    final INodesInPath trgINodesInPath = rootDir.getINodesInPath4Write(target, true);
     final INode[] trgINodes = trgINodesInPath.getINodes();
     INodeFile trgInode = (INodeFile) trgINodes[trgINodes.length-1];
     INodeDirectory trgParent = (INodeDirectory)trgINodes[trgINodes.length-2];
@@ -1032,7 +1012,7 @@ public class FSDirectory implements Clos
     int filesRemoved;
     writeLock();
     try {
-      final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(
+      final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(
           normalizePath(src), false);
       final INode[] inodes = inodesInPath.getINodes();
       if (checkPathINodes(inodes, src) == 0) {
@@ -1093,7 +1073,7 @@ public class FSDirectory implements Clos
   boolean isNonEmptyDirectory(String path) throws UnresolvedLinkException {
     readLock();
     try {
-      final INodesInPath inodesInPath = rootDir.getINodesInPath(path, false);
+      final INodesInPath inodesInPath = rootDir.getLastINodeInPath(path, false);
       final INode inode = inodesInPath.getINode(0);
       if (inode == null || !inode.isDirectory()) {
         //not found or not a directory
@@ -1122,7 +1102,7 @@ public class FSDirectory implements Clos
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     int filesRemoved = 0;
 
-    final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(
+    final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(
         normalizePath(src), false);
     final INode[] inodes = inodesInPath.getINodes();
     if (checkPathINodes(inodes, src) == 0) {
@@ -1255,7 +1235,7 @@ public class FSDirectory implements Clos
 
     readLock();
     try {
-      final INodesInPath inodesInPath = rootDir.getINodesInPath(srcs, true);
+      final INodesInPath inodesInPath = rootDir.getLastINodeInPath(srcs, true);
       final Snapshot snapshot = inodesInPath.getPathSnapshot();
       final INode targetNode = inodesInPath.getINode(0);
       if (targetNode == null)
@@ -1297,7 +1277,7 @@ public class FSDirectory implements Clos
     String srcs = normalizePath(src);
     readLock();
     try {
-      final INodesInPath inodesInPath = rootDir.getINodesInPath(srcs, resolveLink);
+      final INodesInPath inodesInPath = rootDir.getLastINodeInPath(srcs, resolveLink);
       final INode i = inodesInPath.getINode(0);
       return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
           inodesInPath.getPathSnapshot());
@@ -1324,16 +1304,17 @@ public class FSDirectory implements Clos
    * Get {@link INode} associated with the file / directory.
    */
   public INode getINode(String src) throws UnresolvedLinkException {
-    return getINodesInPath(src).getINode(0);
+    return getLastINodeInPath(src).getINode(0);
   }
 
   /**
    * Get {@link INode} associated with the file / directory.
    */
-  public INodesInPath getINodesInPath(String src) throws UnresolvedLinkException {
+  public INodesInPath getLastINodeInPath(String src)
+       throws UnresolvedLinkException {
     readLock();
     try {
-      return rootDir.getINodesInPath(src, true);
+      return rootDir.getLastINodeInPath(src, true);
     } finally {
       readUnlock();
     }
@@ -1342,11 +1323,11 @@ public class FSDirectory implements Clos
   /**
    * Get {@link INode} associated with the file / directory.
    */
-  public INodesInPath getMutableINodesInPath(String src
+  public INodesInPath getINodesInPath4Write(String src
       ) throws UnresolvedLinkException, SnapshotAccessControlException {
     readLock();
     try {
-      return rootDir.getMutableINodesInPath(src, true);
+      return rootDir.getINodesInPath4Write(src, true);
     } finally {
       readUnlock();
     }
@@ -1356,11 +1337,11 @@ public class FSDirectory implements Clos
    * Get {@link INode} associated with the file / directory.
    * @throws SnapshotAccessControlException if path is in RO snapshot
    */
-  public INode getMutableINode(String src) throws UnresolvedLinkException,
+  public INode getINode4Write(String src) throws UnresolvedLinkException,
       SnapshotAccessControlException {
     readLock();
     try {
-      return rootDir.getMutableNode(src, true);
+      return rootDir.getINode4Write(src, true);
     } finally {
       readUnlock();
     }
@@ -1376,7 +1357,7 @@ public class FSDirectory implements Clos
     readLock();
     try {
       if (srcs.startsWith("/") && !srcs.endsWith("/")
-          && rootDir.getMutableNode(srcs, false) == null) {
+          && rootDir.getINode4Write(srcs, false) == null) {
         return true;
       } else {
         return false;
@@ -1409,7 +1390,7 @@ public class FSDirectory implements Clos
     src = normalizePath(src);
     readLock();
     try {
-      INode node = rootDir.getMutableNode(src, false);
+      INode node = rootDir.getINode4Write(src, false);
       return node != null && node.isDirectory();
     } finally {
       readUnlock();
@@ -1426,21 +1407,25 @@ public class FSDirectory implements Clos
    * @throws FileNotFoundException if path does not exist.
    */
   void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
-      throws QuotaExceededException, FileNotFoundException, UnresolvedLinkException {
+      throws QuotaExceededException, FileNotFoundException,
+          UnresolvedLinkException, SnapshotAccessControlException {
     writeLock();
     try {
-      final INodesInPath inodesInPath = rootDir.getExistingPathINodes(path, false);
-      final INode[] inodes = inodesInPath.getINodes();
-      int len = inodes.length;
-      if (inodes[len - 1] == null) {
+      final INodesInPath iip = rootDir.getINodesInPath4Write(path, false);
+      if (iip.getLastINode() == null) {
         throw new FileNotFoundException("Path not found: " + path);
       }
-      updateCount(inodesInPath, len-1, nsDelta, dsDelta, true);
+      updateCount(iip, nsDelta, dsDelta, true);
     } finally {
       writeUnlock();
     }
   }
   
+  private void updateCount(INodesInPath iip, long nsDelta, long dsDelta,
+      boolean checkQuota) throws QuotaExceededException {
+    updateCount(iip, iip.getINodes().length - 1, nsDelta, dsDelta, checkQuota);
+  }
+
   /** update count of each inode with quota
    * 
    * @param inodes an array of inodes on a path
@@ -2011,7 +1996,7 @@ public class FSDirectory implements Clos
     }
     
     String srcs = normalizePath(src);
-    final INodesInPath iip = rootDir.getMutableINodesInPath(srcs, true);
+    final INodesInPath iip = rootDir.getINodesInPath4Write(srcs, true);
     INodeDirectory dirNode = INodeDirectory.valueOf(iip.getLastINode(), srcs);
     if (dirNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
       throw new IllegalArgumentException("Cannot clear namespace quota on root.");
@@ -2091,7 +2076,7 @@ public class FSDirectory implements Clos
   boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force) 
       throws UnresolvedLinkException {
     assert hasWriteLock();
-    final INodesInPath i = getINodesInPath(src); 
+    final INodesInPath i = getLastINodeInPath(src); 
     return unprotectedSetTimes(src, i.getLastINode(), mtime, atime, force,
         i.getLatestSnapshot());
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java?rev=1441193&r1=1441192&r2=1441193&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java Thu Jan 31 21:13:04 2013
@@ -253,7 +253,7 @@ public class FSEditLogLoader {
       // 3. OP_ADD to open file for append
 
       // See if the file already exists (persistBlocks call)
-      final INodesInPath iip = fsDir.getINodesInPath(addCloseOp.path);
+      final INodesInPath iip = fsDir.getLastINodeInPath(addCloseOp.path);
       final INodeFile oldFile = toINodeFile(iip.getINode(0), addCloseOp.path);
       INodeFile newFile = oldFile;
       if (oldFile == null) { // this is OP_ADD on a new file (case 1)
@@ -304,7 +304,7 @@ public class FSEditLogLoader {
             " clientMachine " + addCloseOp.clientMachine);
       }
 
-      final INodesInPath iip = fsDir.getINodesInPath(addCloseOp.path);
+      final INodesInPath iip = fsDir.getLastINodeInPath(addCloseOp.path);
       final INodeFile oldFile = toINodeFile(iip.getINode(0), addCloseOp.path);
       if (oldFile == null) {
         throw new IOException("Operation trying to close non-existent file " +

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java?rev=1441193&r1=1441192&r2=1441193&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java Thu Jan 31 21:13:04 2013
@@ -638,7 +638,7 @@ public class FSImageFormat {
 
         // verify that file exists in namespace
         String path = cons.getLocalName();
-        final INodesInPath iip = fsDir.getINodesInPath(path);
+        final INodesInPath iip = fsDir.getLastINodeInPath(path);
         INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
         cons.setLocalName(oldnode.getLocalNameBytes());
         if (oldnode instanceof FileWithSnapshot

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1441193&r1=1441192&r2=1441193&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Thu Jan 31 21:13:04 2013
@@ -1359,7 +1359,7 @@ public class FSNamesystem implements Nam
           doAccessTime = false;
         }
 
-        final INodesInPath iip = dir.getINodesInPath(src);
+        final INodesInPath iip = dir.getLastINodeInPath(src);
         final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
         if (!iip.isSnapshot() //snapshots are readonly, so don't update atime.
             && doAccessTime && isAccessTimeSupported()) {
@@ -1480,7 +1480,7 @@ public class FSNamesystem implements Nam
     // replication and blocks sizes should be the same for ALL the blocks
 
     // check the target
-    final INodeFile trgInode = INodeFile.valueOf(dir.getMutableINode(target),
+    final INodeFile trgInode = INodeFile.valueOf(dir.getINode4Write(target),
         target);
     if(trgInode.isUnderConstruction()) {
       throw new HadoopIllegalArgumentException("concat: target file "
@@ -1516,7 +1516,7 @@ public class FSNamesystem implements Nam
       if(i==srcs.length-1)
         endSrc=true;
 
-      final INodeFile srcInode = INodeFile.valueOf(dir.getMutableINode(src), src);
+      final INodeFile srcInode = INodeFile.valueOf(dir.getINode4Write(src), src);
       if(src.isEmpty() 
           || srcInode.isUnderConstruction()
           || srcInode.numBlocks() == 0) {
@@ -1599,7 +1599,7 @@ public class FSNamesystem implements Nam
       if (isPermissionEnabled) {
         checkPathAccess(src, FsAction.WRITE);
       }
-      final INodesInPath iip = dir.getMutableINodesInPath(src);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
       final INode inode = iip.getLastINode();
       if (inode != null) {
         dir.setTimes(src, inode, mtime, atime, true, iip.getLatestSnapshot());
@@ -1882,8 +1882,9 @@ public class FSNamesystem implements Nam
     }
 
     // Verify that the destination does not exist as a directory already.
-    boolean pathExists = dir.existsMutable(src);
-    if (pathExists && dir.isDir(src)) {
+    final INodesInPath iip = dir.getINodesInPath4Write(src);
+    final INode myFile = iip.getLastINode();
+    if (myFile != null && myFile.isDirectory()) {
       throw new FileAlreadyExistsException("Cannot create file " + src
           + "; already exists as a directory.");
     }
@@ -1891,7 +1892,7 @@ public class FSNamesystem implements Nam
     boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
     boolean append = flag.contains(CreateFlag.APPEND);
     if (isPermissionEnabled) {
-      if (append || (overwrite && pathExists)) {
+      if (append || (overwrite && myFile != null)) {
         checkPathAccess(src, FsAction.WRITE);
       } else {
         checkAncestorAccess(src, FsAction.WRITE);
@@ -1906,8 +1907,6 @@ public class FSNamesystem implements Nam
       blockManager.verifyReplication(src, replication, clientMachine);
       boolean create = flag.contains(CreateFlag.CREATE);
       
-      final INodesInPath iip = dir.getINodesInPath(src);
-      final INode myFile = iip.getINode(0);
       if (myFile == null) {
         if (!create) {
           throw new FileNotFoundException("failed to overwrite or append to non-existent file "
@@ -2326,7 +2325,7 @@ public class FSNamesystem implements Nam
         throw new SafeModeException("Cannot add block to " + src, safeMode);
       }
 
-      final INodesInPath iip = dir.rootDir.getExistingPathINodes(src, true);
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
       final INodeFileUnderConstruction pendingFile
           = checkLease(src, clientName, iip.getLastINode());
                                                            
@@ -2504,7 +2503,7 @@ public class FSNamesystem implements Nam
       throw new SafeModeException("Cannot complete file " + src, safeMode);
     }
 
-    final INodesInPath iip = dir.getINodesInPath(src);
+    final INodesInPath iip = dir.getLastINodeInPath(src);
     final INodeFileUnderConstruction pendingFile;
     try {
       pendingFile = checkLease(src, holder, iip.getINode(0)); 
@@ -3144,7 +3143,7 @@ public class FSNamesystem implements Nam
     assert !isInSafeMode();
     assert hasWriteLock();
 
-    final INodesInPath iip = dir.getINodesInPath(src);
+    final INodesInPath iip = dir.getLastINodeInPath(src);
     final INodeFileUnderConstruction pendingFile
         = INodeFileUnderConstruction.valueOf(iip.getINode(0), src);
     int nrBlocks = pendingFile.numBlocks();

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java?rev=1441193&r1=1441192&r2=1441193&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java Thu Jan 31 21:13:04 2013
@@ -122,7 +122,7 @@ class FSPermissionChecker {
     }
     // check if (parentAccess != null) && file exists, then check sb
       // Resolve symlinks, the check is performed on the link target.
-      final INodesInPath inodesInPath = root.getExistingPathINodes(path, true); 
+      final INodesInPath inodesInPath = root.getINodesInPath(path, true); 
       final Snapshot snapshot = inodesInPath.getPathSnapshot();
       final INode[] inodes = inodesInPath.getINodes();
       int ancestorIndex = inodes.length - 2;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java?rev=1441193&r1=1441192&r2=1441193&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java Thu Jan 31 21:13:04 2013
@@ -122,10 +122,6 @@ public class INodeDirectory extends INod
     }
     return i;
   }
-  
-  protected INode getExistingChild(int i) {
-    return children.get(i);
-  }
 
   INode removeChild(INode node) {
     assertChildrenNonNull();
@@ -266,15 +262,22 @@ public class INodeDirectory extends INod
   }
 
   /** @return the {@link INodesInPath} containing only the last inode. */
-  INodesInPath getINodesInPath(String path, boolean resolveLink
+  INodesInPath getLastINodeInPath(String path, boolean resolveLink
       ) throws UnresolvedLinkException {
     return getExistingPathINodes(getPathComponents(path), 1, resolveLink);
   }
 
+  /** @return the {@link INodesInPath} containing all inodes in the path. */
+  INodesInPath getINodesInPath(String path, boolean resolveLink
+      ) throws UnresolvedLinkException {
+    final byte[][] components = getPathComponents(path);
+    return getExistingPathINodes(components, components.length, resolveLink);
+  }
+
   /** @return the last inode in the path. */
   INode getNode(String path, boolean resolveLink) 
     throws UnresolvedLinkException {
-    return getINodesInPath(path, resolveLink).getINode(0);
+    return getLastINodeInPath(path, resolveLink).getINode(0);
   }
 
   /**
@@ -283,10 +286,9 @@ public class INodeDirectory extends INod
    * @throws UnresolvedLinkException if symlink can't be resolved
    * @throws SnapshotAccessControlException if path is in RO snapshot
    */
-  INode getMutableNode(String src, boolean resolveLink)
+  INode getINode4Write(String src, boolean resolveLink)
       throws UnresolvedLinkException, SnapshotAccessControlException {
-    INode[] inodes = getMutableINodesInPath(src, resolveLink).getINodes();
-    return inodes[inodes.length - 1];
+    return getINodesInPath4Write(src, resolveLink).getLastINode();
   }
 
   /**
@@ -294,23 +296,14 @@ public class INodeDirectory extends INod
    * @throws UnresolvedLinkException if symlink can't be resolved
    * @throws SnapshotAccessControlException if path is in RO snapshot
    */
-  INodesInPath getMutableINodesInPath(String src, boolean resolveLink)
-      throws UnresolvedLinkException, SnapshotAccessControlException {
-    return getMutableINodesInPath(INode.getPathComponents(src), resolveLink);
-  }
-  
-  /**
-   * @return the INodesInPath of the components in src
-   * @throws UnresolvedLinkException if symlink can't be resolved
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   */
-  INodesInPath getMutableINodesInPath(byte[][] components, boolean resolveLink)
+  INodesInPath getINodesInPath4Write(String src, boolean resolveLink)
       throws UnresolvedLinkException, SnapshotAccessControlException {
+    final byte[][] components = INode.getPathComponents(src);
     INodesInPath inodesInPath = getExistingPathINodes(components,
         components.length, resolveLink);
     if (inodesInPath.isSnapshot()) {
       throw new SnapshotAccessControlException(
-          "Modification on RO snapshot is disallowed");
+          "Modification on a read-only snapshot is disallowed");
     }
     return inodesInPath;
   }
@@ -447,27 +440,6 @@ public class INodeDirectory extends INod
     return pathComponent == null ? false : HdfsConstants.DOT_SNAPSHOT_DIR
         .equalsIgnoreCase(DFSUtil.bytes2String(pathComponent));
   }
-  
-  /**
-   * Retrieve the existing INodes along the given path. The first INode
-   * always exist and is this INode.
-   * 
-   * @param path the path to explore
-   * @param resolveLink indicates whether UnresolvedLinkException should 
-   *        be thrown when the path refers to a symbolic link.
-   * @return INodes array containing the existing INodes in the order they
-   *         appear when following the path from the root INode to the
-   *         deepest INodes. The array size will be the number of expected
-   *         components in the path, and non existing components will be
-   *         filled with null
-   *         
-   * @see #getExistingPathINodes(byte[][], int, boolean)
-   */
-  INodesInPath getExistingPathINodes(String path, boolean resolveLink) 
-    throws UnresolvedLinkException {
-    byte[][] components = getPathComponents(path);
-    return getExistingPathINodes(components, components.length, resolveLink);
-  }
 
   /**
    * Given a child's name, return the index of the next child
@@ -728,6 +700,10 @@ public class INodeDirectory extends INod
     public INode getLastINode() {
       return inodes[inodes.length - 1];
     }
+
+    byte[] getLastLocalName() {
+      return path[path.length - 1];
+    }
     
     /**
      * @return index of the {@link INodeDirectoryWithSnapshot} in

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java?rev=1441193&r1=1441192&r2=1441193&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java Thu Jan 31 21:13:04 2013
@@ -66,7 +66,7 @@ public class SnapshotManager implements 
    * If the path is already a snapshottable directory, update the quota.
    */
   public void setSnapshottable(final String path) throws IOException {
-    final INodesInPath iip = fsdir.getINodesInPath(path);
+    final INodesInPath iip = fsdir.getLastINodeInPath(path);
     final INodeDirectory d = INodeDirectory.valueOf(iip.getINode(0), path);
     if (d.isSnapshottable()) {
       //The directory is already a snapshottable directory.
@@ -88,7 +88,7 @@ public class SnapshotManager implements 
    */
   public void resetSnapshottable(final String path
       ) throws IOException {
-    final INodesInPath iip = fsdir.getINodesInPath(path);
+    final INodesInPath iip = fsdir.getLastINodeInPath(path);
     final INodeDirectorySnapshottable s = INodeDirectorySnapshottable.valueOf(
         iip.getINode(0), path);
     if (s.getNumSnapshots() > 0) {
@@ -117,7 +117,7 @@ public class SnapshotManager implements 
   public void createSnapshot(final String path, final String snapshotName
       ) throws IOException {
     // Find the source root directory path where the snapshot is taken.
-    final INodesInPath i = fsdir.getMutableINodesInPath(path);
+    final INodesInPath i = fsdir.getINodesInPath4Write(path);
     final INodeDirectorySnapshottable srcRoot
         = INodeDirectorySnapshottable.valueOf(i.getLastINode(), path);
     srcRoot.addSnapshot(snapshotCounter, snapshotName);
@@ -137,7 +137,7 @@ public class SnapshotManager implements 
   public void deleteSnapshot(final String path, final String snapshotName,
       BlocksMapUpdateInfo collectedBlocks) throws IOException {
     // parse the path, and check if the path is a snapshot path
-    INodesInPath inodesInPath = fsdir.getMutableINodesInPath(path.toString());
+    INodesInPath inodesInPath = fsdir.getINodesInPath4Write(path.toString());
     // transfer the inode for path to an INodeDirectorySnapshottable.
     // the INodeDirectorySnapshottable#valueOf method will throw Exception 
     // if the path is not for a snapshottable directory
@@ -253,7 +253,7 @@ public class SnapshotManager implements 
 
     // Find the source root directory path where the snapshots were taken.
     // All the check for path has been included in the valueOf method.
-    INodesInPath inodesInPath = fsdir.getMutableINodesInPath(path.toString());
+    INodesInPath inodesInPath = fsdir.getINodesInPath4Write(path.toString());
     final INodeDirectorySnapshottable snapshotRoot = INodeDirectorySnapshottable
         .valueOf(inodesInPath.getLastINode(), path);