浏览代码

Revert "HDFS-12130. Optimizing permission check for getContentSummary." to fix commit message.

This reverts commit 2ba41729abef22f9049fbd6260e8194ac1abf5cc.
Tsz-Wo Nicholas Sze 8 年之前
父节点
当前提交
d51395363a
共有 12 个文件被更改,包括 16 次插入304 次删除
  1. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  2. 0 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryComputationContext.java
  3. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java
  4. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
  5. 0 32
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  6. 3 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  7. 2 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  8. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  9. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
  10. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  11. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
  12. 0 222
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetContentSummaryWithPermission.java

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java

@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.security.AccessControlException;
 
 /** 
  * This interface is used by the block manager to expose a
@@ -37,8 +36,7 @@ public interface BlockCollection {
   /** 
    * Get content summary.
    */
-  public ContentSummary computeContentSummary(BlockStoragePolicySuite bsps)
-      throws AccessControlException;
+  public ContentSummary computeContentSummary(BlockStoragePolicySuite bsps);
 
   /**
    * @return the number of blocks

+ 0 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryComputationContext.java

@@ -20,9 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-import org.apache.hadoop.security.AccessControlException;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -38,7 +36,6 @@ public class ContentSummaryComputationContext {
   private long sleepMilliSec = 0;
   private int sleepNanoSec = 0;
 
-  private FSPermissionChecker pc;
   /**
    * Constructor
    *
@@ -50,12 +47,6 @@ public class ContentSummaryComputationContext {
    */
   public ContentSummaryComputationContext(FSDirectory dir,
       FSNamesystem fsn, long limitPerRun, long sleepMicroSec) {
-    this(dir, fsn, limitPerRun, sleepMicroSec, null);
-  }
-
-  public ContentSummaryComputationContext(FSDirectory dir,
-      FSNamesystem fsn, long limitPerRun, long sleepMicroSec,
-      FSPermissionChecker pc) {
     this.dir = dir;
     this.fsn = fsn;
     this.limitPerRun = limitPerRun;
@@ -64,7 +55,6 @@ public class ContentSummaryComputationContext {
     this.snapshotCounts = new ContentCounts.Builder().build();
     this.sleepMilliSec = sleepMicroSec/1000;
     this.sleepNanoSec = (int)((sleepMicroSec%1000)*1000);
-    this.pc = pc;
   }
 
   /** Constructor for blocking computation. */
@@ -148,12 +138,4 @@ public class ContentSummaryComputationContext {
     return (bsps != null) ? bsps:
         fsn.getBlockManager().getStoragePolicySuite();
   }
-
-  void checkPermission(INodeDirectory inode, int snapshotId, FsAction access)
-      throws AccessControlException {
-    if (dir != null && dir.isPermissionEnabled()
-        && pc != null && !pc.isSuperUser()) {
-      pc.checkPermission(inode, snapshotId, access);
-    }
-  }
 }

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

