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 wa...@apache.org on 2014/07/29 23:11:26 UTC

svn commit: r1614490 - in /hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/test/java/org/apache/hadoop/hdfs/

Author: wang
Date: Tue Jul 29 21:11:26 2014
New Revision: 1614490

URL: http://svn.apache.org/r1614490
Log:
HDFS-6509. Create a special /.reserved/raw directory for raw access to encrypted data. Contributed by Charles Lamb.

Modified:
    hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt
    hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
    hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
    hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt?rev=1614490&r1=1614489&r2=1614490&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt Tue Jul 29 21:11:26 2014
@@ -62,6 +62,9 @@ fs-encryption (Unreleased)
     HDFS-6724. Decrypt EDEK before creating
     CryptoInputStream/CryptoOutputStream. (wang)
 
+    HDFS-6509. Create a special /.reserved/raw directory for raw access to
+    encrypted data. (clamb via wang)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/branches/fs-encryption/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/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=1614490&r1=1614489&r2=1614490&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Tue Jul 29 21:11:26 2014
@@ -120,6 +120,8 @@ public class FSDirectory implements Clos
       + DOT_RESERVED_STRING;
   public final static byte[] DOT_RESERVED = 
       DFSUtil.string2Bytes(DOT_RESERVED_STRING);
+  private final static String RAW_STRING = "raw";
+  private final static byte[] RAW = DFSUtil.string2Bytes(RAW_STRING);
   public final static String DOT_INODES_STRING = ".inodes";
   public final static byte[] DOT_INODES = 
       DFSUtil.string2Bytes(DOT_INODES_STRING);
@@ -1315,6 +1317,7 @@ public class FSDirectory implements Clos
   DirectoryListing getListing(String src, byte[] startAfter,
       boolean needLocation) throws UnresolvedLinkException, IOException {
     String srcs = normalizePath(src);
+    final boolean isRawPath = isReservedRawName(src);
 
     readLock();
     try {
@@ -1330,7 +1333,7 @@ public class FSDirectory implements Clos
       if (!targetNode.isDirectory()) {
         return new DirectoryListing(
             new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME,
-                targetNode, needLocation, snapshot)}, 0);
+                targetNode, needLocation, snapshot, isRawPath)}, 0);
       }
 
       final INodeDirectory dirInode = targetNode.asDirectory();
