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 wh...@apache.org on 2014/11/21 04:21:43 UTC

hadoop git commit: HDFS-7415. Move FSNameSystem.resolvePath() to FSDirectory. Contributed by Haohui Mai.

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 3a4211ec2 -> 7f8d5dc80


HDFS-7415. Move FSNameSystem.resolvePath() to FSDirectory. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7f8d5dc8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7f8d5dc8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7f8d5dc8

Branch: refs/heads/branch-2
Commit: 7f8d5dc802ae7922772b0f6056f5c21a8cb7b01a
Parents: 3a4211e
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Nov 20 19:21:23 2014 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Nov 20 19:21:35 2014 -0800

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSDirectory.java       |  28 +++++
 .../hdfs/server/namenode/FSNamesystem.java      | 104 ++++++++-----------
 2 files changed, 72 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f8d5dc8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 9ca50c4..929968d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -151,6 +151,8 @@ public class FSDirectory implements Closeable {
   // lock to protect the directory and BlockMap
   private final ReentrantReadWriteLock dirLock;
 
+  private final boolean isPermissionEnabled;
+
   // utility methods to acquire and release read lock and write lock
   void readLock() {
     this.dirLock.readLock().lock();
@@ -197,6 +199,9 @@ public class FSDirectory implements Closeable {
     this.dirLock = new ReentrantReadWriteLock(true); // fair
     rootDir = createRoot(ns);
     inodeMap = INodeMap.newInstance(rootDir);
+    this.isPermissionEnabled = conf.getBoolean(
+      DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY,
+      DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT);
     int configuredLimit = conf.getInt(
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     this.lsLimit = configuredLimit>0 ?
@@ -838,6 +843,29 @@ public class FSDirectory implements Closeable {
     checkSnapshot(srcInode, null);
   }
 
+  /**
+   * This is a wrapper for resolvePath(). If the path passed
+   * is prefixed with /.reserved/raw, then it checks to ensure that the caller
+   * has super user has super user privileges.
+   *
+   * @param pc The permission checker used when resolving path.
+   * @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
+   */
+  String resolvePath(FSPermissionChecker pc, String path, byte[][] pathComponents)
+      throws FileNotFoundException, AccessControlException {
+    if (isReservedRawName(path) && isPermissionEnabled) {
+      pc.checkSuperuserPrivilege();
+    }
+    return resolvePath(path, pathComponents, this);
+  }
+
   private class RenameOperation {
     private final INodesInPath srcIIP;
     private final INodesInPath dstIIP;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f8d5dc8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 9e51b2d..9df2983 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1696,7 +1696,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set permission for " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOwner(pc, src);
       dir.setPermission(src, permission);
       getEditLog().logSetPermissions(src, permission);
@@ -1735,7 +1735,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set owner for " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOwner(pc, src);
       if (!pc.isSuperUser()) {
         if (username != null && !pc.getUser().equals(username)) {
@@ -1852,7 +1852,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         writeLock(); // writelock is needed to set accesstime
       }
       try {
-        src = resolvePath(src, pathComponents);
+        src = dir.resolvePath(pc, src, pathComponents);
         if (isReadOp) {
           checkOperation(OperationCategory.READ);
         } else {
@@ -2145,7 +2145,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set times " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
 
       // Write access is required to set access and modification times
       if (isPermissionEnabled) {
@@ -2217,7 +2217,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create symlink " + link);
-      link = resolvePath(link, pathComponents);
+      link = dir.resolvePath(pc, link, pathComponents);
       if (!createParent) {
         verifyParentDir(link);
       }
@@ -2277,7 +2277,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set replication for " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       if (isPermissionEnabled) {
         checkPathAccess(pc, src, FsAction.WRITE);
       }
@@ -2384,7 +2384,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      filename = resolvePath(filename, pathComponents);
+      filename = dir.resolvePath(pc, filename, pathComponents);
       if (isPermissionEnabled) {
         checkTraverse(pc, filename);
       }
@@ -2576,7 +2576,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (provider != null) {
       readLock();
       try {
-        src = resolvePath(src, pathComponents);
+        src = dir.resolvePath(pc, src, pathComponents);
         INodesInPath iip = dir.getINodesInPath4Write(src);
         // Nothing to do if the path is not within an EZ
         final EncryptionZone zone = dir.getEZForPath(iip);
@@ -2612,7 +2612,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create file" + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       toRemoveBlocks = startFileInternal(pc, src, permissions, holder, 
           clientMachine, create, overwrite, createParent, replication, 
           blockSize, isLazyPersist, suite, protocolVersion, edek, logRetryCache);
@@ -2928,7 +2928,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot recover the lease of " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
       if (!inode.isUnderConstruction()) {
         return true;
@@ -3081,7 +3081,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot append to file" + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       lb = appendFileInternal(pc, src, holder, clientMachine, logRetryCache);
     } catch (StandbyException se) {
       skipSync = true;
@@ -3146,10 +3146,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // Part I. Analyze the state of the file with respect to the input data.
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       LocatedBlock[] onRetryBlock = new LocatedBlock[1];
       FileState fileState = analyzeFileState(
           src, fileId, clientName, previous, onRetryBlock);
@@ -3394,12 +3395,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final List<DatanodeStorageInfo> chosen;
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
       //check safe mode
       checkNameNodeSafeMode("Cannot add datanode; src=" + src + ", blk=" + blk);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
 
       //check lease
       final INode inode;
@@ -3450,12 +3452,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = getPermissionChecker();
     waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
 
       final INode inode;
       if (fileId == INodeId.GRANDFATHER_INODE_ID) {
@@ -3549,12 +3552,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean success = false;
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = getPermissionChecker();
     waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot complete file " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       success = completeFileInternal(src, holder,
         ExtendedBlock.getLocalBlock(last), fileId);
     } finally {
@@ -3756,8 +3760,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename " + src);
       waitForLoadingFSImage();
-      src = resolvePath(src, srcComponents);
-      dst = resolvePath(dst, dstComponents);
+      src = dir.resolvePath(pc, src, srcComponents);
+      dst = dir.resolvePath(pc, dst, dstComponents);
       checkOperation(OperationCategory.WRITE);
       status = renameToInternal(pc, src, dst, logRetryCache);
       if (status) {
@@ -3833,8 +3837,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename " + src);
-      src = resolvePath(src, srcComponents);
-      dst = resolvePath(dst, dstComponents);
+      src = dir.resolvePath(pc, src, srcComponents);
+      dst = dir.resolvePath(pc, dst, dstComponents);
       renameToInternal(pc, src, dst, cacheEntry != null, 
           collectedBlocks, options);
       resultingStat = getAuditFileInfo(dst, false);
@@ -3952,7 +3956,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot delete " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       if (!recursive && dir.isNonEmptyDirectory(src)) {
         throw new PathIsNotEmptyDirectoryException(src + " is non empty");
       }
@@ -4121,7 +4125,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       boolean isSuperUser = true;
       if (isPermissionEnabled) {
         checkPermission(pc, src, false, null, null, null, null, false,
@@ -4152,7 +4156,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOperation(OperationCategory.READ);
       if (isPermissionEnabled) {
         checkTraverse(pc, src);
@@ -4201,7 +4205,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);   
       checkNameNodeSafeMode("Cannot create directory " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       status = mkdirsInternal(pc, src, permissions, createParent);
       if (status) {
         resultingStat = getAuditFileInfo(src, false);
@@ -4378,7 +4382,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean success = true;
     try {
       checkOperation(OperationCategory.READ);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       if (isPermissionEnabled) {
         checkPermission(pc, src, false, null, null, null, FsAction.READ_EXECUTE);
       }
@@ -4435,12 +4439,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
 
+    FSPermissionChecker pc = getPermissionChecker();
     waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot fsync file " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       final INode inode;
       if (fileId == INodeId.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
@@ -4918,7 +4923,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
 
       // Get file name when startAfter is an INodePath
       if (FSDirectory.isReservedName(startAfterString)) {
@@ -6447,28 +6452,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     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 {
@@ -8576,7 +8559,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOwner(pc, src);
       List<AclEntry> newAcl = dir.modifyAclEntries(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
@@ -8603,7 +8586,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOwner(pc, src);
       List<AclEntry> newAcl = dir.removeAclEntries(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
@@ -8629,7 +8612,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOwner(pc, src);
       List<AclEntry> newAcl = dir.removeDefaultAcl(src);
       getEditLog().logSetAcl(src, newAcl);
@@ -8655,7 +8638,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL on " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOwner(pc, src);
       dir.removeAcl(src);
       getEditLog().logSetAcl(src, AclFeature.EMPTY_ENTRY_LIST);
@@ -8681,7 +8664,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set ACL on " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOwner(pc, src);
       List<AclEntry> newAcl = dir.setAcl(src, aclSpec);
       getEditLog().logSetAcl(src, newAcl);
@@ -8705,7 +8688,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       if (isPermissionEnabled) {
         checkPermission(pc, src, false, null, null, null, null);
       }
@@ -8781,12 +8764,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final byte[][] pathComponents =
       FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkSuperuserPrivilege();
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create encryption zone on " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
 
       final CipherSuite suite = CipherSuite.convert(cipher);
       // For now this is hardcoded, as we only support one method.
@@ -8828,7 +8812,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         checkPathAccess(pc, src, FsAction.READ);
       }
       checkOperation(OperationCategory.READ);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       final INodesInPath iip = dir.getINodesInPath(src, true);
       final EncryptionZone ret = dir.getEZForPath(iip);
       resultingStat = getAuditFileInfo(src, false);
@@ -8907,7 +8891,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set XAttr on " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkXAttrChangeAccess(src, xAttr, pc);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(xAttr);
@@ -8960,7 +8944,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOperation(OperationCategory.READ);
       if (isPermissionEnabled) {
         checkPathAccess(pc, src, FsAction.READ);
@@ -9008,7 +8992,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     try {
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkOperation(OperationCategory.READ);
       if (isPermissionEnabled) {
         /* To access xattr names, you need EXECUTE in the owning directory. */
@@ -9069,7 +9053,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove XAttr entry on " + src);
-      src = resolvePath(src, pathComponents);
+      src = dir.resolvePath(pc, src, pathComponents);
       checkXAttrChangeAccess(src, xAttr, pc);
 
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);