You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ka...@apache.org on 2016/10/26 18:30:42 UTC

[16/50] [abbrv] hadoop git commit: HDFS-10997. Reduce number of path resolving methods. Contributed by Daryn Sharp.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index c9b1c76..107d563 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -17,16 +17,19 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.Stack;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.security.AccessControlException;
@@ -42,12 +45,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 class FSPermissionChecker implements AccessControlEnforcer {
   static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
 
-  private static String constructPath(INodeAttributes[] inodes, int end) {
-    byte[][] components = new byte[end+1][];
-    for (int i=0; i <= end; i++) {
-      components[i] = inodes[i].getLocalNameBytes();
-    }
-    return DFSUtil.byteArray2PathString(components);
+  private static String getPath(byte[][] components, int start, int end) {
+    return DFSUtil.byteArray2PathString(components, start, end - start + 1);
   }
 
   /** @return a string for throwing {@link AccessControlException} */
@@ -203,21 +202,27 @@ class FSPermissionChecker implements AccessControlEnforcer {
     for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null;
         ancestorIndex--);
 
-    checkTraverse(inodeAttrs, ancestorIndex);
+    try {
+      checkTraverse(inodeAttrs, inodes, components, ancestorIndex);
+    } catch (UnresolvedPathException | ParentNotDirectoryException ex) {
+      // must tunnel these exceptions out to avoid breaking interface for
+      // external enforcer
+      throw new TraverseAccessControlException(ex);
+    }
 
     final INodeAttributes last = inodeAttrs[inodeAttrs.length - 1];
     if (parentAccess != null && parentAccess.implies(FsAction.WRITE)
         && inodeAttrs.length > 1 && last != null) {
-      checkStickyBit(inodeAttrs, inodeAttrs.length - 2);
+      checkStickyBit(inodeAttrs, components, inodeAttrs.length - 2);
     }
     if (ancestorAccess != null && inodeAttrs.length > 1) {
-      check(inodeAttrs, ancestorIndex, ancestorAccess);
+      check(inodeAttrs, components, ancestorIndex, ancestorAccess);
     }
     if (parentAccess != null && inodeAttrs.length > 1) {
-      check(inodeAttrs, inodeAttrs.length - 2, parentAccess);
+      check(inodeAttrs, components, inodeAttrs.length - 2, parentAccess);
     }
     if (access != null) {
-      check(inodeAttrs, inodeAttrs.length - 1, access);
+      check(inodeAttrs, components, inodeAttrs.length - 1, access);
     }
     if (subAccess != null) {
       INode rawLast = inodes[inodeAttrs.length - 1];
@@ -225,7 +230,7 @@ class FSPermissionChecker implements AccessControlEnforcer {
           snapshotId, subAccess, ignoreEmptyDir);
     }
     if (doCheckOwner) {
-      checkOwner(inodeAttrs, inodeAttrs.length - 1);
+      checkOwner(inodeAttrs, components, inodeAttrs.length - 1);
     }
   }
 
@@ -243,29 +248,27 @@ class FSPermissionChecker implements AccessControlEnforcer {
   }
 
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkOwner(INodeAttributes[] inodes, int i)
+  private void checkOwner(INodeAttributes[] inodes, byte[][] components, int i)
       throws AccessControlException {
     if (getUser().equals(inodes[i].getUserName())) {
       return;
     }
     throw new AccessControlException(
         "Permission denied. user=" + getUser() +
-        " is not the owner of inode=" + constructPath(inodes, i));
+        " is not the owner of inode=" + getPath(components, 0, i));
   }
 