@@ -1344,7 +1347,7 @@ public class FSDirectory implements Clos
       for (int i=0; i<numOfListing && locationBudget>0; i++) {
         INode cur = contents.get(startChild+i);
         listing[i] = createFileStatus(cur.getLocalNameBytes(), cur,
-            needLocation, snapshot);
+            needLocation, snapshot, isRawPath);
         listingCnt++;
         if (needLocation) {
             // Once we  hit lsLimit locations, stop.
@@ -1395,7 +1398,7 @@ public class FSDirectory implements Clos
     for (int i = 0; i < numOfListing; i++) {
       Root sRoot = snapshots.get(i + skipSize).getRoot();
       listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot,
-          Snapshot.CURRENT_STATE_ID);
+          Snapshot.CURRENT_STATE_ID, false);
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);
@@ -1403,12 +1406,13 @@ public class FSDirectory implements Clos
 
   /** Get the file info for a specific file.
    * @param src The string representation of the path to the file
-   * @param resolveLink whether to throw UnresolvedLinkException 
+   * @param resolveLink whether to throw UnresolvedLinkException
+   * @param isRawPath true if a /.reserved/raw pathname was passed by the user
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  HdfsFileStatus getFileInfo(String src, boolean resolveLink)
-    throws UnresolvedLinkException, IOException {
+  HdfsFileStatus getFileInfo(String src, boolean resolveLink, boolean isRawPath)
+    throws IOException {
     String srcs = normalizePath(src);
     readLock();
     try {
@@ -1418,9 +1422,8 @@ public class FSDirectory implements Clos
       final INodesInPath inodesInPath = getLastINodeInPath(srcs, resolveLink);
       final INode i = inodesInPath.getINode(0);
 
-      final int snapshotId = inodesInPath.getPathSnapshotId();
       return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
-          inodesInPath.getPathSnapshotId());
+          inodesInPath.getPathSnapshotId(), isRawPath);
     } finally {
       readUnlock();
     }
@@ -2259,22 +2262,25 @@ public class FSDirectory implements Clos
    * @param path the local name
    * @param node inode
    * @param needLocation if block locations need to be included or not
+   * @param isRawPath true if this is being called on behalf of a path in
+   *                  /.reserved/raw
    * @return a file status
    * @throws IOException if any error occurs
    */
   private HdfsFileStatus createFileStatus(byte[] path, INode node,
-      boolean needLocation, int snapshot) throws IOException {
+      boolean needLocation, int snapshot, boolean isRawPath)
+      throws IOException {
     if (needLocation) {
-      return createLocatedFileStatus(path, node, snapshot);
+      return createLocatedFileStatus(path, node, snapshot, isRawPath);
     } else {
-      return createFileStatus(path, node, snapshot);
+      return createFileStatus(path, node, snapshot, isRawPath);
     }
   }
   /**
    * Create FileStatus by file INode 
    */
    HdfsFileStatus createFileStatus(byte[] path, INode node,
-       int snapshot) throws IOException {
+       int snapshot, boolean isRawPath) throws IOException {
      long size = 0;     // length is zero for directories
      short replication = 0;
      long blocksize = 0;
@@ -2287,7 +2293,8 @@ public class FSDirectory implements Clos
      int childrenNum = node.isDirectory() ? 
          node.asDirectory().getChildrenNum(snapshot) : 0;
 
-     FileEncryptionInfo feInfo = getFileEncryptionInfo(node, snapshot);
+     FileEncryptionInfo feInfo = isRawPath ? null :
+         getFileEncryptionInfo(node, snapshot);
 
      return new HdfsFileStatus(
         size, 
@@ -2310,12 +2317,14 @@ public class FSDirectory implements Clos
    * Create FileStatus with location info by file INode
    */
   private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path,
-      INode node, int snapshot) throws IOException {
+      INode node, int snapshot, boolean isRawPath) throws IOException {
     assert hasReadLock();
     long size = 0; // length is zero for directories
     short replication = 0;
     long blocksize = 0;
     LocatedBlocks loc = null;
+    final FileEncryptionInfo feInfo = isRawPath ? null :
+        getFileEncryptionInfo(node, snapshot);
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
@@ -2326,7 +2335,6 @@ public class FSDirectory implements Clos
       final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
       final long fileSize = !inSnapshot && isUc ? 
           fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
-      final FileEncryptionInfo feInfo = getFileEncryptionInfo(node, snapshot);
 
       loc = getFSNamesystem().getBlockManager().createLocatedBlocks(
           fileNode.getBlocks(), fileSize, isUc, 0L, size, false,
@@ -2338,8 +2346,6 @@ public class FSDirectory implements Clos
     int childrenNum = node.isDirectory() ? 
         node.asDirectory().getChildrenNum(snapshot) : 0;
 
-    final FileEncryptionInfo feInfo = getFileEncryptionInfo(node, snapshot);
-
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
           blocksize, node.getModificationTime(snapshot),
@@ -2894,27 +2900,73 @@ public class FSDirectory implements Clos
     return src.startsWith(DOT_RESERVED_PATH_PREFIX);
   }
 
+  static boolean isReservedRawName(String src) {
+    return src.startsWith(DOT_RESERVED_PATH_PREFIX +
+        Path.SEPARATOR + RAW_STRING);
+  }
+
   /**
-   * Resolve the path of /.reserved/.inodes/<inodeid>/... to a regular path
+   * Resolve a /.reserved/... path to a non-reserved path.
+   * <p/>
+   * There are two special hierarchies under /.reserved/:
+   * <p/>
+   * /.reserved/.inodes/<inodeid> performs a path lookup by inodeid,
+   * <p/>
+   * /.reserved/raw/... returns the encrypted (raw) bytes of a file in an
+   * encryption zone. For instance, if /ezone is an encryption zone, then
+   * /ezone/a refers to the decrypted file and /.reserved/raw/ezone/a refers to
+   * the encrypted (raw) bytes of /ezone/a.
+   * <p/>
+   * Pathnames in the /.reserved/raw directory that resolve to files not in an
+   * encryption zone are equivalent to the corresponding non-raw path. Hence,
+   * if /a/b/c refers to a file that is not in an encryption zone, then
+   * /.reserved/raw/a/b/c is equivalent (they both refer to the same
+   * unencrypted file).
    * 
    * @param src path that is being processed
    * @param pathComponents path components corresponding to the path
    * @param fsd FSDirectory
-   * @return if the path indicates an inode, return path after replacing upto
+   * @return if the path indicates an inode, return path after replacing up to
    *         <inodeid> with the corresponding path of the inode, else the path
-   *         in {@code src} as is.
+   *         in {@code src} as is. If the path refers to a path in the "raw"
+   *         directory, return the non-raw pathname.
    * @throws FileNotFoundException if inodeid is invalid
    */
-  static String resolvePath(String src, byte[][] pathComponents, FSDirectory fsd)
-      throws FileNotFoundException {
-    if (pathComponents == null || pathComponents.length <= 3) {
+  static String resolvePath(String src, byte[][] pathComponents,
+      FSDirectory fsd) throws FileNotFoundException {
+    final int nComponents = (pathComponents == null) ?
+        0 : pathComponents.length;
+    if (nComponents <= 2) {
       return src;
     }
-    // Not /.reserved/.inodes
-    if (!Arrays.equals(DOT_RESERVED, pathComponents[1])
-        || !Arrays.equals(DOT_INODES, pathComponents[2])) { // Not .inodes path
+    if (!Arrays.equals(DOT_RESERVED, pathComponents[1])) {
+      /* This is not a /.reserved/ path so do nothing. */
+      return src;
+    }
+
+    if (Arrays.equals(DOT_INODES, pathComponents[2])) {
+      /* It's a /.reserved/.inodes path. */
+      if (nComponents > 3) {
+        return resolveDotInodesPath(src, pathComponents, fsd);
+      } else {
+        return src;
+      }
+    } else if (Arrays.equals(RAW, pathComponents[2])) {
+      /* It's /.reserved/raw so strip off the /.reserved/raw prefix. */
+      if (nComponents == 3) {
+        return Path.SEPARATOR;
+      } else {
+        return constructRemainingPath("", pathComponents, 3);
+      }
+    } else {
+      /* It's some sort of /.reserved/<unknown> path. Ignore it. */
       return src;
     }
+  }
+
+  private static String resolveDotInodesPath(String src,
+      byte[][] pathComponents, FSDirectory fsd)
+      throws FileNotFoundException {
     final String inodeId = DFSUtil.bytes2String(pathComponents[3]);
     final long id;
     try {
@@ -2943,10 +2995,20 @@ public class FSDirectory implements Clos
       }
     }
 
-    StringBuilder path = id == INodeId.ROOT_INODE_ID ? new StringBuilder()
-        : new StringBuilder(inode.getFullPathName());
-    for (int i = 4; i < pathComponents.length; i++) {
-      path.append(Path.SEPARATOR).append(DFSUtil.bytes2String(pathComponents[i]));
+    String path = "";
+    if (id != INodeId.ROOT_INODE_ID) {
+      path = inode.getFullPathName();
+    }
+    return constructRemainingPath(path, pathComponents, 4);
+  }
+
+  private static String constructRemainingPath(String pathPrefix,
+      byte[][] pathComponents, int startAt) {
+
+    StringBuilder path = new StringBuilder(pathPrefix);
+    for (int i = startAt; i < pathComponents.length; i++) {
+      path.append(Path.SEPARATOR).append(
+          DFSUtil.bytes2String(pathComponents[i]));
     }
     if (NameNode.LOG.isDebugEnabled()) {
       NameNode.LOG.debug("Resolved path is " + path);

Modified: hadoop/common/branches/fs-encryption/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/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java?rev=1614490&r1=1614489&r2=1614490&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java Tue Jul 29 21:11:26 2014
@@ -364,7 +364,8 @@ public class FSEditLogLoader {
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
           HdfsFileStatus stat = fsNamesys.dir.createFileStatus(
-              HdfsFileStatus.EMPTY_NAME, newFile, Snapshot.CURRENT_STATE_ID);
+              HdfsFileStatus.EMPTY_NAME, newFile, Snapshot.CURRENT_STATE_ID,
+              false);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
               addCloseOp.rpcCallId, stat);
         }

Modified: hadoop/common/branches/fs-encryption/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/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1614490&r1=1614489&r2=1614490&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Tue Jul 29 21:11:26 2014
@@ -338,7 +338,7 @@ public class FSNamesystem implements Nam
   private HdfsFileStatus getAuditFileInfo(String path, boolean resolveSymlink)
       throws IOException {
     return (isAuditEnabled() && isExternalInvocation())
-        ? dir.getFileInfo(path, resolveSymlink) : null;
+        ? dir.getFileInfo(path, resolveSymlink, false) : null;
   }
   
   private void logAuditEvent(boolean succeeded, String cmd, String src)
@@ -1663,9 +1663,10 @@ public class FSNamesystem implements Nam
     }
   }
 
-  private void setPermissionInt(String src, FsPermission permission)
+  private void setPermissionInt(final String srcArg, FsPermission permission)
       throws AccessControlException, FileNotFoundException, SafeModeException,
       UnresolvedLinkException, IOException {
+    String src = srcArg;
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -1674,7 +1675,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set permission for " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOwner(pc, src);
       dir.setPermission(src, permission);
       getEditLog().logSetPermissions(src, permission);
@@ -1683,7 +1684,7 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "setPermission", src, null, resultingStat);
+    logAuditEvent(true, "setPermission", srcArg, null, resultingStat);
   }
 
   /**
@@ -1701,9 +1702,10 @@ public class FSNamesystem implements Nam
     } 
   }
 
-  private void setOwnerInt(String src, String username, String group)
+  private void setOwnerInt(final String srcArg, String username, String group)
       throws AccessControlException, FileNotFoundException, SafeModeException,
       UnresolvedLinkException, IOException {
+    String src = srcArg;
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -1712,7 +1714,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set owner for " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOwner(pc, src);
       if (!pc.isSuperUser()) {
         if (username != null && !pc.getUser().equals(username)) {
@@ -1729,7 +1731,7 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "setOwner", src, null, resultingStat);
+    logAuditEvent(true, "setOwner", srcArg, null, resultingStat);
   }
 
   /**
@@ -1812,10 +1814,11 @@ public class FSNamesystem implements Nam
    * Get block locations within the specified range, updating the
    * access times if necessary. 
    */
-  private LocatedBlocks getBlockLocationsUpdateTimes(String src, long offset,
-      long length, boolean doAccessTime, boolean needBlockToken)
+  private LocatedBlocks getBlockLocationsUpdateTimes(final String srcArg,
+      long offset, long length, boolean doAccessTime, boolean needBlockToken)
       throws FileNotFoundException,
       UnresolvedLinkException, IOException {
+    String src = srcArg;
     FSPermissionChecker pc = getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     for (int attempt = 0; attempt < 2; attempt++) {
@@ -1827,7 +1830,7 @@ public class FSNamesystem implements Nam
         checkOperation(OperationCategory.WRITE);
         writeLock(); // writelock is needed to set accesstime
       }
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       try {
         if (isReadOp) {
           checkOperation(OperationCategory.READ);
@@ -1872,8 +1875,9 @@ public class FSNamesystem implements Nam
           isUc = false;
         }
 
-        final FileEncryptionInfo feInfo = dir.getFileEncryptionInfo(inode,
-            iip.getPathSnapshotId());
+        final FileEncryptionInfo feInfo =
+          FSDirectory.isReservedRawName(srcArg) ?
+          null : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId());
 
         final LocatedBlocks blocks =
           blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
@@ -2098,8 +2102,9 @@ public class FSNamesystem implements Nam
     }
   }
 
-  private void setTimesInt(String src, long mtime, long atime) 
+  private void setTimesInt(final String srcArg, long mtime, long atime)
     throws IOException, UnresolvedLinkException {
+    String src = srcArg;
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
@@ -2108,7 +2113,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set times " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
 
       // Write access is required to set access and modification times
       if (isPermissionEnabled) {
@@ -2129,7 +2134,7 @@ public class FSNamesystem implements Nam
     } finally {
       writeUnlock();
     }
-    logAuditEvent(true, "setTimes", src, null, resultingStat);
+    logAuditEvent(true, "setTimes", srcArg, null, resultingStat);
   }
 
   /**
@@ -2160,9 +2165,10 @@ public class FSNamesystem implements Nam
     }
   }
 
-  private void createSymlinkInt(String target, String link,
+  private void createSymlinkInt(String target, final String linkArg,
       PermissionStatus dirPerms, boolean createParent, boolean logRetryCache) 
       throws IOException, UnresolvedLinkException {
+    String link = linkArg;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.createSymlink: target="
           + target + " link=" + link);
@@ -2175,7 +2181,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create symlink " + link);
-      link = FSDirectory.resolvePath(link, pathComponents, dir);
+      link = resolvePath(link, pathComponents);
       if (!createParent) {
         verifyParentDir(link);
       }
@@ -2196,7 +2202,7 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "createSymlink", link, target, resultingStat);
+    logAuditEvent(true, "createSymlink", linkArg, target, resultingStat);
   }
 
   /**
@@ -2222,8 +2228,9 @@ public class FSNamesystem implements Nam
     }
   }
 
-  private boolean setReplicationInt(String src, final short replication)
-      throws IOException {
+  private boolean setReplicationInt(final String srcArg,
+      final short replication) throws IOException {
+    String src = srcArg;
     blockManager.verifyReplication(src, replication, null);
     final boolean isFile;
     FSPermissionChecker pc = getPermissionChecker();
@@ -2234,7 +2241,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set replication for " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       if (isPermissionEnabled) {
         checkPathAccess(pc, src, FsAction.WRITE);
       }
@@ -2252,7 +2259,7 @@ public class FSNamesystem implements Nam
 
     getEditLog().logSync();
     if (isFile) {
-      logAuditEvent(true, "setReplication", src);
+      logAuditEvent(true, "setReplication", srcArg);
     }
     return isFile;
   }
@@ -2265,7 +2272,7 @@ public class FSNamesystem implements Nam
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      filename = FSDirectory.resolvePath(filename, pathComponents, dir);
+      filename = resolvePath(filename, pathComponents);
       if (isPermissionEnabled) {
         checkTraverse(pc, filename);
       }
@@ -2395,13 +2402,14 @@ public class FSNamesystem implements Nam
     return status;
   }
 
-  private HdfsFileStatus startFileInt(String src, PermissionStatus permissions,
-      String holder, String clientMachine, EnumSet<CreateFlag> flag,
-      boolean createParent, short replication, long blockSize,
-      List<CipherSuite> cipherSuites, boolean logRetryCache)
+  private HdfsFileStatus startFileInt(final String srcArg,
+      PermissionStatus permissions, String holder, String clientMachine,
+      EnumSet<CreateFlag> flag, boolean createParent, short replication,
+      long blockSize, List<CipherSuite> cipherSuites, boolean logRetryCache)
       throws AccessControlException, SafeModeException,
       FileAlreadyExistsException, UnresolvedLinkException,
       FileNotFoundException, ParentNotDirectoryException, IOException {
+    String src = srcArg;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       StringBuilder builder = new StringBuilder();
       builder.append("DIR* NameSystem.startFile: src=" + src
@@ -2467,7 +2475,7 @@ public class FSNamesystem implements Nam
         String ezKeyName = null;
         readLock();
         try {
-          src = FSDirectory.resolvePath(src, pathComponents, dir);
+          src = resolvePath(src, pathComponents);
           INodesInPath iip = dir.getINodesInPath4Write(src);
           // Nothing to do if the path is not within an EZ
           if (dir.isInAnEZ(iip)) {
@@ -2496,11 +2504,12 @@ public class FSNamesystem implements Nam
         try {
           checkOperation(OperationCategory.WRITE);
           checkNameNodeSafeMode("Cannot create file" + src);
-          src = FSDirectory.resolvePath(src, pathComponents, dir);
+          src = resolvePath(src, pathComponents);
           startFileInternal(pc, src, permissions, holder, clientMachine, create,
               overwrite, createParent, replication, blockSize, suite, edek,
               logRetryCache);
-          stat = dir.getFileInfo(src, false);
+          stat = dir.getFileInfo(src, false,
+              FSDirectory.isReservedRawName(srcArg));
         } catch (StandbyException se) {
           skipSync = true;
           throw se;
@@ -2522,7 +2531,7 @@ public class FSNamesystem implements Nam
       }
     }
 
-    logAuditEvent(true, "create", src, null, stat);
+    logAuditEvent(true, "create", srcArg, null, stat);
     return stat;
   }
 
@@ -2767,7 +2776,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot recover the lease of " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
       if (!inode.isUnderConstruction()) {
         return true;
@@ -2894,11 +2903,12 @@ public class FSNamesystem implements Nam
     }
   }
 
-  private LocatedBlock appendFileInt(String src, String holder,
+  private LocatedBlock appendFileInt(final String srcArg, String holder,
       String clientMachine, boolean logRetryCache)
       throws AccessControlException, SafeModeException,
       FileAlreadyExistsException, FileNotFoundException,
       ParentNotDirectoryException, IOException {
+    String src = srcArg;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src=" + src
           + ", holder=" + holder
@@ -2913,7 +2923,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot append to file" + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       lb = appendFileInternal(pc, src, holder, clientMachine, logRetryCache);
     } catch (StandbyException se) {
       skipSync = true;
@@ -2934,7 +2944,7 @@ public class FSNamesystem implements Nam
             +" block size " + lb.getBlock().getNumBytes());
       }
     }
-    logAuditEvent(true, "append", src);
+    logAuditEvent(true, "append", srcArg);
     return lb;
   }
 
@@ -2979,7 +2989,7 @@ public class FSNamesystem implements Nam
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       LocatedBlock[] onRetryBlock = new LocatedBlock[1];
       FileState fileState = analyzeFileState(
           src, fileId, clientName, previous, onRetryBlock);
@@ -3202,7 +3212,7 @@ public class FSNamesystem implements Nam
       checkOperation(OperationCategory.READ);
       //check safe mode
       checkNameNodeSafeMode("Cannot add datanode; src=" + src + ", blk=" + blk);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
 
       //check lease
       final INode inode;
@@ -3255,7 +3265,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
 
       final INode inode;
       if (fileId == INodeId.GRANDFATHER_INODE_ID) {
@@ -3337,9 +3347,10 @@ public class FSNamesystem implements Nam
    *         (e.g if not all blocks have reached minimum replication yet)
    * @throws IOException on error (eg lease mismatch, file not open, file deleted)
    */
-  boolean completeFile(String src, String holder,
+  boolean completeFile(final String srcArg, String holder,
                        ExtendedBlock last, long fileId)
     throws SafeModeException, UnresolvedLinkException, IOException {
+    String src = srcArg;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " +
           src + " for " + holder);
@@ -3353,7 +3364,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot complete file " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       success = completeFileInternal(src, holder,
         ExtendedBlock.getLocalBlock(last), fileId);
     } finally {
@@ -3361,7 +3372,7 @@ public class FSNamesystem implements Nam
     }
     getEditLog().logSync();
     if (success) {
-      NameNode.stateChangeLog.info("DIR* completeFile: " + src
+      NameNode.stateChangeLog.info("DIR* completeFile: " + srcArg
           + " is closed by " + holder);
     }
     return success;
@@ -3529,8 +3540,11 @@ public class FSNamesystem implements Nam
     return ret;
   }
 