@@ -25,7 +25,6 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.EnumCounters;
-import org.apache.hadoop.security.AccessControlException;
 
 /**
  * Quota feature for {@link INodeDirectory}. 
@@ -126,8 +125,7 @@ public final class DirectoryWithQuotaFeature implements INode.Feature {
   }
 
   ContentSummaryComputationContext computeContentSummary(final INodeDirectory dir,
-      final ContentSummaryComputationContext summary)
-      throws AccessControlException {
+      final ContentSummaryComputationContext summary) {
     final long original = summary.getCounts().getStoragespace();
     long oldYieldCount = summary.getYieldCount();
     dir.computeDirectoryContentSummary(summary, Snapshot.CURRENT_STATE_ID);

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -125,8 +125,10 @@ class FSDirStatAndListingOp {
       FSDirectory fsd, String src) throws IOException {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.READ_LINK);
-    // getContentSummaryInt() call will check access (if enabled) when
-    // traversing all sub directories.
+    if (fsd.isPermissionEnabled()) {
+      fsd.checkPermission(pc, iip, false, null, null, null,
+          FsAction.READ_EXECUTE);
+    }
     return getContentSummaryInt(fsd, iip);
   }
 
@@ -501,8 +503,7 @@ class FSDirStatAndListingOp {
         // processed. 0 means disabled. I.e. blocking for the entire duration.
         ContentSummaryComputationContext cscc =
             new ContentSummaryComputationContext(fsd, fsd.getFSNamesystem(),
-                fsd.getContentCountLimit(), fsd.getContentSleepMicroSec(),
-                fsd.getPermissionChecker());
+                fsd.getContentCountLimit(), fsd.getContentSleepMicroSec());
         ContentSummary cs = targetNode.computeAndConvertContentSummary(
             iip.getPathSnapshotId(), cscc);
         fsd.addYieldCount(cscc.getYieldCount());

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

@@ -191,38 +191,6 @@ class FSPermissionChecker implements AccessControlEnforcer {
         ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir);
   }
 
-  /**
-   * Check permission only for the given inode (not checking the children's
-   * access).
-   *
-   * @param inode the inode to check.
-   * @param snapshotId the snapshot id.
-   * @param access the target access.
-   * @throws AccessControlException
-   */
-  void checkPermission(INode inode, int snapshotId, FsAction access)
-      throws AccessControlException {
-    try {
-      byte[][] localComponents = {inode.getLocalNameBytes()};
-      INodeAttributes[] iNodeAttr = {inode.getSnapshotINode(snapshotId)};
-      AccessControlEnforcer enforcer = getAccessControlEnforcer();
-      enforcer.checkPermission(
-          fsOwner, supergroup, callerUgi,
-          iNodeAttr, // single inode attr in the array
-          new INode[]{inode}, // single inode in the array
-          localComponents, snapshotId,
-          null, -1, // this will skip checkTraverse() because
-          // not checking ancestor here
-          false, null, null,
-          access, // the target access to be checked against the inode
-          null, // passing null sub access avoids checking children
-          false);
-    } catch (AccessControlException ace) {
-      throw new AccessControlException(
-          toAccessControlString(inode, inode.getFullPathName(), access));
-    }
-  }
-
   @Override
   public void checkPermission(String fsOwner, String supergroup,
       UserGroupInformation callerUgi, INodeAttributes[] inodeAttrs,

+ 3 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.Diff;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.StringUtils;
 
@@ -419,8 +418,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public abstract void destroyAndCollectBlocks(ReclaimContext reclaimContext);
 
   /** Compute {@link ContentSummary}. Blocking call */
-  public final ContentSummary computeContentSummary(
-      BlockStoragePolicySuite bsps) throws AccessControlException {
+  public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps) {
     return computeAndConvertContentSummary(Snapshot.CURRENT_STATE_ID,
         new ContentSummaryComputationContext(bsps));
   }
@@ -429,7 +427,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * Compute {@link ContentSummary}. 
    */
   public final ContentSummary computeAndConvertContentSummary(int snapshotId,
-      ContentSummaryComputationContext summary) throws AccessControlException {
+      ContentSummaryComputationContext summary) {
     computeContentSummary(snapshotId, summary);
     final ContentCounts counts = summary.getCounts();
     final ContentCounts snapshotCounts = summary.getSnapshotCounts();
@@ -462,8 +460,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * @return The same objects as summary.
    */
   public abstract ContentSummaryComputationContext computeContentSummary(
-      int snapshotId, ContentSummaryComputationContext summary)
-      throws AccessControlException;
+      int snapshotId, ContentSummaryComputationContext summary);
 
 
   /**

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

@@ -26,7 +26,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
-import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
@@ -44,7 +43,6 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.security.AccessControlException;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
@@ -632,7 +630,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
   @Override
   public ContentSummaryComputationContext computeContentSummary(int snapshotId,
-      ContentSummaryComputationContext summary) throws AccessControlException {
+      ContentSummaryComputationContext summary) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null && snapshotId == Snapshot.CURRENT_STATE_ID) {
       final ContentCounts counts = new ContentCounts.Builder().build();
@@ -654,10 +652,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   protected ContentSummaryComputationContext computeDirectoryContentSummary(
-      ContentSummaryComputationContext summary, int snapshotId)
-      throws AccessControlException{
-    // throws exception if failing the permission check
-    summary.checkPermission(this, snapshotId, FsAction.READ_EXECUTE);
+      ContentSummaryComputationContext summary, int snapshotId) {
     ReadOnlyList<INode> childrenList = getChildrenList(snapshotId);
     // Explicit traversing is done to enable repositioning after relinquishing
     // and reacquiring locks.

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

@@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeat
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.security.AccessControlException;
 
 /**
  * An anonymous reference to an inode.
@@ -315,7 +314,7 @@ public abstract class INodeReference extends INode {
 
   @Override
   public ContentSummaryComputationContext computeContentSummary(int snapshotId,
-      ContentSummaryComputationContext summary) throws AccessControlException {
+      ContentSummaryComputationContext summary) {
     return referred.computeContentSummary(snapshotId, summary);
   }
 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java

@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -222,7 +221,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
 
   @Override
   public void computeContentSummary4Snapshot(final BlockStoragePolicySuite bsps,
-      final ContentCounts counts) throws AccessControlException {
+      final ContentCounts counts) {
     counts.addContent(Content.SNAPSHOT, snapshotsByNames.size());
     counts.addContent(Content.SNAPSHOTTABLE_DIRECTORY, 1);
     super.computeContentSummary4Snapshot(bsps, counts);

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java

@@ -47,7 +47,6 @@ import org.apache.hadoop.hdfs.util.Diff.UndoInfo;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.security.AccessControlException;
 
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
@@ -631,7 +630,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   }
 
   public void computeContentSummary4Snapshot(final BlockStoragePolicySuite bsps,
-      final ContentCounts counts) throws AccessControlException {
+      final ContentCounts counts) {
     // Create a new blank summary context for blocking processing of subtree.
     ContentSummaryComputationContext summary = 
         new ContentSummaryComputationContext(bsps);

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java

@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import org.apache.hadoop.security.AccessControlException;
 
 /** Snapshot of a sub-tree in the namesystem. */
 @InterfaceAudience.Private
@@ -177,8 +176,7 @@ public class Snapshot implements Comparable<byte[]> {
     
     @Override
     public ContentSummaryComputationContext computeContentSummary(
-        int snapshotId, ContentSummaryComputationContext summary)
-        throws AccessControlException {
+        int snapshotId, ContentSummaryComputationContext summary) {
       return computeDirectoryContentSummary(summary, snapshotId);
     }
     

+ 0 - 222
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetContentSummaryWithPermission.java

@@ -1,222 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-
-import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * This class tests get content summary with permission settings.
- */
-public class TestGetContentSummaryWithPermission {
-  protected static final short REPLICATION = 3;
-  protected static final long BLOCKSIZE = 1024;
-
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private DistributedFileSystem dfs;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = new Configuration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
-    cluster =
-        new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION).build();
-    cluster.waitActive();
-
-    dfs = cluster.getFileSystem();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  /**
-   * Test getContentSummary for super user. For super user, whatever
-   * permission the directories are with, always allowed to access
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testGetContentSummarySuperUser() throws Exception {
-    final Path foo = new Path("/fooSuper");
-    final Path bar = new Path(foo, "barSuper");
-    final Path baz = new Path(bar, "bazSuper");
-    dfs.mkdirs(bar);
-    DFSTestUtil.createFile(dfs, baz, 10, REPLICATION, 0L);
-
-    ContentSummary summary;
-
-    summary = cluster.getNameNodeRpc().getContentSummary(
-        foo.toString());
-    verifySummary(summary, 2, 1, 10);
-
-    dfs.setPermission(foo, new FsPermission((short)0));
-
-    summary = cluster.getNameNodeRpc().getContentSummary(
-        foo.toString());
-    verifySummary(summary, 2, 1, 10);
-
-    dfs.setPermission(bar, new FsPermission((short)0));
-
-    summary = cluster.getNameNodeRpc().getContentSummary(
-        foo.toString());
-    verifySummary(summary, 2, 1, 10);
-
-    dfs.setPermission(baz, new FsPermission((short)0));
-
-    summary = cluster.getNameNodeRpc().getContentSummary(
-        foo.toString());
-    verifySummary(summary, 2, 1, 10);
-  }
-
-  /**
-   * Test getContentSummary for non-super, non-owner. Such users are restricted
-   * by permission of subdirectories. Namely if there is any subdirectory that
-   * does not have READ_EXECUTE access, AccessControlException will be thrown.
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testGetContentSummaryNonSuperUser() throws Exception {
-    final Path foo = new Path("/fooNoneSuper");
-    final Path bar = new Path(foo, "barNoneSuper");
-    final Path baz = new Path(bar, "bazNoneSuper");
-    // run as some random non-superuser, non-owner user.
-    final UserGroupInformation userUgi  =
-        UserGroupInformation.createUserForTesting(
-            "randomUser", new String[]{"randomGroup"});
-    dfs.mkdirs(bar);
-    DFSTestUtil.createFile(dfs, baz, 10, REPLICATION, 0L);
-
-    // by default, permission is rwxr-xr-x, as long as READ and EXECUTE are set,
-    // content summary should accessible
-    FileStatus fileStatus;
-    fileStatus = dfs.getFileStatus(foo);
-    assertEquals((short)755, fileStatus.getPermission().toOctal());
-    fileStatus = dfs.getFileStatus(bar);
-    assertEquals((short)755, fileStatus.getPermission().toOctal());
-    // file has no EXECUTE, it is rw-r--r-- default
-    fileStatus = dfs.getFileStatus(baz);
-    assertEquals((short)644, fileStatus.getPermission().toOctal());
-
-    // by default, can get content summary
-    ContentSummary summary =
-        userUgi.doAs(new PrivilegedExceptionAction<ContentSummary>() {
-          @Override
-          public ContentSummary run() throws IOException {
-            return cluster.getNameNodeRpc().getContentSummary(
-                foo.toString());
-          }
-        });
-    verifySummary(summary, 2, 1, 10);
-
-    // set empty access on root dir, should disallow content summary
-    dfs.setPermission(foo, new FsPermission((short)0));
-    try {
-      userUgi.doAs(new PrivilegedExceptionAction<ContentSummary>() {
-        @Override
-        public ContentSummary run() throws IOException {
-          return cluster.getNameNodeRpc().getContentSummary(
-              foo.toString());
-        }
-      });
-      fail("Should've fail due to access control exception.");
-    } catch (AccessControlException e) {
-      assertTrue(e.getMessage().contains("Permission denied"));
-    }
-
-    // restore foo's permission to allow READ_EXECUTE
-    dfs.setPermission(foo,
-        new FsPermission(READ_EXECUTE, READ_EXECUTE, READ_EXECUTE));
-
-    // set empty access on subdir, should disallow content summary from root dir
-    dfs.setPermission(bar, new FsPermission((short)0));
-
-    try {
-      userUgi.doAs(new PrivilegedExceptionAction<ContentSummary>() {
-        @Override
-        public ContentSummary run() throws IOException {
-          return cluster.getNameNodeRpc().getContentSummary(
-              foo.toString());
-        }
-      });
-      fail("Should've fail due to access control exception.");
-    } catch (AccessControlException e) {
-      assertTrue(e.getMessage().contains("Permission denied"));
-    }
-
-    // restore the permission of subdir to READ_EXECUTE. enable
-    // getContentSummary again for root
-    dfs.setPermission(bar,
-        new FsPermission(READ_EXECUTE, READ_EXECUTE, READ_EXECUTE));
-
-    summary = userUgi.doAs(new PrivilegedExceptionAction<ContentSummary>() {
-      @Override
-      public ContentSummary run() throws IOException {
-        return cluster.getNameNodeRpc().getContentSummary(
-            foo.toString());
-      }
-    });
-    verifySummary(summary, 2, 1, 10);
-
-    // permission of files under the directory does not affect
-    // getContentSummary
-    dfs.setPermission(baz, new FsPermission((short)0));
-    summary = userUgi.doAs(new PrivilegedExceptionAction<ContentSummary>() {
-      @Override
-      public ContentSummary run() throws IOException {
-        return cluster.getNameNodeRpc().getContentSummary(
-            foo.toString());
-      }
-    });
-    verifySummary(summary, 2, 1, 10);
-  }
-
-  private void verifySummary(ContentSummary summary, int dirCount,
-      int fileCount, int length) {
-    assertEquals(dirCount, summary.getDirectoryCount());
-    assertEquals(fileCount, summary.getFileCount());
-    assertEquals(length, summary.getLength());
-  }
-
-}