-  /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkTraverse(INodeAttributes[] inodeAttrs, int last)
-      throws AccessControlException {
+  /** Guarded by {@link FSNamesystem#readLock()}
+   * @throws AccessControlException
+   * @throws ParentNotDirectoryException
+   * @throws UnresolvedPathException
+   */
+  private void checkTraverse(INodeAttributes[] inodeAttrs, INode[] inodes,
+      byte[][] components, int last) throws AccessControlException,
+          UnresolvedPathException, ParentNotDirectoryException {
     for (int i=0; i <= last; i++) {
-      INodeAttributes inode = inodeAttrs[i];
-      if (!inode.isDirectory()) {
-        throw new AccessControlException(
-            constructPath(inodeAttrs, i) + " (is not a directory)");
-      }
-      if (!hasPermission(inode, FsAction.EXECUTE)) {
-        throw new AccessControlException(toAccessControlString(
-            inode, constructPath(inodeAttrs, i), FsAction.EXECUTE));
-      }
+      checkIsDirectory(inodes[i], components, i);
+      check(inodeAttrs, components, i, FsAction.EXECUTE);
     }
   }
 
@@ -300,12 +303,12 @@ class FSPermissionChecker implements AccessControlEnforcer {
   }
 
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void check(INodeAttributes[] inodes, int i, FsAction access)
-      throws AccessControlException {
+  private void check(INodeAttributes[] inodes, byte[][] components, int i,
+      FsAction access) throws AccessControlException {
     INodeAttributes inode = (i >= 0) ? inodes[i] : null;
     if (inode != null && !hasPermission(inode, access)) {
       throw new AccessControlException(
-          toAccessControlString(inode, constructPath(inodes, i), access));
+          toAccessControlString(inode, getPath(components, 0, i), access));
     }
   }
 
@@ -415,8 +418,8 @@ class FSPermissionChecker implements AccessControlEnforcer {
   }
 
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkStickyBit(INodeAttributes[] inodes, int index)
-      throws AccessControlException {
+  private void checkStickyBit(INodeAttributes[] inodes, byte[][] components,
+      int index) throws AccessControlException {
     INodeAttributes parent = inodes[index];
     if (!parent.getFsPermission().getStickyBit()) {
       return;
@@ -436,10 +439,10 @@ class FSPermissionChecker implements AccessControlEnforcer {
     throw new AccessControlException(String.format(
         "Permission denied by sticky bit: user=%s, path=\"%s\":%s:%s:%s%s, " +
         "parent=\"%s\":%s:%s:%s%s", user,
-        constructPath(inodes, index + 1),
+        getPath(components, 0, index + 1),
         inode.getUserName(), inode.getGroupName(),
         inode.isDirectory() ? "d" : "-", inode.getFsPermission().toString(),
-        constructPath(inodes, index),
+        getPath(components, 0, index),
         parent.getUserName(), parent.getGroupName(),
         parent.isDirectory() ? "d" : "-", parent.getFsPermission().toString()));
   }
@@ -472,4 +475,100 @@ class FSPermissionChecker implements AccessControlEnforcer {
         + pool.getPoolName() + ": user " + getUser() + " does not have "
         + access.toString() + " permissions.");
   }
+
+  /**
+   * Verifies that all existing ancestors are directories.  If a permission
+   * checker is provided then the user must have exec access.  Ancestor
+   * symlinks will throw an unresolved exception, and resolveLink determines
+   * if the last inode will throw an unresolved exception.  This method
+   * should always be called after a path is resolved into an IIP.
+   * @param pc for permission checker, null for no checking
+   * @param iip path to verify
+   * @param resolveLink whether last inode may be a symlink
+   * @throws AccessControlException
+   * @throws UnresolvedPathException
+   * @throws ParentNotDirectoryException
+   */
+  static void checkTraverse(FSPermissionChecker pc, INodesInPath iip,
+      boolean resolveLink) throws AccessControlException,
+          UnresolvedPathException, ParentNotDirectoryException {
+    try {
+      if (pc == null || pc.isSuperUser()) {
+        checkSimpleTraverse(iip);
+      } else {
+        pc.checkPermission(iip, false, null, null, null, null, false);
+      }
+    } catch (TraverseAccessControlException tace) {
+      // unwrap the non-ACE (unresolved, parent not dir) exception
+      // tunneled out of checker.
+      tace.throwCause();
+    }
+    // maybe check that the last inode is a symlink
+    if (resolveLink) {
+      int last = iip.length() - 1;
+      checkNotSymlink(iip.getINode(last), iip.getPathComponents(), last);
+    }
+  }
+
+  // rudimentary permission-less directory check
+  private static void checkSimpleTraverse(INodesInPath iip)
+      throws UnresolvedPathException, ParentNotDirectoryException {
+    byte[][] components = iip.getPathComponents();
+    for (int i=0; i < iip.length() - 1; i++) {
+      INode inode = iip.getINode(i);
+      if (inode == null) {
+        break;
+      }
+      checkIsDirectory(inode, components, i);
+    }
+  }
+
+  private static void checkIsDirectory(INode inode, byte[][] components, int i)
+      throws UnresolvedPathException, ParentNotDirectoryException {
+    if (inode != null && !inode.isDirectory()) {
+      checkNotSymlink(inode, components, i);
+      throw new ParentNotDirectoryException(
+          getPath(components, 0, i) + " (is not a directory)");
+    }
+  }
+
+  private static void checkNotSymlink(INode inode, byte[][] components, int i)
+      throws UnresolvedPathException {
+    if (inode != null && inode.isSymlink()) {
+      final int last = components.length - 1;
+      final String path = getPath(components, 0, last);
+      final String preceding = getPath(components, 0, i - 1);
+      final String remainder = getPath(components, i + 1, last);
+      final String target = inode.asSymlink().getSymlinkString();
+      if (LOG.isDebugEnabled()) {
+        final String link = inode.getLocalName();
+        LOG.debug("UnresolvedPathException " +
+            " path: " + path + " preceding: " + preceding +
+            " count: " + i + " link: " + link + " target: " + target +
+            " remainder: " + remainder);
+      }
+      throw new UnresolvedPathException(path, preceding, remainder, target);
+    }
+  }
+
+  //used to tunnel non-ACE exceptions encountered during path traversal.
+  //ops that create inodes are expected to throw ParentNotDirectoryExceptions.
+  //the signature of other methods requires the PNDE to be thrown as an ACE.
+  @SuppressWarnings("serial")
+  static class TraverseAccessControlException extends AccessControlException {
+    TraverseAccessControlException(IOException ioe) {
+      super(ioe);
+    }
+    public void throwCause() throws UnresolvedPathException,
+        ParentNotDirectoryException, AccessControlException {
+      Throwable ioe = getCause();
+      if (ioe instanceof UnresolvedPathException) {
+        throw (UnresolvedPathException)ioe;
+      }
+      if (ioe instanceof ParentNotDirectoryException) {
+        throw (ParentNotDirectoryException)ioe;
+      }
+      throw this;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index f05fa37..b37321d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -24,11 +24,8 @@ import java.util.NoSuchElementException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -77,34 +74,12 @@ public class INodesInPath {
   }
 
   /**
-   * Given some components, create a path name.
-   * @param components The path components
-   * @param start index
-   * @param end index
-   * @return concatenated path
-   */
-  private static String constructPath(byte[][] components, int start, int end) {
-    StringBuilder buf = new StringBuilder();
-    for (int i = start; i < end; i++) {
-      buf.append(DFSUtil.bytes2String(components[i]));
-      if (i < end - 1) {
-        buf.append(Path.SEPARATOR);
-      }
-    }
-    return buf.toString();
-  }
-
-  /**
-   * Retrieve existing INodes from a path. For non-snapshot path,
-   * the number of INodes is equal to the number of path components. For
-   * snapshot path (e.g., /foo/.snapshot/s1/bar), the number of INodes is
-   * (number_of_path_components - 1).
-   * 
-   * An UnresolvedPathException is always thrown when an intermediate path 
-   * component refers to a symbolic link. If the final path component refers 
-   * to a symbolic link then an UnresolvedPathException is only thrown if
-   * resolveLink is true.  
-   * 
+   * Retrieve existing INodes from a path.  The number of INodes is equal
+   * to the number of path components.  For a snapshot path
+   * (e.g. /foo/.snapshot/s1/bar), the ".snapshot/s1" will be represented in
+   * one path component corresponding to its Snapshot.Root inode.  This 1-1
+   * mapping ensures the path can always be properly reconstructed.
+   *
    * <p>
    * Example: <br>
    * Given the path /c1/c2/c3 where only /c1/c2 exists, resulting in the
@@ -118,19 +93,15 @@ public class INodesInPath {
    * 
    * @param startingDir the starting directory
    * @param components array of path component name
-   * @param resolveLink indicates whether UnresolvedLinkException should
-   *        be thrown when the path refers to a symbolic link.
    * @return the specified number of existing INodes in the path
    */
   static INodesInPath resolve(final INodeDirectory startingDir,
-      final byte[][] components, final boolean resolveLink)
-      throws UnresolvedLinkException {
-    return resolve(startingDir, components, false, resolveLink);
+      final byte[][] components) {
+    return resolve(startingDir, components, false);
   }
 
   static INodesInPath resolve(final INodeDirectory startingDir,
-      final byte[][] components, final boolean isRaw,
-      final boolean resolveLink) throws UnresolvedLinkException {
+      byte[][] components, final boolean isRaw) {
     Preconditions.checkArgument(startingDir.compareTo(components[0]) == 0);
 
     INode curNode = startingDir;
@@ -179,30 +150,13 @@ public class INodesInPath {
           }
         }
       }
-      if (curNode.isSymlink() && (!lastComp || resolveLink)) {
-        final String path = constructPath(components, 0, components.length);
-        final String preceding = constructPath(components, 0, count);
-        final String remainder =
-          constructPath(components, count + 1, components.length);
-        final String link = DFSUtil.bytes2String(components[count]);
-        final String target = curNode.asSymlink().getSymlinkString();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("UnresolvedPathException " +
-            " path: " + path + " preceding: " + preceding +
-            " count: " + count + " link: " + link + " target: " + target +
-            " remainder: " + remainder);
-        }
-        throw new UnresolvedPathException(path, preceding, remainder, target);
-      }
       if (lastComp || !isDir) {
         break;
       }
-      final byte[] childName = components[count + 1];
-      
+
+      final byte[] childName = components[++count];
       // check if the next byte[] in components is for ".snapshot"
       if (isDotSnapshotDir(childName) && dir.isSnapshottable()) {
-        // skip the ".snapshot" in components
-        count++;
         isSnapshot = true;
         // check if ".snapshot" is the last element of components
         if (count == components.length - 1) {
@@ -216,19 +170,25 @@ public class INodesInPath {
           curNode = s.getRoot();
           snapshotId = s.getId();
         }
+        // combine .snapshot & name into 1 component element to ensure
+        // 1-to-1 correspondence between components and inodes arrays is
+        // preserved so a path can be reconstructed.
+        byte[][] componentsCopy =
+            Arrays.copyOf(components, components.length - 1);
+        componentsCopy[count] = DFSUtil.string2Bytes(
+            DFSUtil.byteArray2PathString(components, count, 2));
+        // shift the remaining components after snapshot name
+        int start = count + 2;
+        System.arraycopy(components, start, componentsCopy, count + 1,
+            components.length - start);
+        components = componentsCopy;
+        // reduce the inodes array to compensate for reduction in components
+        inodes = Arrays.copyOf(inodes, components.length);
       } else {
         // normal case, and also for resolving file/dir under snapshot root
         curNode = dir.getChild(childName,
             isSnapshot ? snapshotId : CURRENT_STATE_ID);
       }
-      count++;
-    }
-    if (isSnapshot && !isDotSnapshotDir(components[components.length - 1])) {
-      // for snapshot path shrink the inode array. however, for path ending with
-      // .snapshot, still keep last the null inode in the array
-      INode[] newNodes = new INode[components.length - 1];
-      System.arraycopy(inodes, 0, newNodes, 0, newNodes.length);
-      inodes = newNodes;
     }
     return new INodesInPath(inodes, components, isRaw, isSnapshot, snapshotId);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index c738d64..8ad7824 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -108,7 +109,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    */
   public void setSnapshottable(final String path, boolean checkNestedSnapshottable)
       throws IOException {
-    final INodesInPath iip = fsdir.getINodesInPath4Write(path);
+    final INodesInPath iip = fsdir.getINodesInPath(path, DirOp.WRITE);
     final INodeDirectory d = INodeDirectory.valueOf(iip.getLastINode(), path);
     if (checkNestedSnapshottable) {
       checkNestedSnapshottable(d, path);
@@ -149,7 +150,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    * @throws SnapshotException if there are snapshots in the directory.
    */
   public void resetSnapshottable(final String path) throws IOException {
-    final INodesInPath iip = fsdir.getINodesInPath4Write(path);
+    final INodesInPath iip = fsdir.getINodesInPath(path, DirOp.WRITE);
     final INodeDirectory d = INodeDirectory.valueOf(iip.getLastINode(), path);
     DirectorySnapshottableFeature sf = d.getDirectorySnapshottableFeature();
     if (sf == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
index dcb7af9..f115859 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
@@ -117,8 +117,8 @@ public class TestFileStatus {
       dfsClient.getFileInfo("non-absolute");
       fail("getFileInfo for a non-absolute path did not throw IOException");
     } catch (RemoteException re) {
-      assertTrue("Wrong exception for invalid file name", 
-          re.toString().contains("Invalid file name"));
+      assertTrue("Wrong exception for invalid file name: "+re,
+          re.toString().contains("Absolute path required"));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java
index 5416739..3f57dcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -112,11 +113,11 @@ public class TestReservedRawPaths {
     FSDirectory fsd = cluster.getNamesystem().getFSDirectory();
     final String path = "/path";
 
-    INodesInPath iip = fsd.resolvePath(null, path);
+    INodesInPath iip = fsd.resolvePath(null, path, DirOp.READ);
     assertFalse(iip.isRaw());
     assertEquals(path, iip.getPath());
 
-    iip = fsd.resolvePath(null, "/.reserved/raw" + path);
+    iip = fsd.resolvePath(null, "/.reserved/raw" + path, DirOp.READ);
     assertTrue(iip.isRaw());
     assertEquals(path, iip.getPath());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
index 71614f6..9f5528e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -862,8 +863,8 @@ public abstract class FSAclBaseTest {
     fs.setPermission(path,
       new FsPermissionExtension(FsPermission.
           createImmutable((short)0755), true, true));
-    INode inode = cluster.getNamesystem().getFSDirectory().getINode(
-        path.toUri().getPath(), false);
+    INode inode = cluster.getNamesystem().getFSDirectory()
+        .getINode(path.toUri().getPath(), DirOp.READ_LINK);
     assertNotNull(inode);
     FsPermission perm = inode.getFsPermission();
     assertNotNull(perm);
@@ -1764,7 +1765,7 @@ public abstract class FSAclBaseTest {
   public static AclFeature getAclFeature(Path pathToCheck,
       MiniDFSCluster cluster) throws IOException {
     INode inode = cluster.getNamesystem().getFSDirectory()
-        .getINode(pathToCheck.toUri().getPath(), false);
+        .getINode(pathToCheck.toUri().getPath(), DirOp.READ_LINK);
     assertNotNull(inode);
     AclFeature aclFeature = inode.getAclFeature();
     return aclFeature;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index 1ae9fb2..ed6c92a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
@@ -143,9 +144,11 @@ public class NameNodeAdapter {
     final FSNamesystem fsn = nn.getNamesystem();
     INode inode;
     try {
-      inode = fsn.getFSDirectory().getINode(path, false);
+      inode = fsn.getFSDirectory().getINode(path, DirOp.READ);
     } catch (UnresolvedLinkException e) {
       throw new RuntimeException("Lease manager should not support symlinks");
+    } catch (IOException ioe) {
+      return null; // unresolvable path, ex. parent dir is a file
     }
     return inode == null ? null : fsn.leaseManager.getLease((INodeFile) inode);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
index 071bdf7..c35c95a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -395,16 +397,16 @@ public class TestFSDirectory {
     hdfs.createNewFile(new Path("/dir1/file"));
     hdfs.createNewFile(new Path("/dir1/dir2/file"));
 
-    INodesInPath iip = fsdir.resolvePath(null, "/");
+    INodesInPath iip = fsdir.resolvePath(null, "/", DirOp.READ);
     fsdir.verifyParentDir(iip);
 
-    iip = fsdir.resolvePath(null, "/dir1");
+    iip = fsdir.resolvePath(null, "/dir1", DirOp.READ);
     fsdir.verifyParentDir(iip);
 
-    iip = fsdir.resolvePath(null, "/dir1/file");
+    iip = fsdir.resolvePath(null, "/dir1/file", DirOp.READ);
     fsdir.verifyParentDir(iip);
 
-    iip = fsdir.resolvePath(null, "/dir-nonexist/file");
+    iip = fsdir.resolvePath(null, "/dir-nonexist/file", DirOp.READ);
     try {
       fsdir.verifyParentDir(iip);
       fail("expected FNF");
@@ -412,13 +414,13 @@ public class TestFSDirectory {
       // expected.
     }
 
-    iip = fsdir.resolvePath(null, "/dir1/dir2");
+    iip = fsdir.resolvePath(null, "/dir1/dir2", DirOp.READ);
     fsdir.verifyParentDir(iip);
 
-    iip = fsdir.resolvePath(null, "/dir1/dir2/file");
+    iip = fsdir.resolvePath(null, "/dir1/dir2/file", DirOp.READ);
     fsdir.verifyParentDir(iip);
 
-    iip = fsdir.resolvePath(null, "/dir1/dir-nonexist/file");
+    iip = fsdir.resolvePath(null, "/dir1/dir-nonexist/file", DirOp.READ);
     try {
       fsdir.verifyParentDir(iip);
       fail("expected FNF");
@@ -426,12 +428,23 @@ public class TestFSDirectory {
       // expected.
     }
 
-    iip = fsdir.resolvePath(null, "/dir1/file/fail");
     try {
-      fsdir.verifyParentDir(iip);
-      fail("expected FNF");
-    } catch (ParentNotDirectoryException pnd) {
-      // expected.
+      iip = fsdir.resolvePath(null, "/dir1/file/fail", DirOp.READ);
+      fail("expected ACE");
+    } catch (AccessControlException ace) {
+      assertTrue(ace.getMessage().contains("is not a directory"));
+    }
+    try {
+      iip = fsdir.resolvePath(null, "/dir1/file/fail", DirOp.WRITE);
+      fail("expected ACE");
+    } catch (AccessControlException ace) {
+      assertTrue(ace.getMessage().contains("is not a directory"));
+    }
+    try {
+      iip = fsdir.resolvePath(null, "/dir1/file/fail", DirOp.CREATE);
+      fail("expected PNDE");
+    } catch (ParentNotDirectoryException pnde) {
+      assertTrue(pnde.getMessage().contains("is not a directory"));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
index 9903906..53fb97d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -403,7 +404,7 @@ public class TestFSPermissionChecker {
 
   private void assertPermissionGranted(UserGroupInformation user, String path,
       FsAction access) throws IOException {
-    INodesInPath iip = dir.getINodesInPath(path, true);
+    INodesInPath iip = dir.getINodesInPath(path, DirOp.READ);
     dir.getPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(iip,
       false, null, null, access, null, false);
   }
@@ -411,7 +412,7 @@ public class TestFSPermissionChecker {
   private void assertPermissionDenied(UserGroupInformation user, String path,
       FsAction access) throws IOException {
     try {
-      INodesInPath iip = dir.getINodesInPath(path, true);
+      INodesInPath iip = dir.getINodesInPath(path, DirOp.READ);
       dir.getPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(iip,
         false, null, null, access, null, false);
       fail("expected AccessControlException for user + " + user + ", path = " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index dd4dfbd..d203413 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
@@ -1008,7 +1009,7 @@ public class TestFileTruncate {
     byte[] contents = AppendTestUtil.initBuffer(BLOCK_SIZE);
     writeContents(contents, BLOCK_SIZE, srcPath);
 
-    INodesInPath iip = fsn.getFSDirectory().getINodesInPath4Write(src, true);
+    INodesInPath iip = fsn.getFSDirectory().getINodesInPath(src, DirOp.WRITE);
     INodeFile file = iip.getLastINode().asFile();
     long initialGenStamp = file.getLastBlock().getGenerationStamp();
     // Test that prepareFileForTruncate sets up in-place truncate.
@@ -1039,7 +1040,7 @@ public class TestFileTruncate {
     writeContents(contents, BLOCK_SIZE, srcPath);
     fs.allowSnapshot(parent);
     fs.createSnapshot(parent, "ss0");
-    iip = fsn.getFSDirectory().getINodesInPath(src, true);
+    iip = fsn.getFSDirectory().getINodesInPath(src, DirOp.WRITE);
     file = iip.getLastINode().asFile();
     file.recordModification(iip.getLatestSnapshotId(), true);
     assertThat(file.isBlockInLatestSnapshot(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 12ae858..81a25fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.mock;
@@ -103,6 +104,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ReplicationResult;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ErasureCodingResult;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSck;
@@ -971,7 +973,7 @@ public class TestFsck {
 
     // intentionally corrupt NN data structure
     INodeFile node = (INodeFile) cluster.getNamesystem().dir.getINode(
-        fileName, true);
+        fileName, DirOp.READ);
     final BlockInfo[] blocks = node.getBlocks();
     assertEquals(blocks.length, 1);
     blocks[0].setNumBytes(-1L);  // set the block length to be negative
@@ -1224,7 +1226,7 @@ public class TestFsck {
     when(fsName.getBlockManager()).thenReturn(blockManager);
     when(fsName.getFSDirectory()).thenReturn(fsd);
     when(fsd.getFSNamesystem()).thenReturn(fsName);
-    when(fsd.resolvePath(anyObject(), anyString())).thenReturn(iip);
+    when(fsd.resolvePath(anyObject(), anyString(), any(DirOp.class))).thenReturn(iip);
     when(blockManager.getDatanodeManager()).thenReturn(dnManager);
 
     NamenodeFsck fsck = new NamenodeFsck(conf, namenode, nettop, pmap, out,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
index 0eb7132..214c9a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -68,7 +69,7 @@ public class TestGetBlockLocations {
 
       @Override
       public Void answer(InvocationOnMock invocation) throws Throwable {
-        INodesInPath iip = fsd.getINodesInPath(FILE_PATH, true);
+        INodesInPath iip = fsd.getINodesInPath(FILE_PATH, DirOp.READ);
         FSDirDeleteOp.delete(fsd, iip, new INode.BlocksMapUpdateInfo(),
                              new ArrayList<INode>(), new ArrayList<Long>(),
                              now());
@@ -119,7 +120,7 @@ public class TestGetBlockLocations {
     final FSNamesystem fsn = new FSNamesystem(conf, image, true);
 
     final FSDirectory fsd = fsn.getFSDirectory();
-    INodesInPath iip = fsd.getINodesInPath("/", true);
+    INodesInPath iip = fsd.getINodesInPath("/", DirOp.READ);
     PermissionStatus perm = new PermissionStatus(
         "hdfs", "supergroup",
         FsPermission.createImmutable((short) 0x1ff));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
index 07f01d0..d1d915e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
@@ -200,6 +200,11 @@ public class TestSnapshotPathINodes {
     // SnapshotRootIndex should be 3: {root, Testsnapshot, sub1, s1, file1}
     final Snapshot snapshot = getSnapshot(nodesInPath, "s1", 3);
     assertSnapshot(nodesInPath, true, snapshot, 3);
+    assertEquals(".snapshot/s1",
+        DFSUtil.bytes2String(nodesInPath.getPathComponent(3)));
+    assertTrue(nodesInPath.getINode(3) instanceof Snapshot.Root);
+    assertEquals("s1", nodesInPath.getINode(3).getLocalName());
+
     // Check the INode for file1 (snapshot file)
     INode snapshotFileNode = nodesInPath.getLastINode();
     assertINodeFile(snapshotFileNode, file1);
@@ -219,6 +224,9 @@ public class TestSnapshotPathINodes {
     // The number of INodes returned should still be components.length
     // since we put a null in the inode array for ".snapshot"
     assertEquals(nodesInPath.length(), components.length);
+    assertEquals(".snapshot",
+        DFSUtil.bytes2String(nodesInPath.getLastLocalName()));
+    assertNull(nodesInPath.getLastINode());
     // ensure parent inodes can strip the .snapshot
     assertEquals(sub1.toString(),
         nodesInPath.getParentINodesInPath().getPath());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
index 8b19b6d..87fb54e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
@@ -469,7 +469,13 @@ public class SnapshotTestHelper {
   public static void dumpTree(String message, MiniDFSCluster cluster
       ) throws UnresolvedLinkException {
     System.out.println("XXX " + message);
-    cluster.getNameNode().getNamesystem().getFSDirectory().getINode("/"
-        ).dumpTreeRecursively(System.out);
+    try {
+      cluster.getNameNode().getNamesystem().getFSDirectory().getINode("/"
+          ).dumpTreeRecursively(System.out);
+    } catch (UnresolvedLinkException ule) {
+      throw ule;
+    } catch (IOException ioe) {
+      throw new RuntimeException(ioe);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java
index d073228..20cb270 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -146,7 +147,7 @@ public class TestSnapshotReplication {
     }
     // Then check replication for every snapshot
     for (Path ss : snapshotRepMap.keySet()) {
-      final INodesInPath iip = fsdir.getINodesInPath(ss.toString(), true);
+      final INodesInPath iip = fsdir.getINodesInPath(ss.toString(), DirOp.READ);
       final INodeFile ssInode = iip.getLastINode().asFile();
       // The replication number derived from the
       // INodeFileWithLink#getPreferredBlockReplication should

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d175853/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermissionSymlinks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermissionSymlinks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermissionSymlinks.java
index bc41edc..7bd29d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermissionSymlinks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermissionSymlinks.java
@@ -27,7 +27,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.io.FileNotFoundException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 
@@ -424,8 +423,12 @@ public class TestPermissionSymlinks {
     try {
       myfc.access(badPath, FsAction.READ);
       fail("The access call should have failed");
-    } catch (FileNotFoundException e) {
+    } catch (AccessControlException ace) {
       // expected
+      String message = ace.getMessage();
+      assertTrue(message, message.contains("is not a directory"));
+      assertTrue(message.contains(target.toString()));
+      assertFalse(message.contains(badPath.toString()));
     }
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org