-  private boolean renameToInt(String src, String dst, boolean logRetryCache) 
+  private boolean renameToInt(final String srcArg, final String dstArg,
+    boolean logRetryCache)
     throws IOException, UnresolvedLinkException {
+    String src = srcArg;
+    String dst = dstArg;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src +
           " to " + dst);
@@ -3549,8 +3563,8 @@ public class FSNamesystem implements Nam
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename " + src);
       waitForLoadingFSImage();
-      src = FSDirectory.resolvePath(src, srcComponents, dir);
-      dst = FSDirectory.resolvePath(dst, dstComponents, dir);
+      src = resolvePath(src, srcComponents);
+      dst = resolvePath(dst, dstComponents);
       checkOperation(OperationCategory.WRITE);
       status = renameToInternal(pc, src, dst, logRetryCache);
       if (status) {
@@ -3561,7 +3575,7 @@ public class FSNamesystem implements Nam
     }
     getEditLog().logSync();
     if (status) {
-      logAuditEvent(true, "rename", src, dst, resultingStat);
+      logAuditEvent(true, "rename", srcArg, dstArg, resultingStat);
     }
     return status;
   }
@@ -3599,8 +3613,10 @@ public class FSNamesystem implements Nam
   
 
   /** Rename src to dst */
-  void renameTo(String src, String dst, Options.Rename... options)
-      throws IOException, UnresolvedLinkException {
+  void renameTo(final String srcArg, final String dstArg,
+      Options.Rename... options) throws IOException, UnresolvedLinkException {
+    String src = srcArg;
+    String dst = dstArg;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
           + src + " to " + dst);
@@ -3623,8 +3639,8 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename " + src);
-      src = FSDirectory.resolvePath(src, srcComponents, dir);
-      dst = FSDirectory.resolvePath(dst, dstComponents, dir);
+      src = resolvePath(src, srcComponents);
+      dst = resolvePath(dst, dstComponents);
       renameToInternal(pc, src, dst, cacheEntry != null, options);
       resultingStat = getAuditFileInfo(dst, false);
       success = true;
@@ -3638,7 +3654,7 @@ public class FSNamesystem implements Nam
       for (Rename option : options) {
         cmd.append(option.value()).append(" ");
       }
-      logAuditEvent(true, cmd.toString(), src, dst, resultingStat);
+      logAuditEvent(true, cmd.toString(), srcArg, dstArg, resultingStat);
     }
   }
 
