Revert "HDFS-10390. Implement asynchronous setAcl/getAclStatus for 
DistributedFileSystem.  Contributed by Xiaobing Zhou"

This reverts commit 02d4e478a398c24a5e5e8ea2b0822a5b9d4a97ae.


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

Branch: refs/heads/HDFS-1312
Commit: b82c74b9102ba95eae776501ed4484be9edd8c96
Parents: 5ee5912
Author: Andrew Wang <w...@apache.org>
Authored: Fri Jun 3 18:09:14 2016 -0700
Committer: Andrew Wang <w...@apache.org>
Committed: Fri Jun 3 18:09:14 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |  59 ----
 .../hadoop/hdfs/DistributedFileSystem.java      |   3 -
 .../ClientNamenodeProtocolTranslatorPB.java     |  30 +-
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    | 310 -------------------
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  |  15 +-
 .../hdfs/server/namenode/FSAclBaseTest.java     |  12 +-
 6 files changed, 18 insertions(+), 411 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b82c74b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
index 29bac2a..6bfd71d 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
@@ -19,16 +19,12 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
 import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.hadoop.ipc.Client;
@@ -87,7 +83,6 @@ public class AsyncDistributedFileSystem {
   public Future<Void> rename(Path src, Path dst,
       final Options.Rename... options) throws IOException {
     dfs.getFsStatistics().incrementWriteOps(1);
-    dfs.getDFSOpsCountStatistics().incrementOpCounter(OpType.RENAME);
 
     final Path absSrc = dfs.fixRelativePart(src);
     final Path absDst = dfs.fixRelativePart(dst);
@@ -116,7 +111,6 @@ public class AsyncDistributedFileSystem {
   public Future<Void> setPermission(Path p, final FsPermission permission)
       throws IOException {
     dfs.getFsStatistics().incrementWriteOps(1);
-    dfs.getDFSOpsCountStatistics().incrementOpCounter(OpType.SET_PERMISSION);
     final Path absPath = dfs.fixRelativePart(p);
     final boolean isAsync = Client.isAsynchronousMode();
     Client.setAsynchronousMode(true);
@@ -148,7 +142,6 @@ public class AsyncDistributedFileSystem {
     }
 
     dfs.getFsStatistics().incrementWriteOps(1);
-    dfs.getDFSOpsCountStatistics().incrementOpCounter(OpType.SET_OWNER);
     final Path absPath = dfs.fixRelativePart(p);
     final boolean isAsync = Client.isAsynchronousMode();
     Client.setAsynchronousMode(true);
@@ -159,56 +152,4 @@ public class AsyncDistributedFileSystem {
       Client.setAsynchronousMode(isAsync);
     }
   }
-
-  /**
-   * Fully replaces ACL of files and directories, discarding all existing
-   * entries.
-   *
-   * @param p
-   *          Path to modify
-   * @param aclSpec
-   *          List<AclEntry> describing modifications, must include entries for
-   *          user, group, and others for compatibility with permission bits.
-   * @throws IOException
-   *           if an ACL could not be modified
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<Void> setAcl(Path p, final List<AclEntry> aclSpec)
-      throws IOException {
-    dfs.getFsStatistics().incrementWriteOps(1);
-    dfs.getDFSOpsCountStatistics().incrementOpCounter(OpType.SET_ACL);
-    final Path absPath = dfs.fixRelativePart(p);
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().setAcl(dfs.getPathName(absPath), aclSpec);
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
-
-  /**
-   * Gets the ACL of a file or directory.
-   *
-   * @param p
-   *          Path to get
-   * @return AclStatus describing the ACL of the file or directory
-   * @throws IOException
-   *           if an ACL could not be read
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<AclStatus> getAclStatus(Path p) throws IOException {
-    final Path absPath = dfs.fixRelativePart(p);
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().getAclStatus(dfs.getPathName(absPath));
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b82c74b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 66ee42f..0ae4d70 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2529,7 +2529,4 @@ public class DistributedFileSystem extends FileSystem {
     return statistics;
   }
 
-  DFSOpsCountStatistics getDFSOpsCountStatistics() {
-    return storageStatistics;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b82c74b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 2373da7..939c1ac 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -72,7 +72,6 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import 
org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
-import 
org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
 import 
org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto;
 import 
org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto;
@@ -164,7 +163,7 @@ import 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Trunca
 import 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyRequestProto;
 import 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.*;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
 import 
org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import 
org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
 import 
org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
@@ -1347,12 +1346,7 @@ public class ClientNamenodeProtocolTranslatorPB 
implements
         .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec))
         .build();
     try {
-      if (Client.isAsynchronousMode()) {
-        rpcProxy.setAcl(null, req);
-        setAsyncReturnValue();
-      } else {
-        rpcProxy.setAcl(null, req);
-      }
+      rpcProxy.setAcl(null, req);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1363,25 +1357,7 @@ public class ClientNamenodeProtocolTranslatorPB 
implements
     GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      if (Client.isAsynchronousMode()) {
-        rpcProxy.getAclStatus(null, req);
-        final AsyncGet<Message, Exception> asyncReturnMessage
-            = ProtobufRpcEngine.getAsyncReturnMessage();
-        final AsyncGet<AclStatus, Exception> asyncGet =
-            new AsyncGet<AclStatus, Exception>() {
-              @Override
-              public AclStatus get(long timeout, TimeUnit unit)
-                  throws Exception {
-                return PBHelperClient
-                    .convert((GetAclStatusResponseProto) asyncReturnMessage
-                        .get(timeout, unit));
-              }
-            };
-        ASYNC_RETURN_VALUE.set(asyncGet);
-        return null;
-      } else {
-        return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
-      }
+      return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b82c74b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java
deleted file mode 100644
index 67262dd..0000000
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java
+++ /dev/null
@@ -1,310 +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;
-
-import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
-import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
-import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
-import static org.apache.hadoop.fs.permission.AclEntryType.MASK;
-import static org.apache.hadoop.fs.permission.AclEntryType.OTHER;
-import static org.apache.hadoop.fs.permission.AclEntryType.USER;
-import static org.apache.hadoop.fs.permission.FsAction.ALL;
-import static org.apache.hadoop.fs.permission.FsAction.NONE;
-import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
-import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.server.namenode.AclTestHelpers;
-import org.apache.hadoop.hdfs.server.namenode.FSAclBaseTest;
-import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-/**
- * Unit tests for asynchronous distributed filesystem.
- * */
-public class TestAsyncDFS {
-  public static final Log LOG = LogFactory.getLog(TestAsyncDFS.class);
-  private static final int NUM_TESTS = 1000;
-  private static final int NUM_NN_HANDLER = 10;
-  private static final int ASYNC_CALL_LIMIT = 100;
-
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private FileSystem fs;
-
-  @Before
-  public void setup() throws IOException {
-    conf = new HdfsConfiguration();
-    // explicitly turn on acl
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
-    // explicitly turn on ACL
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
-    // set the limit of max async calls
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
-        ASYNC_CALL_LIMIT);
-    // set server handlers
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY, NUM_NN_HANDLER);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
-    cluster.waitActive();
-    fs = FileSystem.get(conf);
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    if (fs != null) {
-      fs.close();
-      fs = null;
-    }
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  static class AclQueueEntry {
-    private final Object future;
-    private final Path path;
-    private final Boolean isSetAcl;
-
-    AclQueueEntry(final Object future, final Path path,
-        final Boolean isSetAcl) {
-      this.future = future;
-      this.path = path;
-      this.isSetAcl = isSetAcl;
-    }
-
-    public final Object getFuture() {
-      return future;
-    }
-
-    public final Path getPath() {
-      return path;
-    }
-
-    public final Boolean isSetAcl() {
-      return this.isSetAcl;
-    }
-  }
-
-  @Test(timeout=60000)
-  public void testBatchAsyncAcl() throws Exception {
-    final String basePath = "testBatchAsyncAcl";
-    final Path parent = new Path(String.format("/test/%s/", basePath));
-
-    AsyncDistributedFileSystem adfs = cluster.getFileSystem()
-        .getAsyncDistributedFileSystem();
-
-    // prepare test
-    int count = NUM_TESTS;
-    final Path[] paths = new Path[count];
-    for (int i = 0; i < count; i++) {
-      paths[i] = new Path(parent, "acl" + i);
-      FileSystem.mkdirs(fs, paths[i],
-          FsPermission.createImmutable((short) 0750));
-      assertTrue(fs.exists(paths[i]));
-      assertTrue(fs.getFileStatus(paths[i]).isDirectory());
-    }
-
-    final List<AclEntry> aclSpec = getAclSpec();
-    final AclEntry[] expectedAclSpec = getExpectedAclSpec();
-    Map<Integer, Future<Void>> setAclRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<AclStatus>> getAclRetFutures =
-        new HashMap<Integer, Future<AclStatus>>();
-    int start = 0, end = 0;
-    try {
-      // test setAcl
-      for (int i = 0; i < count; i++) {
-        for (;;) {
-          try {
-            Future<Void> retFuture = adfs.setAcl(paths[i], aclSpec);
-            setAclRetFutures.put(i, retFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            start = end;
-            end = i;
-            waitForAclReturnValues(setAclRetFutures, start, end);
-          }
-        }
-      }
-      waitForAclReturnValues(setAclRetFutures, end, count);
-
-      // test getAclStatus
-      start = 0;
-      end = 0;
-      for (int i = 0; i < count; i++) {
-        for (;;) {
-          try {
-            Future<AclStatus> retFuture = adfs.getAclStatus(paths[i]);
-            getAclRetFutures.put(i, retFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            start = end;
-            end = i;
-            waitForAclReturnValues(getAclRetFutures, start, end, paths,
-                expectedAclSpec);
-          }
-        }
-      }
-      waitForAclReturnValues(getAclRetFutures, end, count, paths,
-          expectedAclSpec);
-    } catch (Exception e) {
-      throw e;
-    }
-  }
-
-  private void waitForAclReturnValues(
-      final Map<Integer, Future<Void>> aclRetFutures, final int start,
-      final int end) throws InterruptedException, ExecutionException {
-    for (int i = start; i < end; i++) {
-      aclRetFutures.get(i).get();
-    }
-  }
-
-  private void waitForAclReturnValues(
-      final Map<Integer, Future<AclStatus>> aclRetFutures, final int start,
-      final int end, final Path[] paths, final AclEntry[] expectedAclSpec)
-      throws InterruptedException, ExecutionException, IOException {
-    for (int i = start; i < end; i++) {
-      AclStatus aclStatus = aclRetFutures.get(i).get();
-      verifyGetAcl(aclStatus, expectedAclSpec, paths[i]);
-    }
-  }
-
-  private void verifyGetAcl(final AclStatus aclStatus,
-      final AclEntry[] expectedAclSpec, final Path path) throws IOException {
-    if (aclStatus == null) {
-      return;
-    }
-
-    // verify permission and acl
-    AclEntry[] returned = aclStatus.getEntries().toArray(new AclEntry[0]);
-    assertArrayEquals(expectedAclSpec, returned);
-    assertPermission(path, (short) 010770);
-    FSAclBaseTest.assertAclFeature(cluster, path, true);
-  }
-
-  private List<AclEntry> getAclSpec() {
-    return Lists.newArrayList(
-        aclEntry(ACCESS, USER, ALL),
-        aclEntry(ACCESS, USER, "foo", ALL),
-        aclEntry(ACCESS, GROUP, READ_EXECUTE),
-        aclEntry(ACCESS, OTHER, NONE),
-        aclEntry(DEFAULT, USER, "foo", ALL));
-  }
-
-  private AclEntry[] getExpectedAclSpec() {
-    return new AclEntry[] {
-        aclEntry(ACCESS, USER, "foo", ALL),
-        aclEntry(ACCESS, GROUP, READ_EXECUTE),
-        aclEntry(DEFAULT, USER, ALL),
-        aclEntry(DEFAULT, USER, "foo", ALL),
-        aclEntry(DEFAULT, GROUP, READ_EXECUTE),
-        aclEntry(DEFAULT, MASK, ALL),
-        aclEntry(DEFAULT, OTHER, NONE) };
-  }
-
-  private void assertPermission(final Path pathToCheck, final short perm)
-      throws IOException {
-    AclTestHelpers.assertPermission(fs, pathToCheck, perm);
-  }
-
-  @Test(timeout=60000)
-  public void testAsyncAPIWithException() throws Exception {
-    String group1 = "group1";
-    String group2 = "group2";
-    String user1 = "user1";
-    UserGroupInformation ugi1;
-
-    // create fake mapping for the groups
-    Map<String, String[]> u2gMap = new HashMap<String, String[]>(1);
-    u2gMap.put(user1, new String[] {group1, group2});
-    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2gMap);
-
-    // Initiate all four users
-    ugi1 = UserGroupInformation.createUserForTesting(user1, new String[] {
-        group1, group2 });
-
-    final Path parent = new Path("/test/async_api_exception/");
-    final Path aclDir = new Path(parent, "aclDir");
-    fs.mkdirs(aclDir, FsPermission.createImmutable((short) 0770));
-
-    AsyncDistributedFileSystem adfs = ugi1
-        .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
-          @Override
-          public AsyncDistributedFileSystem run() throws Exception {
-            return cluster.getFileSystem().getAsyncDistributedFileSystem();
-          }
-        });
-
-    Future<Void> retFuture;
-    // test setAcl
-    try {
-      retFuture = adfs.setAcl(aclDir,
-          Lists.newArrayList(aclEntry(ACCESS, USER, ALL)));
-      retFuture.get();
-      fail("setAcl should fail with permission denied");
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, aclDir, user1);
-    }
-
-    // test getAclStatus
-    try {
-      Future<AclStatus> aclRetFuture = adfs.getAclStatus(aclDir);
-      aclRetFuture.get();
-      fail("getAclStatus should fail with permission denied");
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, aclDir, user1);
-    }
-  }
-
-  public static void checkPermissionDenied(final Exception e, final Path dir,
-      final String user) {
-    assertTrue(e.getCause() instanceof ExecutionException);
-    assertTrue("Permission denied messages must carry AccessControlException",
-        e.getMessage().contains("AccessControlException"));
-    assertTrue("Permission denied messages must carry the username", e
-        .getMessage().contains(user));
-    assertTrue("Permission denied messages must carry the name of the path",
-        e.getMessage().contains(dir.getName()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b82c74b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
index 03c8151..7539fbd 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
@@ -520,7 +520,7 @@ public class TestAsyncDFSRename {
       retFuture = adfs.rename(src, dst, Rename.OVERWRITE);
       retFuture.get();
     } catch (ExecutionException e) {
-      TestAsyncDFS.checkPermissionDenied(e, src, user1);
+      checkPermissionDenied(e, src, user1);
       assertTrue("Permission denied messages must carry the path parent", e
           .getMessage().contains(src.getParent().toUri().getPath()));
     }
@@ -530,7 +530,7 @@ public class TestAsyncDFSRename {
       retFuture = adfs.setPermission(src, fsPerm);
       retFuture.get();
     } catch (ExecutionException e) {
-      TestAsyncDFS.checkPermissionDenied(e, src, user1);
+      checkPermissionDenied(e, src, user1);
       assertTrue("Permission denied messages must carry the name of the path",
           e.getMessage().contains(src.getName()));
     }
@@ -539,7 +539,7 @@ public class TestAsyncDFSRename {
       retFuture = adfs.setOwner(src, "user1", "group2");
       retFuture.get();
     } catch (ExecutionException e) {
-      TestAsyncDFS.checkPermissionDenied(e, src, user1);
+      checkPermissionDenied(e, src, user1);
       assertTrue("Permission denied messages must carry the name of the path",
           e.getMessage().contains(src.getName()));
     } finally {
@@ -551,4 +551,13 @@ public class TestAsyncDFSRename {
       }
     }
   }
+
+  private void checkPermissionDenied(final Exception e, final Path dir,
+      final String user) {
+    assertTrue(e.getCause() instanceof ExecutionException);
+    assertTrue("Permission denied messages must carry AccessControlException",
+        e.getMessage().contains("AccessControlException"));
+    assertTrue("Permission denied messages must carry the username", e
+        .getMessage().contains(user));
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b82c74b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
index 216147a..002f7c0 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
@@ -1637,23 +1637,17 @@ public abstract class FSAclBaseTest {
     assertAclFeature(path, expectAclFeature);
   }
 
-  private static void assertAclFeature(Path pathToCheck,
-      boolean expectAclFeature) throws IOException {
-    assertAclFeature(cluster, pathToCheck, expectAclFeature);
-  }
-
   /**
    * Asserts whether or not the inode for a specific path has an AclFeature.
    *
-   * @param miniCluster the cluster into which the path resides
    * @param pathToCheck Path inode to check
    * @param expectAclFeature boolean true if an AclFeature must be present,
    *   false if an AclFeature must not be present
    * @throws IOException thrown if there is an I/O error
    */
-  public static void assertAclFeature(final MiniDFSCluster miniCluster,
-      Path pathToCheck, boolean expectAclFeature) throws IOException {
-    AclFeature aclFeature = getAclFeature(pathToCheck, miniCluster);
+  private static void assertAclFeature(Path pathToCheck,
+      boolean expectAclFeature) throws IOException {
+    AclFeature aclFeature = getAclFeature(pathToCheck, cluster);
     if (expectAclFeature) {
       assertNotNull(aclFeature);
       // Intentionally capturing a reference to the entries, not using nested


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

Reply via email to