소스 검색

HDFS-6304. Merge r1591411 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1591412 13f79535-47bb-0310-9956-ffa450edef68
Haohui Mai 11 년 전
부모
커밋
915b7bc3a6

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -80,6 +80,9 @@ Release 2.5.0 - UNRELEASED
     HDFS-6269. NameNode Audit Log should differentiate between webHDFS open and
     HDFS open. (Eric Payne via jeagles)
 
+    HDFS-6304. Consolidate the logic of path resolution in FSDirectory.
+    (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

+ 91 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -516,7 +516,7 @@ public class FSDirectory implements Closeable {
     }
 
     // update space consumed
-    final INodesInPath iip = rootDir.getINodesInPath4Write(path, true);
+    final INodesInPath iip = getINodesInPath4Write(path, true);
     updateCount(iip, 0, -fileNode.getBlockDiskspace(), true);
     return true;
   }
@@ -588,7 +588,7 @@ public class FSDirectory implements Closeable {
     throws QuotaExceededException, UnresolvedLinkException, 
     FileAlreadyExistsException, SnapshotAccessControlException, IOException {
     assert hasWriteLock();
-    INodesInPath srcIIP = rootDir.getINodesInPath4Write(src, false);
+    INodesInPath srcIIP = getINodesInPath4Write(src, false);
     final INode srcInode = srcIIP.getLastINode();
     
     // check the validation of the source
@@ -814,7 +814,7 @@ public class FSDirectory implements Closeable {
       }
     }
     String error = null;
-    final INodesInPath srcIIP = rootDir.getINodesInPath4Write(src, false);
+    final INodesInPath srcIIP = getINodesInPath4Write(src, false);
     final INode srcInode = srcIIP.getLastINode();
     // validate source
     if (srcInode == null) {
@@ -852,7 +852,7 @@ public class FSDirectory implements Closeable {
           + error);
       throw new IOException(error);
     }
-    INodesInPath dstIIP = rootDir.getINodesInPath4Write(dst, false);
+    INodesInPath dstIIP = getINodesInPath4Write(dst, false);
     if (dstIIP.getINodes().length == 1) {
       error = "rename destination cannot be the root";
       NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
@@ -957,7 +957,7 @@ public class FSDirectory implements Closeable {
       // src and dst file/dir are in the same directory, and the dstParent has
       // been replaced when we removed the src. Refresh the dstIIP and
       // dstParent.
-      dstIIP = rootDir.getINodesInPath4Write(dst, false);
+      dstIIP = getINodesInPath4Write(dst, false);
     }
     
     boolean undoRemoveDst = false;
@@ -1114,7 +1114,7 @@ public class FSDirectory implements Closeable {
       UnresolvedLinkException, SnapshotAccessControlException {
     assert hasWriteLock();
 
-    final INodesInPath iip = rootDir.getINodesInPath4Write(src, true);
+    final INodesInPath iip = getINodesInPath4Write(src, true);
     final INode inode = iip.getLastINode();
     if (inode == null || !inode.isFile()) {
       return null;
@@ -1155,7 +1155,7 @@ public class FSDirectory implements Closeable {
       FileNotFoundException, IOException {
     readLock();
     try {
-      return INodeFile.valueOf(rootDir.getNode(path, false), path
+      return INodeFile.valueOf(getNode(path, false), path
           ).getPreferredBlockSize();
     } finally {
       readUnlock();
@@ -1166,7 +1166,7 @@ public class FSDirectory implements Closeable {
     src = normalizePath(src);
     readLock();
     try {
-      INode inode = rootDir.getNode(src, false);
+      INode inode = getNode(src, false);
       if (inode == null) {
          return false;
       }
@@ -1192,7 +1192,7 @@ public class FSDirectory implements Closeable {
       throws FileNotFoundException, UnresolvedLinkException,
       QuotaExceededException, SnapshotAccessControlException {
     assert hasWriteLock();
-    final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
+    final INodesInPath inodesInPath = getINodesInPath4Write(src, true);
     final INode inode = inodesInPath.getLastINode();
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
@@ -1217,7 +1217,7 @@ public class FSDirectory implements Closeable {
       throws FileNotFoundException, UnresolvedLinkException,
       QuotaExceededException, SnapshotAccessControlException {
     assert hasWriteLock();
-    final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
+    final INodesInPath inodesInPath = getINodesInPath4Write(src, true);
     INode inode = inodesInPath.getLastINode();
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: " + src);
@@ -1264,7 +1264,7 @@ public class FSDirectory implements Closeable {
     }
     // do the move
     
-    final INodesInPath trgIIP = rootDir.getINodesInPath4Write(target, true);
+    final INodesInPath trgIIP = getINodesInPath4Write(target, true);
     final INode[] trgINodes = trgIIP.getINodes();
     final INodeFile trgInode = trgIIP.getLastINode().asFile();
     INodeDirectory trgParent = trgINodes[trgINodes.length-2].asDirectory();
@@ -1333,7 +1333,7 @@ public class FSDirectory implements Closeable {
     final long filesRemoved;
     writeLock();
     try {
-      final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(
+      final INodesInPath inodesInPath = getINodesInPath4Write(
           normalizePath(src), false);
       if (!deleteAllowed(inodesInPath, src) ) {
         filesRemoved = -1;
@@ -1383,7 +1383,7 @@ public class FSDirectory implements Closeable {
   boolean isNonEmptyDirectory(String path) throws UnresolvedLinkException {
     readLock();
     try {
-      final INodesInPath inodesInPath = rootDir.getLastINodeInPath(path, false);
+      final INodesInPath inodesInPath = getLastINodeInPath(path, false);
       final INode inode = inodesInPath.getINode(0);
       if (inode == null || !inode.isDirectory()) {
         //not found or not a directory
@@ -1412,7 +1412,7 @@ public class FSDirectory implements Closeable {
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     List<INode> removedINodes = new ChunkedArrayList<INode>();
 
-    final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(
+    final INodesInPath inodesInPath = getINodesInPath4Write(
         normalizePath(src), false);
     long filesRemoved = -1;
     if (deleteAllowed(inodesInPath, src)) {
@@ -1537,7 +1537,7 @@ public class FSDirectory implements Closeable {
       if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
         return getSnapshotsListing(srcs, startAfter);
       }
-      final INodesInPath inodesInPath = rootDir.getLastINodeInPath(srcs, true);
+      final INodesInPath inodesInPath = getLastINodeInPath(srcs, true);
       final int snapshot = inodesInPath.getPathSnapshotId();
       final INode targetNode = inodesInPath.getINode(0);
       if (targetNode == null)
@@ -1627,7 +1627,7 @@ public class FSDirectory implements Closeable {
       if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
         return getFileInfo4DotSnapshot(srcs);
       }
-      final INodesInPath inodesInPath = rootDir.getLastINodeInPath(srcs, resolveLink);
+      final INodesInPath inodesInPath = getLastINodeInPath(srcs, resolveLink);
       final INode i = inodesInPath.getINode(0);
       return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
           inodesInPath.getPathSnapshotId());
@@ -1676,7 +1676,7 @@ public class FSDirectory implements Closeable {
     waitForReady();
     readLock();
     try {
-      final INode i = rootDir.getNode(src, false);
+      final INode i = getNode(src, false);
       return i != null && i.isFile()? i.asFile().getBlocks(): null;
     } finally {
       readUnlock();
@@ -1703,7 +1703,7 @@ public class FSDirectory implements Closeable {
        throws UnresolvedLinkException {
     readLock();
     try {
-      return rootDir.getLastINodeInPath(src, true);
+      return getLastINodeInPath(src, true);
     } finally {
       readUnlock();
     }
@@ -1716,7 +1716,7 @@ public class FSDirectory implements Closeable {
       ) throws UnresolvedLinkException, SnapshotAccessControlException {
     readLock();
     try {
-      return rootDir.getINodesInPath4Write(src, true);
+      return getINodesInPath4Write(src, true);
     } finally {
       readUnlock();
     }
@@ -1730,7 +1730,7 @@ public class FSDirectory implements Closeable {
       SnapshotAccessControlException {
     readLock();
     try {
-      return rootDir.getINode4Write(src, true);
+      return getINode4Write(src, true);
     } finally {
       readUnlock();
     }
@@ -1746,7 +1746,7 @@ public class FSDirectory implements Closeable {
     readLock();
     try {
       if (srcs.startsWith("/") && !srcs.endsWith("/")
-          && rootDir.getINode4Write(srcs, false) == null) {
+          && getINode4Write(srcs, false) == null) {
         return true;
       } else {
         return false;
@@ -1763,7 +1763,7 @@ public class FSDirectory implements Closeable {
     src = normalizePath(src);
     readLock();
     try {
-      INode node = rootDir.getNode(src, false);
+      INode node = getNode(src, false);
       return node != null && node.isDirectory();
     } finally {
       readUnlock();
@@ -1779,7 +1779,7 @@ public class FSDirectory implements Closeable {
     src = normalizePath(src);
     readLock();
     try {
-      INode node = rootDir.getINode4Write(src, false);
+      INode node = getINode4Write(src, false);
       return node != null && node.isDirectory();
     } finally {
       readUnlock();
@@ -1800,7 +1800,7 @@ public class FSDirectory implements Closeable {
           UnresolvedLinkException, SnapshotAccessControlException {
     writeLock();
     try {
-      final INodesInPath iip = rootDir.getINodesInPath4Write(path, false);
+      final INodesInPath iip = getINodesInPath4Write(path, false);
       if (iip.getLastINode() == null) {
         throw new FileNotFoundException("Path not found: " + path);
       }
@@ -2372,7 +2372,7 @@ public class FSDirectory implements Closeable {
     String srcs = normalizePath(src);
     readLock();
     try {
-      INode targetNode = rootDir.getNode(srcs, false);
+      INode targetNode = getNode(srcs, false);
       if (targetNode == null) {
         throw new FileNotFoundException("File does not exist: " + srcs);
       }
@@ -2470,7 +2470,7 @@ public class FSDirectory implements Closeable {
     }
     
     String srcs = normalizePath(src);
-    final INodesInPath iip = rootDir.getINodesInPath4Write(srcs, true);
+    final INodesInPath iip = getINodesInPath4Write(srcs, true);
     INodeDirectory dirNode = INodeDirectory.valueOf(iip.getLastINode(), srcs);
     if (dirNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
       throw new IllegalArgumentException("Cannot clear namespace quota on root.");
@@ -2753,7 +2753,7 @@ public class FSDirectory implements Closeable {
   private List<AclEntry> unprotectedModifyAclEntries(String src,
       List<AclEntry> aclSpec) throws IOException {
     assert hasWriteLock();
-    INodesInPath iip = rootDir.getINodesInPath4Write(normalizePath(src), true);
+    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
     INode inode = resolveLastINode(src, iip);
     int snapshotId = iip.getLatestSnapshotId();
     List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
@@ -2776,7 +2776,7 @@ public class FSDirectory implements Closeable {
   private List<AclEntry> unprotectedRemoveAclEntries(String src,
       List<AclEntry> aclSpec) throws IOException {
     assert hasWriteLock();
-    INodesInPath iip = rootDir.getINodesInPath4Write(normalizePath(src), true);
+    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
     INode inode = resolveLastINode(src, iip);
     int snapshotId = iip.getLatestSnapshotId();
     List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
@@ -2799,7 +2799,7 @@ public class FSDirectory implements Closeable {
   private List<AclEntry> unprotectedRemoveDefaultAcl(String src)
       throws IOException {
     assert hasWriteLock();
-    INodesInPath iip = rootDir.getINodesInPath4Write(normalizePath(src), true);
+    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
     INode inode = resolveLastINode(src, iip);
     int snapshotId = iip.getLatestSnapshotId();
     List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
@@ -2821,7 +2821,7 @@ public class FSDirectory implements Closeable {
 
   private void unprotectedRemoveAcl(String src) throws IOException {
     assert hasWriteLock();
-    INodesInPath iip = rootDir.getINodesInPath4Write(normalizePath(src), true);
+    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
     INode inode = resolveLastINode(src, iip);
     int snapshotId = iip.getLatestSnapshotId();
     AclStorage.removeINodeAcl(inode, snapshotId);
@@ -2846,7 +2846,7 @@ public class FSDirectory implements Closeable {
     }
 
     assert hasWriteLock();
-    INodesInPath iip = rootDir.getINodesInPath4Write(normalizePath(src), true);
+    INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
     INode inode = resolveLastINode(src, iip);
     int snapshotId = iip.getLatestSnapshotId();
     List<AclEntry> existingAcl = AclStorage.readINodeLogicalAcl(inode);
@@ -2866,7 +2866,7 @@ public class FSDirectory implements Closeable {
           getINode4DotSnapshot(srcs) != null) {
         return new AclStatus.Builder().owner("").group("").build();
       }
-      INodesInPath iip = rootDir.getLastINodeInPath(srcs, true);
+      INodesInPath iip = getLastINodeInPath(srcs, true);
       INode inode = resolveLastINode(src, iip);
       int snapshotId = iip.getPathSnapshotId();
       List<AclEntry> acl = AclStorage.readINodeAcl(inode, snapshotId);
@@ -2978,7 +2978,18 @@ public class FSDirectory implements Closeable {
   static byte[][] getPathComponentsForReservedPath(String src) {
     return !isReservedName(src) ? null : INode.getPathComponents(src);
   }
-  
+
+  /** Check if a given inode name is reserved */
+  public static boolean isReservedName(INode inode) {
+    return CHECK_RESERVED_FILE_NAMES
+            && Arrays.equals(inode.getLocalNameBytes(), DOT_RESERVED);
+  }
+
+  /** Check if a given path is reserved */
+  public static boolean isReservedName(String src) {
+    return src.startsWith(DOT_RESERVED_PATH_PREFIX);
+  }
+
   /**
    * Resolve the path of /.reserved/.inodes/<inodeid>/... to a regular path
    * 
@@ -3038,15 +3049,53 @@ public class FSDirectory implements Closeable {
     }
     return path.toString();
   }
-  
-  /** Check if a given inode name is reserved */
-  public static boolean isReservedName(INode inode) {
-    return CHECK_RESERVED_FILE_NAMES
-        && Arrays.equals(inode.getLocalNameBytes(), DOT_RESERVED);
+
+  /** @return the {@link INodesInPath} containing only the last inode. */
+  private INodesInPath getLastINodeInPath(String path, boolean resolveLink
+  ) throws UnresolvedLinkException {
+    return INodesInPath.resolve(rootDir, INode.getPathComponents(path), 1,
+            resolveLink);
   }
-  
-  /** Check if a given path is reserved */
-  public static boolean isReservedName(String src) {
-    return src.startsWith(DOT_RESERVED_PATH_PREFIX);
+
+  /** @return the {@link INodesInPath} containing all inodes in the path. */
+  INodesInPath getINodesInPath(String path, boolean resolveLink
+  ) throws UnresolvedLinkException {
+    final byte[][] components = INode.getPathComponents(path);
+    return INodesInPath.resolve(rootDir, components, components.length,
+            resolveLink);
+  }
+
+  /** @return the last inode in the path. */
+  INode getNode(String path, boolean resolveLink)
+          throws UnresolvedLinkException {
+    return getLastINodeInPath(path, resolveLink).getINode(0);
+  }
+
+  /**
+   * @return the INode of the last component in src, or null if the last
+   * component does not exist.
+   * @throws UnresolvedLinkException if symlink can't be resolved
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   */
+  private INode getINode4Write(String src, boolean resolveLink)
+          throws UnresolvedLinkException, SnapshotAccessControlException {
+    return getINodesInPath4Write(src, resolveLink).getLastINode();
+  }
+
+  /**
+   * @return the INodesInPath of the components in src
+   * @throws UnresolvedLinkException if symlink can't be resolved
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   */
+  private INodesInPath getINodesInPath4Write(String src, boolean resolveLink)
+          throws UnresolvedLinkException, SnapshotAccessControlException {
+    final byte[][] components = INode.getPathComponents(src);
+    INodesInPath inodesInPath = INodesInPath.resolve(rootDir, components,
+            components.length, resolveLink);
+    if (inodesInPath.isSnapshot()) {
+      throw new SnapshotAccessControlException(
+              "Modification on a read-only snapshot is disallowed");
+    }
+    return inodesInPath;
   }
 }

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

@@ -495,7 +495,7 @@ public class FSImageFormat {
      // Rename .snapshot paths if we're doing an upgrade
      parentPath = renameReservedPathsOnUpgrade(parentPath, getLayoutVersion());
      final INodeDirectory parent = INodeDirectory.valueOf(
-         namesystem.dir.rootDir.getNode(parentPath, true), parentPath);
+         namesystem.dir.getNode(parentPath, true), parentPath);
      return loadChildren(parent, in, counter);
    }
 

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

@@ -5506,7 +5506,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       dir.waitForReady();
       readLock();
       try {
-        pc.checkPermission(path, dir.rootDir, doCheckOwner, ancestorAccess,
+        pc.checkPermission(path, dir, doCheckOwner, ancestorAccess,
             parentAccess, access, subAccess, resolveLink);
       } finally {
         readUnlock();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -144,7 +144,7 @@ class FSPermissionChecker {
    * Guarded by {@link FSNamesystem#readLock()}
    * Caller of this method must hold that lock.
    */
-  void checkPermission(String path, INodeDirectory root, boolean doCheckOwner,
+  void checkPermission(String path, FSDirectory dir, boolean doCheckOwner,
       FsAction ancestorAccess, FsAction parentAccess, FsAction access,
       FsAction subAccess, boolean resolveLink)
       throws AccessControlException, UnresolvedLinkException {
@@ -159,7 +159,7 @@ class FSPermissionChecker {
     }
     // check if (parentAccess != null) && file exists, then check sb
     // If resolveLink, the check is performed on the link target.
-    final INodesInPath inodesInPath = root.getINodesInPath(path, resolveLink);
+    final INodesInPath inodesInPath = dir.getINodesInPath(path, resolveLink);
     final int snapshotId = inodesInPath.getPathSnapshotId();
     final INode[] inodes = inodesInPath.getINodes();
     int ancestorIndex = inodes.length - 2;

+ 0 - 47
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -403,53 +403,6 @@ public class INodeDirectory extends INodeWithAdditionalFields
         : ReadOnlyList.Util.asReadOnlyList(children);
   }
 
-  /** @return the {@link INodesInPath} containing only the last inode. */
-  INodesInPath getLastINodeInPath(String path, boolean resolveLink
-      ) throws UnresolvedLinkException {
-    return INodesInPath.resolve(this, getPathComponents(path), 1, resolveLink);
-  }
-
-  /** @return the {@link INodesInPath} containing all inodes in the path. */
-  INodesInPath getINodesInPath(String path, boolean resolveLink
-      ) throws UnresolvedLinkException {
-    final byte[][] components = getPathComponents(path);
-    return INodesInPath.resolve(this, components, components.length, resolveLink);
-  }
-
-  /** @return the last inode in the path. */
-  INode getNode(String path, boolean resolveLink) 
-    throws UnresolvedLinkException {
-    return getLastINodeInPath(path, resolveLink).getINode(0);
-  }
-
-  /**
-   * @return the INode of the last component in src, or null if the last
-   * component does not exist.
-   * @throws UnresolvedLinkException if symlink can't be resolved
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   */
-  INode getINode4Write(String src, boolean resolveLink)
-      throws UnresolvedLinkException, SnapshotAccessControlException {
-    return getINodesInPath4Write(src, resolveLink).getLastINode();
-  }
-
-  /**
-   * @return the INodesInPath of the components in src
-   * @throws UnresolvedLinkException if symlink can't be resolved
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   */
-  INodesInPath getINodesInPath4Write(String src, boolean resolveLink)
-      throws UnresolvedLinkException, SnapshotAccessControlException {
-    final byte[][] components = INode.getPathComponents(src);
-    INodesInPath inodesInPath = INodesInPath.resolve(this, components,
-        components.length, resolveLink);
-    if (inodesInPath.isSnapshot()) {
-      throw new SnapshotAccessControlException(
-          "Modification on a read-only snapshot is disallowed");
-    }
-    return inodesInPath;
-  }
-
   /**
    * Given a child's name, return the index of the next child
    *

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

@@ -1306,7 +1306,7 @@ public abstract class FSAclBaseTest {
    */
   private static void assertAclFeature(Path pathToCheck,
       boolean expectAclFeature) throws IOException {
-    INode inode = cluster.getNamesystem().getFSDirectory().getRoot()
+    INode inode = cluster.getNamesystem().getFSDirectory()
       .getNode(pathToCheck.toUri().getPath(), false);
     assertNotNull(inode);
     AclFeature aclFeature = inode.getAclFeature();

+ 20 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java

@@ -26,6 +26,7 @@ import static org.junit.Assert.*;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.hadoop.conf.Configuration;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -38,7 +39,10 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
+import static org.mockito.Mockito.*;
 /**
  * Unit tests covering FSPermissionChecker.  All tests in this suite have been
  * cross-validated against Linux setfacl/getfacl to check for consistency of the
@@ -56,14 +60,24 @@ public class TestFSPermissionChecker {
   private static final UserGroupInformation CLARK =
     UserGroupInformation.createUserForTesting("clark", new String[] { "execs" });
 
+  private FSDirectory dir;
   private INodeDirectory inodeRoot;
 
   @Before
   public void setUp() {
-    PermissionStatus permStatus = PermissionStatus.createImmutable(SUPERUSER,
-      SUPERGROUP, FsPermission.createImmutable((short)0755));
-    inodeRoot = new INodeDirectory(INodeId.ROOT_INODE_ID,
-      INodeDirectory.ROOT_NAME, permStatus, 0L);
+    Configuration conf = new Configuration();
+    FSNamesystem fsn = mock(FSNamesystem.class);
+    doAnswer(new Answer() {
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        Object[] args = invocation.getArguments();
+        FsPermission perm = (FsPermission) args[0];
+        return new PermissionStatus(SUPERUSER, SUPERGROUP, perm);
+      }
+    }).when(fsn).createFsOwnerPermissions(any(FsPermission.class));
+    FSImage image = mock(FSImage.class);
+    dir = new FSDirectory(image, fsn, conf);
+    inodeRoot = dir.getRoot();
   }
 
   @Test
@@ -379,14 +393,14 @@ public class TestFSPermissionChecker {
   private void assertPermissionGranted(UserGroupInformation user, String path,
       FsAction access) throws IOException {
     new FSPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(path,
-      inodeRoot, false, null, null, access, null, true);
+      dir, false, null, null, access, null, true);
   }
 
   private void assertPermissionDenied(UserGroupInformation user, String path,
       FsAction access) throws IOException {
     try {
       new FSPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(path,
-        inodeRoot, false, null, null, access, null, true);
+        dir, false, null, null, access, null, true);
       fail("expected AccessControlException for user + " + user + ", path = " +
         path + ", access = " + access);
     } catch (AccessControlException e) {

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

@@ -699,7 +699,7 @@ public class TestFsck {
       DFSTestUtil.waitReplication(fs, filePath, (short)1);
       
       // intentionally corrupt NN data structure
-      INodeFile node = (INodeFile)cluster.getNamesystem().dir.rootDir.getNode(
+      INodeFile node = (INodeFile)cluster.getNamesystem().dir.getNode(
           fileName, true);
       final BlockInfo[] blocks = node.getBlocks(); 
       assertEquals(blocks.length, 1);