@@ -3736,7 +3752,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot delete " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       if (!recursive && dir.isNonEmptyDirectory(src)) {
         throw new PathIsNotEmptyDirectoryException(src + " is non empty");
       }
@@ -3880,7 +3896,7 @@ public class FSNamesystem implements Nam
   /**
    * Get the file info for a specific file.
    *
-   * @param src The string representation of the path to the file
+   * @param srcArg The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException 
    *        if src refers to a symlink
    *
@@ -3891,9 +3907,10 @@ public class FSNamesystem implements Nam
    *         or null if file not found
    * @throws StandbyException 
    */
-  HdfsFileStatus getFileInfo(String src, boolean resolveLink) 
+  HdfsFileStatus getFileInfo(final String srcArg, boolean resolveLink)
     throws AccessControlException, UnresolvedLinkException,
            StandbyException, IOException {
+    String src = srcArg;
     if (!DFSUtil.isValidName(src)) {
       throw new InvalidPathException("Invalid file name: " + src);
     }
@@ -3904,34 +3921,36 @@ public class FSNamesystem implements Nam
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       if (isPermissionEnabled) {
         checkPermission(pc, src, false, null, null, null, null, false,
             resolveLink);
       }
-      stat = dir.getFileInfo(src, resolveLink);
+      stat = dir.getFileInfo(src, resolveLink,
+          FSDirectory.isReservedRawName(srcArg));
     } catch (AccessControlException e) {
-      logAuditEvent(false, "getfileinfo", src);
+      logAuditEvent(false, "getfileinfo", srcArg);
       throw e;
     } finally {
       readUnlock();
     }
-    logAuditEvent(true, "getfileinfo", src);
+    logAuditEvent(true, "getfileinfo", srcArg);
     return stat;
   }
   
   /**
    * Returns true if the file is closed
    */
-  boolean isFileClosed(String src) 
+  boolean isFileClosed(final String srcArg)
       throws AccessControlException, UnresolvedLinkException,
       StandbyException, IOException {
+    String src = srcArg;
     FSPermissionChecker pc = getPermissionChecker();  
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOperation(OperationCategory.READ);
       if (isPermissionEnabled) {
         checkTraverse(pc, src);
@@ -3939,7 +3958,7 @@ public class FSNamesystem implements Nam
       return !INodeFile.valueOf(dir.getINode(src), src).isUnderConstruction();
     } catch (AccessControlException e) {
       if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(false, "isFileClosed", src);
+        logAuditEvent(false, "isFileClosed", srcArg);
       }
       throw e;
     } finally {
@@ -3962,8 +3981,9 @@ public class FSNamesystem implements Nam
     return ret;
   }
 
-  private boolean mkdirsInt(String src, PermissionStatus permissions,
+  private boolean mkdirsInt(final String srcArg, PermissionStatus permissions,
       boolean createParent) throws IOException, UnresolvedLinkException {
+    String src = srcArg;
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
     }
@@ -3979,7 +3999,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);   
       checkNameNodeSafeMode("Cannot create directory " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       status = mkdirsInternal(pc, src, permissions, createParent);
       if (status) {
         resultingStat = getAuditFileInfo(src, false);
@@ -3989,7 +4009,7 @@ public class FSNamesystem implements Nam
     }
     getEditLog().logSync();
     if (status) {
-      logAuditEvent(true, "mkdirs", src, null, resultingStat);
+      logAuditEvent(true, "mkdirs", srcArg, null, resultingStat);
     }
     return status;
   }
@@ -4147,7 +4167,8 @@ public class FSNamesystem implements Nam
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  ContentSummary getContentSummary(String src) throws IOException {
+  ContentSummary getContentSummary(final String srcArg) throws IOException {
+    String src = srcArg;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
@@ -4155,7 +4176,7 @@ public class FSNamesystem implements Nam
     boolean success = true;
     try {
       checkOperation(OperationCategory.READ);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       if (isPermissionEnabled) {
         checkPermission(pc, src, false, null, null, null, FsAction.READ_EXECUTE);
       }
@@ -4166,7 +4187,7 @@ public class FSNamesystem implements Nam
       throw ace;
     } finally {
       readUnlock();
-      logAuditEvent(success, "contentSummary", src);
+      logAuditEvent(success, "contentSummary", srcArg);
     }
   }
 
@@ -4217,7 +4238,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot fsync file " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       final INode inode;
       if (fileId == INodeId.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
@@ -4657,9 +4678,10 @@ public class FSNamesystem implements Nam
     }
   }
 
-  private DirectoryListing getListingInt(String src, byte[] startAfter,
-      boolean needLocation) 
+  private DirectoryListing getListingInt(final String srcArg, byte[] startAfter,
+      boolean needLocation)
     throws AccessControlException, UnresolvedLinkException, IOException {
+    String src = srcArg;
     DirectoryListing dl;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
@@ -4668,7 +4690,7 @@ public class FSNamesystem implements Nam
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
 
       // Get file name when startAfter is an INodePath
       if (FSDirectory.isReservedName(startAfterString)) {
@@ -4692,7 +4714,7 @@ public class FSNamesystem implements Nam
           checkTraverse(pc, src);
         }
       }
-      logAuditEvent(true, "listStatus", src);
+      logAuditEvent(true, "listStatus", srcArg);
       dl = dir.getListing(src, startAfter, needLocation);
     } finally {
       readUnlock();
@@ -6085,6 +6107,28 @@ public class FSNamesystem implements Nam
     checkPermission(pc, path, false, null, null, null, null);
   }
 
+  /**
+   * This is a wrapper for FSDirectory.resolvePath(). If the path passed
+   * is prefixed with /.reserved/raw, then it checks to ensure that the caller
+   * has super user privs.
+   *
+   * @param path The path to resolve.
+   * @param pathComponents path components corresponding to the path
+   * @return if the path indicates an inode, return path after replacing up to
+   *         <inodeid> with the corresponding path of the inode, else the path
+   *         in {@code src} as is. If the path refers to a path in the "raw"
+   *         directory, return the non-raw pathname.
+   * @throws FileNotFoundException
+   * @throws AccessControlException
+   */
+  private String resolvePath(String path, byte[][] pathComponents)
+      throws FileNotFoundException, AccessControlException {
+    if (FSDirectory.isReservedRawName(path)) {
+      checkSuperuserPrivilege();
+    }
+    return FSDirectory.resolvePath(path, pathComponents, dir);
+  }
+
   @Override
   public void checkSuperuserPrivilege()
       throws AccessControlException {
@@ -8279,7 +8323,9 @@ public class FSNamesystem implements Nam
     return results;
   }
 
-  void modifyAclEntries(String src, List<AclEntry> aclSpec) throws IOException {
+  void modifyAclEntries(final String srcArg, List<AclEntry> aclSpec)
+      throws IOException {
+    String src = srcArg;
     nnConf.checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -8289,7 +8335,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOwner(pc, src);
       List<AclEntry> newAcl = dir.modifyAclEntries(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
@@ -8298,10 +8344,12 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "modifyAclEntries", src, null, resultingStat);
+    logAuditEvent(true, "modifyAclEntries", srcArg, null, resultingStat);
   }
 
-  void removeAclEntries(String src, List<AclEntry> aclSpec) throws IOException {
+  void removeAclEntries(final String srcArg, List<AclEntry> aclSpec)
+      throws IOException {
+    String src = srcArg;
     nnConf.checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -8311,7 +8359,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOwner(pc, src);
       List<AclEntry> newAcl = dir.removeAclEntries(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
@@ -8320,10 +8368,11 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "removeAclEntries", src, null, resultingStat);
+    logAuditEvent(true, "removeAclEntries", srcArg, null, resultingStat);
   }
 
-  void removeDefaultAcl(String src) throws IOException {
+  void removeDefaultAcl(final String srcArg) throws IOException {
+    String src = srcArg;
     nnConf.checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -8333,7 +8382,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOwner(pc, src);
       List<AclEntry> newAcl = dir.removeDefaultAcl(src);
       getEditLog().logSetAcl(src, newAcl);
@@ -8342,10 +8391,11 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "removeDefaultAcl", src, null, resultingStat);
+    logAuditEvent(true, "removeDefaultAcl", srcArg, null, resultingStat);
   }
 
-  void removeAcl(String src) throws IOException {
+  void removeAcl(final String srcArg) throws IOException {
+    String src = srcArg;
     nnConf.checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -8355,7 +8405,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOwner(pc, src);
       dir.removeAcl(src);
       getEditLog().logSetAcl(src, AclFeature.EMPTY_ENTRY_LIST);
@@ -8364,10 +8414,11 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "removeAcl", src, null, resultingStat);
+    logAuditEvent(true, "removeAcl", srcArg, null, resultingStat);
   }
 
-  void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+  void setAcl(final String srcArg, List<AclEntry> aclSpec) throws IOException {
+    String src = srcArg;
     nnConf.checkAclsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -8377,7 +8428,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set ACL on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOwner(pc, src);
       List<AclEntry> newAcl = dir.setAcl(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
@@ -8386,7 +8437,7 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "setAcl", src, null, resultingStat);
+    logAuditEvent(true, "setAcl", srcArg, null, resultingStat);
   }
 
   AclStatus getAclStatus(String src) throws IOException {
@@ -8397,7 +8448,7 @@ public class FSNamesystem implements Nam
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       if (isPermissionEnabled) {
         checkPermission(pc, src, false, null, null, null, null);
       }
@@ -8485,7 +8536,7 @@ public class FSNamesystem implements Nam
       checkSuperuserPrivilege();
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create encryption zone on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
 
       final XAttr ezXAttr = dir.createEncryptionZone(src, keyName);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
@@ -8496,7 +8547,7 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "createEncryptionZone", src, null, resultingStat);
+    logAuditEvent(true, "createEncryptionZone", srcArg, null, resultingStat);
   }
 
   /**
@@ -8583,8 +8634,9 @@ public class FSNamesystem implements Nam
     }
   }
   
-  private void setXAttrInt(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
-      boolean logRetryCache) throws IOException {
+  private void setXAttrInt(final String srcArg, XAttr xAttr,
+      EnumSet<XAttrSetFlag> flag, boolean logRetryCache) throws IOException {
+    String src = srcArg;
     nnConf.checkXAttrsConfigFlag();
     checkXAttrSize(xAttr);
     HdfsFileStatus resultingStat = null;
@@ -8596,7 +8648,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set XAttr on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkXAttrChangeAccess(src, xAttr, pc);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(xAttr);
@@ -8607,7 +8659,7 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "setXAttr", src, null, resultingStat);
+    logAuditEvent(true, "setXAttr", srcArg, null, resultingStat);
   }
 
   /**
@@ -8630,7 +8682,9 @@ public class FSNamesystem implements Nam
     }
   }
   
-  List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) throws IOException {
+  List<XAttr> getXAttrs(final String srcArg, List<XAttr> xAttrs)
+      throws IOException {
+    String src = srcArg;
     nnConf.checkXAttrsConfigFlag();
     FSPermissionChecker pc = getPermissionChecker();
     boolean getAll = xAttrs == null || xAttrs.isEmpty();
@@ -8638,7 +8692,7 @@ public class FSNamesystem implements Nam
       try {
         XAttrPermissionFilter.checkPermissionForApi(pc, xAttrs);
       } catch (AccessControlException e) {
-        logAuditEvent(false, "getXAttrs", src);
+        logAuditEvent(false, "getXAttrs", srcArg);
         throw e;
       }
     }
@@ -8646,7 +8700,7 @@ public class FSNamesystem implements Nam
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOperation(OperationCategory.READ);
       if (isPermissionEnabled) {
         checkPathAccess(pc, src, FsAction.READ);
@@ -8679,7 +8733,7 @@ public class FSNamesystem implements Nam
         return toGet;
       }
     } catch (AccessControlException e) {
-      logAuditEvent(false, "getXAttrs", src);
+      logAuditEvent(false, "getXAttrs", srcArg);
       throw e;
     } finally {
       readUnlock();
@@ -8693,7 +8747,7 @@ public class FSNamesystem implements Nam
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkOperation(OperationCategory.READ);
       if (isPermissionEnabled) {
         /* To access xattr names, you need EXECUTE in the owning directory. */
@@ -8740,8 +8794,9 @@ public class FSNamesystem implements Nam
     }
   }
 
-  void removeXAttrInt(String src, XAttr xAttr, boolean logRetryCache)
+  void removeXAttrInt(final String srcArg, XAttr xAttr, boolean logRetryCache)
       throws IOException {
+    String src = srcArg;
     nnConf.checkXAttrsConfigFlag();
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
@@ -8752,7 +8807,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove XAttr entry on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
+      src = resolvePath(src, pathComponents);
       checkXAttrChangeAccess(src, xAttr, pc);
 
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
@@ -8769,7 +8824,7 @@ public class FSNamesystem implements Nam
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "removeXAttr", src, null, resultingStat);
+    logAuditEvent(true, "removeXAttr", srcArg, null, resultingStat);
   }
 
   private void checkXAttrChangeAccess(String src, XAttr xAttr,

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java?rev=1614490&r1=1614489&r2=1614490&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java Tue Jul 29 21:11:26 2014
@@ -83,6 +83,7 @@ import static org.apache.hadoop.hdfs.DFS
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
@@ -1300,4 +1301,51 @@ public class DFSTestUtil {
       sockDir.close();
     }
   }
+
+  /**
+   * Verify that two files have the same contents.
+   *
+   * @param fs The file system containing the two files.
+   * @param p1 The path of the first file.
+   * @param p2 The path of the second file.
+   * @param len The length of the two files.
+   * @throws IOException
+   */
+  public static void verifyFilesEqual(FileSystem fs, Path p1, Path p2, int len)
+      throws IOException {
+    final FSDataInputStream in1 = fs.open(p1);
+    final FSDataInputStream in2 = fs.open(p2);
+    for (int i = 0; i < len; i++) {
+      assertEquals("Mismatch at byte " + i, in1.read(), in2.read());
+    }
+    in1.close();
+    in2.close();
+  }
+
+  /**
+   * Verify that two files have different contents.
+   *
+   * @param fs The file system containing the two files.
+   * @param p1 The path of the first file.
+   * @param p2 The path of the second file.
+   * @param len The length of the two files.
+   * @throws IOException
+   */
+  public static void verifyFilesNotEqual(FileSystem fs, Path p1, Path p2,
+      int len)
+          throws IOException {
+    final FSDataInputStream in1 = fs.open(p1);
+    final FSDataInputStream in2 = fs.open(p2);
+    try {
+      for (int i = 0; i < len; i++) {
+        if (in1.read() != in2.read()) {
+          return;
+        }
+      }
+      fail("files are equal, but should not be");
+    } finally {
+      in1.close();
+      in2.close();
+    }
+  }
 }

Modified: hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java?rev=1614490&r1=1614489&r2=1614490&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java Tue Jul 29 21:11:26 2014
@@ -30,7 +30,6 @@ import org.apache.hadoop.crypto.CipherSu
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSTestWrapper;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContextTestWrapper;
@@ -52,7 +51,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-
+import static org.apache.hadoop.hdfs.DFSTestUtil.verifyFilesEqual;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -316,16 +315,6 @@ public class TestEncryptionZones {
     doRenameEncryptionZone(fcWrapper);
   }
 
-  private void validateFiles(Path p1, Path p2, int len) throws Exception {
-    FSDataInputStream in1 = fs.open(p1);
-    FSDataInputStream in2 = fs.open(p2);
-    for (int i = 0; i < len; i++) {
-      assertEquals("Mismatch at byte " + i, in1.read(), in2.read());
-    }
-    in1.close();
-    in2.close();
-  }
-
   private FileEncryptionInfo getFileEncryptionInfo(Path path) throws Exception {
     LocatedBlocks blocks = fs.getClient().getLocatedBlocks(path.toString(), 0);
     return blocks.getFileEncryptionInfo();
@@ -346,14 +335,14 @@ public class TestEncryptionZones {
     final Path encFile1 = new Path(zone, "myfile");
     DFSTestUtil.createFile(fs, encFile1, len, (short) 1, 0xFEED);
     // Read them back in and compare byte-by-byte
-    validateFiles(baseFile, encFile1, len);
+    verifyFilesEqual(fs, baseFile, encFile1, len);
     // Roll the key of the encryption zone
     List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
     assertEquals("Expected 1 EZ", 1, zones.size());
     String keyName = zones.get(0).getKeyName();
     cluster.getNamesystem().getProvider().rollNewVersion(keyName);
     // Read them back in and compare byte-by-byte
-    validateFiles(baseFile, encFile1, len);
+    verifyFilesEqual(fs, baseFile, encFile1, len);
     // Write a new enc file and validate
     final Path encFile2 = new Path(zone, "myfile2");
     DFSTestUtil.createFile(fs, encFile2, len, (short) 1, 0xFEED);
@@ -366,7 +355,7 @@ public class TestEncryptionZones {
     assertNotEquals("Key was rolled, versions should be different",
         feInfo1.getEzKeyVersionName(), feInfo2.getEzKeyVersionName());
     // Contents still equal
-    validateFiles(encFile1, encFile2, len);
+    verifyFilesEqual(fs, encFile1, encFile2, len);
   }
 
   @Test(timeout = 60000)