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 ae...@apache.org on 2016/06/07 17:34:48 UTC

[01/47] hadoop git commit: HADOOP-13228. Add delegation token to the connection in DelegationTokenAuthenticator. Contributed by Xiao Chen.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-1312 7c79136bb -> 76a1391d5


HADOOP-13228. Add delegation token to the connection in DelegationTokenAuthenticator. Contributed by Xiao Chen.


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

Branch: refs/heads/HDFS-1312
Commit: 35356de1ba1cad0fa469ff546263290109c61b77
Parents: 5870611
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Jun 1 13:13:17 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Jun 1 13:13:17 2016 -0700

----------------------------------------------------------------------
 .../DelegationTokenAuthenticationHandler.java   |   7 ++
 .../web/DelegationTokenAuthenticator.java       |  19 ++++
 .../delegation/web/TestWebDelegationToken.java  | 114 ++++++++++++++++++-
 3 files changed, 137 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35356de1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
index 3f191de..95a849f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
@@ -51,6 +51,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * An {@link AuthenticationHandler} that implements Kerberos SPNEGO mechanism
@@ -78,6 +80,9 @@ import com.google.common.annotations.VisibleForTesting;
 public abstract class DelegationTokenAuthenticationHandler
     implements AuthenticationHandler {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DelegationTokenAuthenticationHandler.class);
+
   protected static final String TYPE_POSTFIX = "-dt";
 
   public static final String PREFIX = "delegation-token.";
@@ -327,6 +332,8 @@ public abstract class DelegationTokenAuthenticationHandler
       throws IOException, AuthenticationException {
     AuthenticationToken token;
     String delegationParam = getDelegationToken(request);
+    LOG.debug("Authenticating with delegationParam: {}, query string: {}",
+        delegationParam, request.getQueryString());
     if (delegationParam != null) {
       try {
         Token<AbstractDelegationTokenIdentifier> dt = new Token();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35356de1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
index 8a3a57f..46a0b1f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
@@ -121,6 +121,24 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     return hasDt;
   }
 
+  /**
+   * Append the delegation token to the request header if needed.
+   */
+  private void appendDelegationToken(final AuthenticatedURL.Token token,
+      final Token<?> dToken, final HttpURLConnection conn) throws IOException {
+    if (token.isSet()) {
+      LOG.debug("Auth token is set, not appending delegation token.");
+      return;
+    }
+    if (dToken == null) {
+      LOG.warn("Delegation token is null, cannot set on request header.");
+      return;
+    }
+    conn.setRequestProperty(
+        DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER,
+        dToken.encodeToUrlString());
+  }
+
   @Override
   public void authenticate(URL url, AuthenticatedURL.Token token)
       throws IOException, AuthenticationException {
@@ -283,6 +301,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     url = new URL(sb.toString());
     AuthenticatedURL aUrl = new AuthenticatedURL(this, connConfigurator);
     HttpURLConnection conn = aUrl.openConnection(url, token);
+    appendDelegationToken(token, dToken, conn);
     conn.setRequestMethod(operation.getHttpMethod());
     HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     if (hasResponse) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35356de1/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
index 73562b5..65c80a2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.security.token.delegation.web;
 
+import static org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator.DelegationTokenOperation;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.minikdc.MiniKdc;
@@ -30,6 +32,7 @@ import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHa
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.After;
 import org.junit.Assert;
@@ -41,6 +44,8 @@ import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
 import org.mortbay.jetty.servlet.FilterHolder;
 import org.mortbay.jetty.servlet.ServletHolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
@@ -60,8 +65,6 @@ import java.io.File;
 import java.io.IOException;
 import java.io.Writer;
 import java.net.HttpURLConnection;
-import java.net.InetAddress;
-import java.net.ServerSocket;
 import java.net.URL;
 import java.security.Principal;
 import java.security.PrivilegedActionException;
@@ -76,6 +79,9 @@ import java.util.UUID;
 import java.util.concurrent.Callable;
 
 public class TestWebDelegationToken {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestWebDelegationToken.class);
   private static final String OK_USER = "ok-user";
   private static final String FAIL_USER = "fail-user";
   private static final String FOO_USER = "foo";
@@ -111,7 +117,7 @@ public class TestWebDelegationToken {
       AuthenticationToken token = null;
       if (request.getParameter("authenticated") != null) {
         token = new AuthenticationToken(request.getParameter("authenticated"),
-            "U", "test");
+            "U", "unsupported type");
       } else {
         response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
         response.setHeader(KerberosAuthenticator.WWW_AUTHENTICATE, "dummy");
@@ -134,6 +140,32 @@ public class TestWebDelegationToken {
     }
   }
 
+  /**
+   * A dummy DelegationTokenAuthenticationHandler to verify that the request
+   * header contains delegation token.
+   */
+  public static class HeaderVerifyingDelegationTokenAuthenticationHandler
+      extends DummyDelegationTokenAuthenticationHandler {
+
+    @Override
+    public boolean managementOperation(AuthenticationToken token,
+        HttpServletRequest request, HttpServletResponse response)
+        throws IOException, AuthenticationException {
+      String op = ServletUtils.getParameter(request,
+          KerberosDelegationTokenAuthenticator.OP_PARAM);
+      if (op != null) {
+        DelegationTokenOperation dtOp = DelegationTokenOperation.valueOf(op);
+        if (dtOp == DelegationTokenOperation.RENEWDELEGATIONTOKEN
+            || dtOp == DelegationTokenOperation.CANCELDELEGATIONTOKEN) {
+          Assert.assertNotNull("Request header should have delegation token",
+              request.getHeader(
+                  DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER));
+        }
+      }
+      return super.managementOperation(token, request, response);
+    }
+  }
+
   public static class AFilter extends DelegationTokenAuthenticationFilter {
 
     @Override
@@ -146,6 +178,24 @@ public class TestWebDelegationToken {
     }
   }
 
+  /**
+   * A dummy DelegationTokenAuthenticationFilter that uses a
+   * {@link HeaderVerifyingDelegationTokenAuthenticationHandler} to verify that
+   * the request header contains delegation token.
+   */
+  public static class HeaderVerifyingFilter
+      extends DelegationTokenAuthenticationFilter {
+
+    @Override
+    protected Properties getConfiguration(String configPrefix,
+        FilterConfig filterConfig) {
+      Properties conf = new Properties();
+      conf.setProperty(AUTH_TYPE,
+          HeaderVerifyingDelegationTokenAuthenticationHandler.class.getName());
+      return conf;
+    }
+  }
+
   public static class PingServlet extends HttpServlet {
 
     @Override
@@ -203,6 +253,7 @@ public class TestWebDelegationToken {
   @After
   public void cleanUp() throws Exception {
     jetty.stop();
+    jetty = null;
 
     // resetting hadoop security to simple
     org.apache.hadoop.conf.Configuration conf =
@@ -427,6 +478,63 @@ public class TestWebDelegationToken {
     }
   }
 
+  @Test(timeout=120000)
+  public void testDelegationTokenAuthenticatorUsingDT() throws Exception {
+    Context context = new Context();
+    context.setContextPath("/foo");
+    jetty.setHandler(context);
+    context.addFilter(new FilterHolder(HeaderVerifyingFilter.class), "/*", 0);
+    context.addServlet(new ServletHolder(PingServlet.class), "/bar");
+
+    jetty.start();
+    final URL nonAuthURL = new URL(getJettyURL() + "/foo/bar");
+    URL authURL = new URL(getJettyURL() + "/foo/bar?authenticated=foo");
+    URL authURL2 = new URL(getJettyURL() + "/foo/bar?authenticated=bar");
+
+    DelegationTokenAuthenticatedURL.Token token =
+        new DelegationTokenAuthenticatedURL.Token();
+    final DelegationTokenAuthenticatedURL aUrl =
+        new DelegationTokenAuthenticatedURL();
+    aUrl.getDelegationToken(authURL, token, FOO_USER);
+    Assert.assertNotNull(token.getDelegationToken());
+    Assert.assertEquals(new Text("token-kind"),
+        token.getDelegationToken().getKind());
+
+    // Create a token that only has dt so that we can test ops when
+    // authenticating with a delegation token.
+    DelegationTokenAuthenticatedURL.Token dtOnlyToken =
+        new DelegationTokenAuthenticatedURL.Token();
+    dtOnlyToken.setDelegationToken(token.getDelegationToken());
+
+    /**
+     * We're using delegation token, so everything comes from that.
+     * {@link DelegationTokenAuthenticationHandler#authenticate}.
+     *
+     * This means that the special logic we injected at
+     * {@link DummyAuthenticationHandler#authenticate}
+     * (check "authenticated" and return 401) wouldn't work any more.
+     */
+
+    aUrl.getDelegationToken(authURL, dtOnlyToken, FOO_USER);
+    aUrl.renewDelegationToken(authURL, dtOnlyToken);
+    aUrl.renewDelegationToken(nonAuthURL, dtOnlyToken);
+    aUrl.renewDelegationToken(authURL2, dtOnlyToken);
+
+    // Verify that after cancelling, we can't renew.
+    // After cancelling, the dt on token will be set to null. Back it up here.
+    DelegationTokenAuthenticatedURL.Token cancelledToken =
+        new DelegationTokenAuthenticatedURL.Token();
+    cancelledToken.setDelegationToken(dtOnlyToken.getDelegationToken());
+    aUrl.cancelDelegationToken(authURL, dtOnlyToken);
+    try {
+      aUrl.renewDelegationToken(authURL, cancelledToken);
+      Assert.fail();
+    } catch (Exception ex) {
+      LOG.info("Intentional exception caught:", ex);
+      GenericTestUtils.assertExceptionContains("can't be found in cache", ex);
+    }
+  }
+
   private static class DummyDelegationTokenSecretManager
       extends AbstractDelegationTokenSecretManager<DelegationTokenIdentifier> {
 


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


[26/47] hadoop git commit: Revert "HDFS-10390. Implement asynchronous setAcl/getAclStatus for DistributedFileSystem. Contributed by Xiaobing Zhou"

Posted by ae...@apache.org.
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 <wa...@apache.org>
Authored: Fri Jun 3 18:09:14 2016 -0700
Committer: Andrew Wang <wa...@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-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[39/47] hadoop git commit: MAPREDUCE-5044. Have AM trigger jstack on task attempts that timeout before killing them. (Eric Payne and Gera Shegalov via mingma)

Posted by ae...@apache.org.
MAPREDUCE-5044. Have AM trigger jstack on task attempts that timeout before killing them. (Eric Payne and Gera Shegalov via mingma)


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

Branch: refs/heads/HDFS-1312
Commit: 4a1cedc010d3fa1d8ef3f2773ca12acadfee5ba5
Parents: 35f255b
Author: Ming Ma <mi...@apache.org>
Authored: Mon Jun 6 14:30:51 2016 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Mon Jun 6 14:30:51 2016 -0700

----------------------------------------------------------------------
 .../hadoop/mapred/LocalContainerLauncher.java   |  28 +++++
 .../v2/app/job/impl/TaskAttemptImpl.java        |   5 +-
 .../v2/app/launcher/ContainerLauncherEvent.java |  21 +++-
 .../v2/app/launcher/ContainerLauncherImpl.java  |  19 ++-
 .../v2/app/launcher/TestContainerLauncher.java  |  10 +-
 .../app/launcher/TestContainerLauncherImpl.java |   8 ++
 .../hadoop/mapred/ResourceMgrDelegate.java      |   5 +-
 .../hadoop/mapred/TestClientRedirect.java       |   2 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  | 119 +++++++++++++++++++
 .../yarn/api/ApplicationClientProtocol.java     |   2 +-
 .../yarn/api/ContainerManagementProtocol.java   |   5 +
 .../SignalContainerResponse.java                |   2 +-
 .../main/proto/applicationclient_protocol.proto |   2 +-
 .../proto/containermanagement_protocol.proto    |   1 +
 .../hadoop/yarn/client/api/YarnClient.java      |   2 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |   4 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   6 +-
 .../yarn/client/api/impl/TestYarnClient.java    |   4 +-
 .../yarn/api/ContainerManagementProtocolPB.java |   7 ++
 .../ApplicationClientProtocolPBClientImpl.java  |   4 +-
 ...ContainerManagementProtocolPBClientImpl.java |  19 +++
 .../ApplicationClientProtocolPBServiceImpl.java |   5 +-
 ...ontainerManagementProtocolPBServiceImpl.java |  20 ++++
 .../hadoop/yarn/TestContainerLaunchRPC.java     |  10 ++
 .../yarn/TestContainerResourceIncreaseRPC.java  |   8 ++
 .../java/org/apache/hadoop/yarn/TestRPC.java    |  10 ++
 .../containermanager/ContainerManagerImpl.java  |  38 ++++--
 .../amrmproxy/MockResourceManagerFacade.java    |   2 +-
 .../server/resourcemanager/ClientRMService.java |   2 +-
 .../yarn/server/resourcemanager/MockRM.java     |   6 +-
 .../server/resourcemanager/NodeManager.java     |   9 +-
 .../resourcemanager/TestAMAuthorization.java    |   8 ++
 .../TestApplicationMasterLauncher.java          |   8 ++
 .../resourcemanager/TestSignalContainer.java    |   2 +-
 34 files changed, 361 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
index da118c5..190d988 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
@@ -20,6 +20,10 @@ package org.apache.hadoop.mapred;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -255,6 +259,30 @@ public class LocalContainerLauncher extends AbstractService implements
 
         } else if (event.getType() == EventType.CONTAINER_REMOTE_CLEANUP) {
 
+          if (event.getDumpContainerThreads()) {
+            try {
+              // Construct full thread dump header
+              System.out.println(new java.util.Date());
+              RuntimeMXBean rtBean = ManagementFactory.getRuntimeMXBean();
+              System.out.println("Full thread dump " + rtBean.getVmName()
+                  + " (" + rtBean.getVmVersion()
+                  + " " + rtBean.getSystemProperties().get("java.vm.info")
+                  + "):\n");
+              // Dump threads' states and stacks
+              ThreadMXBean tmxBean = ManagementFactory.getThreadMXBean();
+              ThreadInfo[] tInfos = tmxBean.dumpAllThreads(
+                  tmxBean.isObjectMonitorUsageSupported(),
+                  tmxBean.isSynchronizerUsageSupported());
+              for (ThreadInfo ti : tInfos) {
+                System.out.println(ti.toString());
+              }
+            } catch (Throwable t) {
+              // Failure to dump stack shouldn't cause method failure.
+              System.out.println("Could not create full thread dump: "
+                  + t.getMessage());
+            }
+          }
+
           // cancel (and interrupt) the current running task associated with the
           // event
           TaskAttemptId taId = event.getTaskAttemptID();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index e03aafa..6ee8e00 100755
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -2115,7 +2115,7 @@ public abstract class TaskAttemptImpl implements
           taskAttempt.attemptId,
           taskAttempt.getAssignedContainerID(), taskAttempt.getAssignedContainerMgrAddress(),
           taskAttempt.container.getContainerToken(),
-          ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP));
+          ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP, false));
       taskAttempt.eventHandler.handle(new TaskTAttemptKilledEvent(
           taskAttempt.attemptId, false));
 
@@ -2179,7 +2179,8 @@ public abstract class TaskAttemptImpl implements
         taskAttempt.container.getId(), StringInterner
         .weakIntern(taskAttempt.container.getNodeId().toString()),
         taskAttempt.container.getContainerToken(),
-        ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP));
+        ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP,
+        event.getType() == TaskAttemptEventType.TA_TIMED_OUT));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherEvent.java
index c663566..9e4e9df 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherEvent.java
@@ -30,17 +30,29 @@ public class ContainerLauncherEvent
   private ContainerId containerID;
   private String containerMgrAddress;
   private Token containerToken;
+  private boolean dumpContainerThreads;
 
   public ContainerLauncherEvent(TaskAttemptId taskAttemptID, 
       ContainerId containerID,
       String containerMgrAddress,
       Token containerToken,
       ContainerLauncher.EventType type) {
+    this(taskAttemptID, containerID, containerMgrAddress, containerToken, type,
+        false);
+  }
+
+  public ContainerLauncherEvent(TaskAttemptId taskAttemptID,
+      ContainerId containerID,
+      String containerMgrAddress,
+      Token containerToken,
+      ContainerLauncher.EventType type,
+      boolean dumpContainerThreads) {
     super(type);
     this.taskAttemptID = taskAttemptID;
     this.containerID = containerID;
     this.containerMgrAddress = containerMgrAddress;
     this.containerToken = containerToken;
+    this.dumpContainerThreads = dumpContainerThreads;
   }
 
   public TaskAttemptId getTaskAttemptID() {
@@ -59,6 +71,10 @@ public class ContainerLauncherEvent
     return containerToken;
   }
 
+  public boolean getDumpContainerThreads() {
+    return dumpContainerThreads;
+  }
+
   @Override
   public String toString() {
     return super.toString() + " for container " + containerID + " taskAttempt "
@@ -77,6 +93,8 @@ public class ContainerLauncherEvent
         + ((containerToken == null) ? 0 : containerToken.hashCode());
     result = prime * result
         + ((taskAttemptID == null) ? 0 : taskAttemptID.hashCode());
+    result = prime * result
+        + (dumpContainerThreads ? 1 : 0);
     return result;
   }
 
@@ -109,7 +127,8 @@ public class ContainerLauncherEvent
         return false;
     } else if (!taskAttemptID.equals(other.taskAttemptID))
       return false;
-    return true;
+
+    return dumpContainerThreads == other.dumpContainerThreads;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
index 189e2ef..58fd7b5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -53,6 +54,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -191,9 +193,13 @@ public class ContainerLauncherImpl extends AbstractService implements
         }
       }
     }
-    
+
+    public void kill() {
+      kill(false);
+    }
+
     @SuppressWarnings("unchecked")
-    public synchronized void kill() {
+    public synchronized void kill(boolean dumpThreads) {
 
       if(this.state == ContainerState.PREP) {
         this.state = ContainerState.KILLED_BEFORE_LAUNCH;
@@ -204,6 +210,13 @@ public class ContainerLauncherImpl extends AbstractService implements
         try {
           proxy = getCMProxy(this.containerMgrAddress, this.containerID);
 
+          if (dumpThreads) {
+            final SignalContainerRequest request = SignalContainerRequest
+                .newInstance(containerID,
+                    SignalContainerCommand.OUTPUT_THREAD_DUMP);
+            proxy.getContainerManagementProtocol().signalToContainer(request);
+          }
+
           // kill the remote container if already launched
           List<ContainerId> ids = new ArrayList<ContainerId>();
           ids.add(this.containerID);
@@ -381,7 +394,7 @@ public class ContainerLauncherImpl extends AbstractService implements
         break;
 
       case CONTAINER_REMOTE_CLEANUP:
-        c.kill();
+        c.kill(event.getDumpContainerThreads());
         break;
 
       case CONTAINER_COMPLETED:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
index 6c3a4d6..f1c5b77 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
@@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -58,6 +57,8 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -73,6 +74,7 @@ import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
@@ -460,5 +462,11 @@ public class TestContainerLauncher {
           "Dummy function cause"));
       throw new IOException(e);
     }
+
+    @Override
+    public SignalContainerResponse signalToContainer(
+        SignalContainerRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
index 610448c..d04f08c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
@@ -50,6 +50,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequ
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
@@ -465,6 +467,12 @@ public class TestContainerLauncherImpl {
     @Override
     public void close() throws IOException {
     }
+
+    @Override
+    public SignalContainerResponse signalToContainer(
+        SignalContainerRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
   
   @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
index af18480..cc164fd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
@@ -496,8 +496,9 @@ public class ResourceMgrDelegate extends YarnClient {
   }
 
   @Override
-  public void signalContainer(ContainerId containerId, SignalContainerCommand command)
+  public void signalToContainer(ContainerId containerId,
+      SignalContainerCommand command)
       throws YarnException, IOException {
-    client.signalContainer(containerId, command);
+    client.signalToContainer(containerId, command);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
index bed7e75..255f998 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
@@ -481,7 +481,7 @@ public class TestClientRedirect {
     }
 
     @Override
-    public SignalContainerResponse signalContainer(
+    public SignalContainerResponse signalToContainer(
         SignalContainerRequest request) throws IOException {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
index ba05d9d..a6647f1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
@@ -24,6 +24,7 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.StringReader;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
@@ -980,6 +981,124 @@ public class TestMRJobs {
     _testDistributedCache(remoteJobJarPath.toUri().toString());
   }
 
+  @Test(timeout = 120000)
+  public void testThreadDumpOnTaskTimeout() throws IOException,
+      InterruptedException, ClassNotFoundException {
+    if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
+      LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
+          + " not found. Not running test.");
+      return;
+    }
+
+    final SleepJob sleepJob = new SleepJob();
+    final JobConf sleepConf = new JobConf(mrCluster.getConfig());
+    sleepConf.setLong(MRJobConfig.TASK_TIMEOUT, 3 * 1000L);
+    sleepConf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
+    sleepJob.setConf(sleepConf);
+    if (this instanceof TestUberAM) {
+      sleepConf.setInt(MRJobConfig.MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS,
+          30 * 1000);
+    }
+    // sleep for 10 seconds to trigger a kill with thread dump
+    final Job job = sleepJob.createJob(1, 0, 10 * 60 * 1000L, 1, 0L, 0);
+    job.setJarByClass(SleepJob.class);
+    job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
+    job.waitForCompletion(true);
+    final JobId jobId = TypeConverter.toYarn(job.getJobID());
+    final ApplicationId appID = jobId.getAppId();
+    int pollElapsed = 0;
+    while (true) {
+      Thread.sleep(1000);
+      pollElapsed += 1000;
+      if (TERMINAL_RM_APP_STATES.contains(mrCluster.getResourceManager()
+          .getRMContext().getRMApps().get(appID).getState())) {
+        break;
+      }
+      if (pollElapsed >= 60000) {
+        LOG.warn("application did not reach terminal state within 60 seconds");
+        break;
+      }
+    }
+
+    // Job finished, verify logs
+    //
+
+    final String appIdStr = appID.toString();
+    final String appIdSuffix = appIdStr.substring("application_".length(),
+        appIdStr.length());
+    final String containerGlob = "container_" + appIdSuffix + "_*_*";
+    final String syslogGlob = appIdStr
+        + Path.SEPARATOR + containerGlob
+        + Path.SEPARATOR + TaskLog.LogName.SYSLOG;
+    int numAppMasters = 0;
+    int numMapTasks = 0;
+
+    for (int i = 0; i < NUM_NODE_MGRS; i++) {
+      final Configuration nmConf = mrCluster.getNodeManager(i).getConfig();
+      for (String logDir :
+               nmConf.getTrimmedStrings(YarnConfiguration.NM_LOG_DIRS)) {
+        final Path absSyslogGlob =
+            new Path(logDir + Path.SEPARATOR + syslogGlob);
+        LOG.info("Checking for glob: " + absSyslogGlob);
+        for (FileStatus syslog : localFs.globStatus(absSyslogGlob)) {
+          boolean foundAppMaster = false;
+          boolean foundThreadDump = false;
+
+          // Determine the container type
+          final BufferedReader syslogReader = new BufferedReader(
+              new InputStreamReader(localFs.open(syslog.getPath())));
+          try {
+            for (String line; (line = syslogReader.readLine()) != null; ) {
+              if (line.contains(MRAppMaster.class.getName())) {
+                foundAppMaster = true;
+                break;
+              }
+            }
+          } finally {
+            syslogReader.close();
+          }
+
+          // Check for thread dump in stdout
+          final Path stdoutPath = new Path(syslog.getPath().getParent(),
+              TaskLog.LogName.STDOUT.toString());
+          final BufferedReader stdoutReader = new BufferedReader(
+              new InputStreamReader(localFs.open(stdoutPath)));
+          try {
+            for (String line; (line = stdoutReader.readLine()) != null; ) {
+              if (line.contains("Full thread dump")) {
+                foundThreadDump = true;
+                break;
+              }
+            }
+          } finally {
+            stdoutReader.close();
+          }
+
+          if (foundAppMaster) {
+            numAppMasters++;
+            if (this instanceof TestUberAM) {
+              Assert.assertTrue("No thread dump", foundThreadDump);
+            } else {
+              Assert.assertFalse("Unexpected thread dump", foundThreadDump);
+            }
+          } else {
+            numMapTasks++;
+            Assert.assertTrue("No thread dump", foundThreadDump);
+          }
+        }
+      }
+    }
+
+    // Make sure we checked non-empty set
+    //
+    Assert.assertEquals("No AppMaster log found!", 1, numAppMasters);
+    if (sleepConf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false)) {
+      Assert.assertSame("MapTask log with uber found!", 0, numMapTasks);
+    } else {
+      Assert.assertSame("No MapTask log found!", 1, numMapTasks);
+    }
+  }
+
   private Path createTempFile(String filename, String contents)
       throws IOException {
     Path path = new Path(TEST_ROOT_DIR, filename);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
index 82b2708..8ee43fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
@@ -563,7 +563,7 @@ public interface ApplicationClientProtocol extends ApplicationBaseProtocol {
    */
   @Public
   @Unstable
-  public SignalContainerResponse signalContainer(
+  SignalContainerResponse signalToContainer(
       SignalContainerRequest request) throws YarnException,
       IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
index 43e1d4c..c6c0de5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequ
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -194,4 +196,7 @@ public interface ContainerManagementProtocol {
   IncreaseContainersResourceResponse increaseContainersResource(
       IncreaseContainersResourceRequest request) throws YarnException,
       IOException;
+
+  SignalContainerResponse signalToContainer(SignalContainerRequest request)
+      throws YarnException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SignalContainerResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SignalContainerResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SignalContainerResponse.java
index 0d773b9..6291dd7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SignalContainerResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/SignalContainerResponse.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
  *
  * <p>Currently it's empty.</p>
  *
- * @see ApplicationClientProtocol#signalContainer(SignalContainerRequest)
+ * @see ApplicationClientProtocol#signalToContainer(SignalContainerRequest)
  */
 @Public
 @Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
index 7046b24..f1c3839 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
@@ -59,5 +59,5 @@ service ApplicationClientProtocolService {
   rpc getLabelsToNodes (GetLabelsToNodesRequestProto) returns (GetLabelsToNodesResponseProto);
   rpc getClusterNodeLabels (GetClusterNodeLabelsRequestProto) returns (GetClusterNodeLabelsResponseProto);
   rpc updateApplicationPriority (UpdateApplicationPriorityRequestProto) returns (UpdateApplicationPriorityResponseProto);
-  rpc signalContainer(SignalContainerRequestProto) returns (SignalContainerResponseProto);
+  rpc signalToContainer(SignalContainerRequestProto) returns (SignalContainerResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
index f06f6cb..fc00505 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
@@ -35,4 +35,5 @@ service ContainerManagementProtocolService {
   rpc stopContainers(StopContainersRequestProto) returns (StopContainersResponseProto);
   rpc getContainerStatuses(GetContainerStatusesRequestProto) returns (GetContainerStatusesResponseProto);
   rpc increaseContainersResource(IncreaseContainersResourceRequestProto) returns (IncreaseContainersResourceResponseProto);
+  rpc signalToContainer(SignalContainerRequestProto) returns (SignalContainerResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
index 930e983..218bb34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
@@ -777,6 +777,6 @@ public abstract class YarnClient extends AbstractService {
    * @throws YarnException
    * @throws IOException
    */
-  public abstract void signalContainer(ContainerId containerId,
+  public abstract void signalToContainer(ContainerId containerId,
       SignalContainerCommand command) throws YarnException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index 36e6dfe..ae8716e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -858,12 +858,12 @@ public class YarnClientImpl extends YarnClient {
   }
 
   @Override
-  public void signalContainer(ContainerId containerId,
+  public void signalToContainer(ContainerId containerId,
       SignalContainerCommand command)
           throws YarnException, IOException {
     LOG.info("Signalling container " + containerId + " with command " + command);
     SignalContainerRequest request =
         SignalContainerRequest.newInstance(containerId, command);
-    rmClient.signalContainer(request);
+    rmClient.signalToContainer(request);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index caa4d46..d9e9fa6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -284,7 +284,7 @@ public class ApplicationCLI extends YarnCLI {
       if (signalArgs.length == 2) {
         command = SignalContainerCommand.valueOf(signalArgs[1]);
       }
-      signalContainer(containerId, command);
+      signalToContainer(containerId, command);
     } else {
       syserr.println("Invalid Command Usage : ");
       printUsage(title, opts);
@@ -299,11 +299,11 @@ public class ApplicationCLI extends YarnCLI {
    * @param command the signal command
    * @throws YarnException
    */
-  private void signalContainer(String containerIdStr,
+  private void signalToContainer(String containerIdStr,
       SignalContainerCommand command) throws YarnException, IOException {
     ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
     sysout.println("Signalling container " + containerIdStr);
-    client.signalContainer(containerId, command);
+    client.signalToContainer(containerId, command);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index 9ce7388..360ff99 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -1689,11 +1689,11 @@ public class TestYarnClient {
         applicationId, 1);
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     SignalContainerCommand command = SignalContainerCommand.OUTPUT_THREAD_DUMP;
-    client.signalContainer(containerId, command);
+    client.signalToContainer(containerId, command);
     final ArgumentCaptor<SignalContainerRequest> signalReqCaptor =
         ArgumentCaptor.forClass(SignalContainerRequest.class);
     verify(((MockYarnClient) client).getRMClient())
-        .signalContainer(signalReqCaptor.capture());
+        .signalToContainer(signalReqCaptor.capture());
     SignalContainerRequest request = signalReqCaptor.getValue();
     Assert.assertEquals(containerId, request.getContainerId());
     Assert.assertEquals(command, request.getCommand());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java
index 82e9ad0..46b4148 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java
@@ -21,6 +21,11 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.yarn.proto.ContainerManagementProtocol.ContainerManagementProtocolService;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerResponseProto;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
 
 @Private
 @Unstable
@@ -29,4 +34,6 @@ import org.apache.hadoop.yarn.proto.ContainerManagementProtocol.ContainerManagem
     protocolVersion = 1)
 public interface ContainerManagementProtocolPB extends ContainerManagementProtocolService.BlockingInterface {
 
+  SignalContainerResponseProto signalToContainer(RpcController arg0,
+      SignalContainerRequestProto proto) throws ServiceException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
index 4d65425..2d755a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
@@ -588,13 +588,13 @@ public class ApplicationClientProtocolPBClientImpl implements ApplicationClientP
   }
 
   @Override
-  public SignalContainerResponse signalContainer(
+  public SignalContainerResponse signalToContainer(
       SignalContainerRequest request) throws YarnException, IOException {
     YarnServiceProtos.SignalContainerRequestProto requestProto =
         ((SignalContainerRequestPBImpl) request).getProto();
     try {
       return new SignalContainerResponsePBImpl(
-          proxy.signalContainer(null, requestProto));
+          proxy.signalToContainer(null, requestProto));
     } catch (ServiceException e) {
       RPCUtil.unwrapAndThrowException(e);
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
index ce18bde..dfc672e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
@@ -34,6 +34,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequ
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
@@ -42,6 +44,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersReso
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
@@ -50,6 +54,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
@@ -148,4 +153,18 @@ public class ContainerManagementProtocolPBClientImpl implements ContainerManagem
       return null;
     }
   }
+
+  @Override
+  public SignalContainerResponse signalToContainer(
+      SignalContainerRequest request) throws YarnException, IOException {
+    SignalContainerRequestProto requestProto =
+        ((SignalContainerRequestPBImpl) request).getProto();
+    try {
+      return new SignalContainerResponsePBImpl(
+          proxy.signalToContainer(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
index 4923794..300ef57 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
@@ -596,11 +596,12 @@ public class ApplicationClientProtocolPBServiceImpl implements ApplicationClient
   }
 
   @Override
-  public SignalContainerResponseProto signalContainer(RpcController controller,
+  public SignalContainerResponseProto signalToContainer(
+      RpcController controller,
       YarnServiceProtos.SignalContainerRequestProto proto) throws ServiceException {
     SignalContainerRequestPBImpl request = new SignalContainerRequestPBImpl(proto);
     try {
-      SignalContainerResponse response = real.signalContainer(request);
+      SignalContainerResponse response = real.signalToContainer(request);
       return ((SignalContainerResponsePBImpl)response).getProto();
     } catch (YarnException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
index 7626441..1744f33 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
@@ -25,12 +25,15 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
@@ -40,6 +43,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResource
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
@@ -116,4 +121,19 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public SignalContainerResponseProto signalToContainer(RpcController arg0,
+      SignalContainerRequestProto proto) throws ServiceException {
+    final SignalContainerRequestPBImpl request =
+        new SignalContainerRequestPBImpl(proto);
+    try {
+      final SignalContainerResponse response = real.signalToContainer(request);
+      return ((SignalContainerResponsePBImpl)response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
index 0a19783..9ac441a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequ
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -174,5 +176,13 @@ public class TestContainerLaunchRPC {
         IncreaseContainersResourceRequest request) throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public SignalContainerResponse signalToContainer(
+        SignalContainerRequest request) throws YarnException, IOException {
+      final Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new YarnException(e);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
index 50ff1e0..3a79ba5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
@@ -158,5 +160,11 @@ public class TestContainerResourceIncreaseRPC {
       }
       throw new YarnException("Shouldn't happen!!");
     }
+
+    @Override
+    public SignalContainerResponse signalToContainer(
+        SignalContainerRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
index e718661..c133070 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResp
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -227,6 +229,14 @@ public class TestRPC {
         IncreaseContainersResourceRequest request) throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public SignalContainerResponse signalToContainer(
+        SignalContainerRequest request) throws YarnException {
+      final Exception e = new Exception(EXCEPTION_MSG,
+          new Exception(EXCEPTION_CAUSE));
+      throw new YarnException(e);
+    }
   }
 
   public static ContainerTokenIdentifier newContainerTokenIdentifier(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index d7800a8..1e3b854 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -60,11 +60,13 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerResponsePBImpl;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
@@ -147,6 +149,7 @@ import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
+
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class ContainerManagerImpl extends CompositeService implements
@@ -1380,16 +1383,7 @@ public class ContainerManagerImpl extends CompositeService implements
           (CMgrSignalContainersEvent) event;
       for (SignalContainerRequest request : containersSignalEvent
           .getContainersToSignal()) {
-        ContainerId containerId = request.getContainerId();
-        Container container = this.context.getContainers().get(containerId);
-        if (container != null) {
-          LOG.info(containerId + " signal request by ResourceManager.");
-          this.dispatcher.getEventHandler().handle(
-              new SignalContainersLauncherEvent(container,
-                  request.getCommand()));
-        } else {
-          LOG.info("Container " + containerId + " no longer exists");
-        }
+        internalSignalToContainer(request, "ResourceManager");
       }
       break;
     default:
@@ -1440,4 +1434,28 @@ public class ContainerManagerImpl extends CompositeService implements
   public void updateQueuingLimit(ContainerQueuingLimit queuingLimit) {
     LOG.trace("Implementation does not support queuing of Containers !!");
   }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public SignalContainerResponse signalToContainer(
+      SignalContainerRequest request) throws YarnException, IOException {
+    internalSignalToContainer(request, "Application Master");
+    return new SignalContainerResponsePBImpl();
+  }
+
+  @SuppressWarnings("unchecked")
+  private void internalSignalToContainer(SignalContainerRequest request,
+      String sentBy) {
+    ContainerId containerId = request.getContainerId();
+    Container container = this.context.getContainers().get(containerId);
+    if (container != null) {
+      LOG.info(containerId + " signal request " + request.getCommand()
+            + " by " + sentBy);
+      this.dispatcher.getEventHandler().handle(
+          new SignalContainersLauncherEvent(container,
+              request.getCommand()));
+    } else {
+      LOG.info("Container " + containerId + " no longer exists");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
index 6420cb0..0652e96 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
@@ -486,7 +486,7 @@ public class MockResourceManagerFacade implements
   }
 
   @Override
-  public SignalContainerResponse signalContainer(
+  public SignalContainerResponse signalToContainer(
       SignalContainerRequest request) throws IOException {
 return null;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index aeeea2b..fc5aec8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -1635,7 +1635,7 @@ public class ClientRMService extends AbstractService implements
    */
   @SuppressWarnings("unchecked")
   @Override
-  public SignalContainerResponse signalContainer(
+  public SignalContainerResponse signalToContainer(
       SignalContainerRequest request) throws YarnException, IOException {
     ContainerId containerId = request.getContainerId();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index dc749be..f7a210c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -1005,11 +1005,11 @@ public class MockRM extends ResourceManager {
     return activeServices;
   }
 
-  public void signalContainer(ContainerId containerId, SignalContainerCommand command)
-      throws Exception {
+  public void signalToContainer(ContainerId containerId,
+      SignalContainerCommand command) throws Exception {
     ApplicationClientProtocol client = getClientRMService();
     SignalContainerRequest req =
         SignalContainerRequest.newInstance(containerId, command);
-    client.signalContainer(req);
+    client.signalToContainer(req);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
index 8c0a907..5afec84 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
@@ -28,13 +28,14 @@ import java.util.Map;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -317,4 +318,10 @@ public class NodeManager implements ContainerManagementProtocol {
     nodeStatus.setNodeHealthStatus(nodeHealthStatus);
     return nodeStatus;
   }
+
+  @Override
+  public synchronized SignalContainerResponse signalToContainer(
+      SignalContainerRequest request) throws YarnException, IOException {
+    throw new YarnException("Not supported yet!");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
index 2787f1e..c51cd87 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
@@ -46,6 +46,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
@@ -138,6 +140,12 @@ public class TestAMAuthorization {
       credentials.readTokenStorageStream(buf);
       return credentials;
     }
+
+    @Override
+    public SignalContainerResponse signalToContainer(
+        SignalContainerRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   public static class MockRMWithAMS extends MockRMWithCustomAMLauncher {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
index 13772dd..3482af2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
@@ -37,6 +37,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequ
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -148,6 +150,12 @@ public class TestApplicationMasterLauncher {
             throws YarnException {
       return null;
     }
+
+    @Override
+    public SignalContainerResponse signalToContainer(
+        SignalContainerRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a1cedc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java
index 39cec99..692924c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestSignalContainer.java
@@ -83,7 +83,7 @@ public class TestSignalContainer {
     Assert.assertEquals(request, contReceived);
 
     for(Container container : conts) {
-      rm.signalContainer(container.getId(),
+      rm.signalToContainer(container.getId(),
           SignalContainerCommand.OUTPUT_THREAD_DUMP);
     }
 


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


[12/47] hadoop git commit: HADOOP-13171. Add StorageStatistics to S3A; instrument some more operations. Contributed by Steve Loughran.

Posted by ae...@apache.org.
HADOOP-13171. Add StorageStatistics to S3A; instrument some more operations. Contributed by Steve Loughran.


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

Branch: refs/heads/HDFS-1312
Commit: c58a59f7081d55dd2108545ebf9ee48cf43ca944
Parents: 97e2449
Author: Chris Nauroth <cn...@apache.org>
Authored: Fri Jun 3 08:55:33 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Fri Jun 3 08:55:33 2016 -0700

----------------------------------------------------------------------
 .../hadoop/fs/contract/ContractTestUtils.java   | 420 +++++++++++++++
 .../fs/s3a/ProgressableProgressListener.java    |  94 ++++
 .../hadoop/fs/s3a/S3AFastOutputStream.java      |  65 +--
 .../org/apache/hadoop/fs/s3a/S3AFileStatus.java |   7 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 507 ++++++++++++++-----
 .../hadoop/fs/s3a/S3AInstrumentation.java       | 218 +++++---
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |  98 +---
 .../hadoop/fs/s3a/S3AStorageStatistics.java     | 104 ++++
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  48 ++
 .../org/apache/hadoop/fs/s3a/Statistic.java     | 143 ++++++
 .../src/site/markdown/tools/hadoop-aws/index.md |  12 +-
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  | 153 ++++++
 .../hadoop/fs/s3a/TestS3AFileOperationCost.java | 191 +++++++
 .../hadoop/fs/s3a/scale/S3AScaleTestBase.java   | 154 ++----
 .../fs/s3a/scale/TestS3ADeleteManyFiles.java    |  10 +-
 .../s3a/scale/TestS3ADirectoryPerformance.java  | 189 +++++++
 .../scale/TestS3AInputStreamPerformance.java    |   6 +-
 .../src/test/resources/log4j.properties         |   4 +-
 18 files changed, 1984 insertions(+), 439 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
index 6343d40..20ba075 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
@@ -22,7 +22,9 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.internal.AssumptionViolatedException;
@@ -34,8 +36,14 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
 import java.util.Properties;
+import java.util.Set;
 import java.util.UUID;
 
 /**
@@ -892,4 +900,416 @@ public class ContractTestUtils extends Assert {
       fs.delete(objectPath, false);
     }
   }
+
+  /**
+   * Make times more readable, by adding a "," every three digits.
+   * @param nanos nanos or other large number
+   * @return a string for logging
+   */
+  public static String toHuman(long nanos) {
+    return String.format(Locale.ENGLISH, "%,d", nanos);
+  }
+
+  /**
+   * Log the bandwidth of a timer as inferred from the number of
+   * bytes processed.
+   * @param timer timer
+   * @param bytes bytes processed in the time period
+   */
+  public static void bandwidth(NanoTimer timer, long bytes) {
+    LOG.info("Bandwidth = {}  MB/S",
+        timer.bandwidthDescription(bytes));
+  }
+
+  /**
+   * Work out the bandwidth in MB/s.
+   * @param bytes bytes
+   * @param durationNS duration in nanos
+   * @return the number of megabytes/second of the recorded operation
+   */
+  public static double bandwidthMBs(long bytes, long durationNS) {
+    return (bytes * 1000.0) / durationNS;
+  }
+
+  /**
+   * Recursively create a directory tree.
+   * Return the details about the created tree. The files and directories
+   * are those created under the path, not the base directory created. That
+   * is retrievable via {@link TreeScanResults#getBasePath()}.
+   * @param fs filesystem
+   * @param current parent dir
+   * @param depth depth of directory tree
+   * @param width width: subdirs per entry
+   * @param files number of files per entry
+   * @param filesize size of files to create in bytes.
+   * @return the details about the created tree.
+   * @throws IOException IO Problems
+   */
+  public static TreeScanResults createSubdirs(FileSystem fs,
+      Path current,
+      int depth,
+      int width,
+      int files,
+      int filesize) throws IOException {
+    return createSubdirs(fs, current, depth, width, files,
+        filesize, "dir-", "file-", "0");
+  }
+
+  /**
+   * Recursively create a directory tree.
+   * @param fs filesystem
+   * @param current the current dir in the walk
+   * @param depth depth of directory tree
+   * @param width width: subdirs per entry
+   * @param files number of files per entry
+   * @param filesize size of files to create in bytes.
+   * @param dirPrefix prefix for directory entries
+   * @param filePrefix prefix for file entries
+   * @param marker string which is slowly built up to uniquely name things
+   * @return the details about the created tree.
+   * @throws IOException IO Problems
+   */
+  public static TreeScanResults createSubdirs(FileSystem fs,
+      Path current,
+      int depth,
+      int width,
+      int files,
+      int filesize,
+      String dirPrefix,
+      String filePrefix,
+      String marker) throws IOException {
+    fs.mkdirs(current);
+    TreeScanResults results = new TreeScanResults(current);
+    if (depth > 0) {
+      byte[] data = dataset(filesize, 'a', 'z');
+      for (int i = 0; i < files; i++) {
+        String name = String.format("%s-%s-%04d.txt", filePrefix, marker, i);
+        Path path = new Path(current, name);
+        createFile(fs, path, true, data);
+        results.add(fs, path);
+      }
+      for (int w = 0; w < width; w++) {
+        String marker2 = String.format("%s-%04d", marker, w);
+        Path child = new Path(current, dirPrefix + marker2);
+        results.add(createSubdirs(fs, child, depth - 1, width, files,
+            filesize, dirPrefix, filePrefix, marker2));
+        results.add(fs, child);
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Predicate to determine if two lists are equivalent, that is, they
+   * contain the same entries.
+   * @param left first collection of paths
+   * @param right second collection of paths
+   * @return true if all entries are in each collection of path.
+   */
+  public static boolean collectionsEquivalent(Collection<Path> left,
+      Collection<Path> right) {
+    Set<Path> leftSet = new HashSet<>(left);
+    Set<Path> rightSet = new HashSet<>(right);
+    return leftSet.containsAll(right) && rightSet.containsAll(left);
+  }
+
+  /**
+   * Predicate to determine if two lists are equivalent, that is, they
+   * contain the same entries.
+   * @param left first collection of paths
+   * @param right second collection of paths
+   * @return true if all entries are in each collection of path.
+   */
+  public static boolean collectionsEquivalentNoDuplicates(Collection<Path> left,
+      Collection<Path> right) {
+    return collectionsEquivalent(left, right) &&
+        !containsDuplicates(left) && !containsDuplicates(right);
+  }
+
+
+  /**
+   * Predicate to test for a collection of paths containing duplicate entries.
+   * @param paths collection of paths
+   * @return true if there are duplicates.
+   */
+  public static boolean containsDuplicates(Collection<Path> paths) {
+    return new HashSet<>(paths).size() != paths.size();
+  }
+
+  /**
+   * Recursively list all entries, with a depth first traversal of the
+   * directory tree.
+   * @param path path
+   * @return the number of entries listed
+   * @throws IOException IO problems
+   */
+  public static TreeScanResults treeWalk(FileSystem fs, Path path)
+      throws IOException {
+    TreeScanResults dirsAndFiles = new TreeScanResults();
+
+    FileStatus[] statuses = fs.listStatus(path);
+    for (FileStatus status : statuses) {
+      LOG.info("{}{}", status.getPath(), status.isDirectory() ? "*" : "");
+    }
+    for (FileStatus status : statuses) {
+      dirsAndFiles.add(status);
+      if (status.isDirectory()) {
+        dirsAndFiles.add(treeWalk(fs, status.getPath()));
+      }
+    }
+    return dirsAndFiles;
+  }
+
+  /**
+   * Results of recursive directory creation/scan operations.
+   */
+  public static final class TreeScanResults {
+
+    private Path basePath;
+    private final List<Path> files = new ArrayList<>();
+    private final List<Path> directories = new ArrayList<>();
+    private final List<Path> other = new ArrayList<>();
+
+
+    public TreeScanResults() {
+    }
+
+    public TreeScanResults(Path basePath) {
+      this.basePath = basePath;
+    }
+
+    /**
+     * Build from a located file status iterator.
+     * @param results results of the listFiles/listStatus call.
+     * @throws IOException IO problems during the iteration.
+     */
+    public TreeScanResults(RemoteIterator<LocatedFileStatus> results)
+        throws IOException {
+      while (results.hasNext()) {
+        add(results.next());
+      }
+    }
+
+    /**
+     * Construct results from an array of statistics.
+     * @param stats statistics array. Must not be null.
+     */
+    public TreeScanResults(FileStatus[] stats) {
+      assertNotNull("Null file status array", stats);
+      for (FileStatus stat : stats) {
+        add(stat);
+      }
+    }
+
+    /**
+     * Add all paths in the other set of results to this instance.
+     * @param that the other instance
+     * @return this instance
+     */
+    public TreeScanResults add(TreeScanResults that) {
+      files.addAll(that.files);
+      directories.addAll(that.directories);
+      other.addAll(that.other);
+      return this;
+    }
+
+    /**
+     * Increment the counters based on the file status.
+     * @param status path status to count.
+     */
+    public void add(FileStatus status) {
+      if (status.isFile()) {
+        files.add(status.getPath());
+      } else if (status.isDirectory()) {
+        directories.add(status.getPath());
+      } else {
+        other.add(status.getPath());
+      }
+    }
+
+    public void add(FileSystem fs, Path path) throws IOException {
+      add(fs.getFileStatus(path));
+    }
+
+    @Override
+    public String toString() {
+      return String.format("%d director%s and %d file%s",
+          getDirCount(),
+          getDirCount() == 1 ? "y" : "ies",
+          getFileCount(),
+          getFileCount() == 1 ? "" : "s");
+    }
+
+    /**
+     * Assert that the state of a listing has the specific number of files,
+     * directories and other entries. The error text will include
+     * the {@code text} param, the field in question, and the entire object's
+     * string value.
+     * @param text text prefix for assertions.
+     * @param f file count
+     * @param d expected directory count
+     * @param o expected other entries.
+     */
+    public void assertSizeEquals(String text, long f, long d, long o) {
+      String self = toString();
+      Assert.assertEquals(text + ": file count in " + self,
+          f, getFileCount());
+      Assert.assertEquals(text + ": directory count in " + self,
+          d, getDirCount());
+      Assert.assertEquals(text + ": 'other' count in " + self,
+          o, getOtherCount());
+    }
+
+    /**
+     * Assert that the trees are equivalent: that every list matches (and
+     * that neither has any duplicates).
+     * @param that the other entry
+     */
+    public void assertEquivalent(TreeScanResults that) {
+      String details = "this= " + this + "; that=" + that;
+      assertFieldsEquivalent("files", that, files, that.files);
+      assertFieldsEquivalent("directories", that,
+          directories, that.directories);
+      assertFieldsEquivalent("other", that, other, that.other);
+    }
+
+    /**
+     * Assert that a field in two instances are equivalent.
+     * @param fieldname field name for error messages
+     * @param that the other instance to scan
+     * @param ours our field's contents
+     * @param theirs the other instance's field constants
+     */
+    public void assertFieldsEquivalent(String fieldname,
+        TreeScanResults that,
+        List<Path> ours, List<Path> theirs) {
+      assertFalse("Duplicate  " + files + " in " + this,
+          containsDuplicates(ours));
+      assertFalse("Duplicate  " + files + " in other " + that,
+          containsDuplicates(theirs));
+      assertTrue(fieldname + " mismatch: between {" + this + "}" +
+              " and {" + that + "}",
+          collectionsEquivalent(files, that.files));
+    }
+
+    public List<Path> getFiles() {
+      return files;
+    }
+
+    public List<Path> getDirectories() {
+      return directories;
+    }
+
+    public List<Path> getOther() {
+      return other;
+    }
+
+    public Path getBasePath() {
+      return basePath;
+    }
+
+    public long getFileCount() {
+      return files.size();
+    }
+
+    public long getDirCount() {
+      return directories.size();
+    }
+
+    public long getOtherCount() {
+      return other.size();
+    }
+
+    /**
+     * Total count of entries.
+     * @return the total number of entries
+     */
+    public long totalCount() {
+      return getFileCount() + getDirCount() + getOtherCount();
+    }
+
+  }
+
+  /**
+   * A simple class for timing operations in nanoseconds, and for
+   * printing some useful results in the process.
+   */
+  public static final class NanoTimer {
+    private final long startTime;
+    private long endTime;
+
+    public NanoTimer() {
+      startTime = now();
+    }
+
+    /**
+     * End the operation.
+     * @return the duration of the operation
+     */
+    public long end() {
+      endTime = now();
+      return duration();
+    }
+
+    /**
+     * End the operation; log the duration.
+     * @param format message
+     * @param args any arguments
+     * @return the duration of the operation
+     */
+    public long end(String format, Object... args) {
+      long d = end();
+      LOG.info("Duration of {}: {} nS",
+          String.format(format, args), toHuman(d));
+      return d;
+    }
+
+    public long now() {
+      return System.nanoTime();
+    }
+
+    public long duration() {
+      return endTime - startTime;
+    }
+
+    public double bandwidth(long bytes) {
+      return bandwidthMBs(bytes, duration());
+    }
+
+    /**
+     * Bandwidth as bytes per second.
+     * @param bytes bytes in
+     * @return the number of bytes per second this operation timed.
+     */
+    public double bandwidthBytes(long bytes) {
+      return (bytes * 1.0) / duration();
+    }
+
+    /**
+     * How many nanoseconds per IOP, byte, etc.
+     * @param operations operations processed in this time period
+     * @return the nanoseconds it took each byte to be processed
+     */
+    public long nanosPerOperation(long operations) {
+      return duration() / operations;
+    }
+
+    /**
+     * Get a description of the bandwidth, even down to fractions of
+     * a MB.
+     * @param bytes bytes processed
+     * @return bandwidth
+     */
+    public String bandwidthDescription(long bytes) {
+      return String.format("%,.6f", bandwidth(bytes));
+    }
+
+    public long getStartTime() {
+      return startTime;
+    }
+
+    public long getEndTime() {
+      return endTime;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
new file mode 100644
index 0000000..0ce022a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
@@ -0,0 +1,94 @@
+/*
+ * 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.fs.s3a;
+
+import com.amazonaws.event.ProgressEvent;
+import com.amazonaws.event.ProgressEventType;
+import com.amazonaws.event.ProgressListener;
+import com.amazonaws.services.s3.transfer.Upload;
+import org.apache.hadoop.util.Progressable;
+import org.slf4j.Logger;
+
+import static com.amazonaws.event.ProgressEventType.TRANSFER_COMPLETED_EVENT;
+import static com.amazonaws.event.ProgressEventType.TRANSFER_PART_STARTED_EVENT;
+
+/**
+ * Listener to progress from AWS regarding transfers.
+ */
+public class ProgressableProgressListener implements ProgressListener {
+  private static final Logger LOG = S3AFileSystem.LOG;
+  private final S3AFileSystem fs;
+  private final String key;
+  private final Progressable progress;
+  private long lastBytesTransferred;
+  private final Upload upload;
+
+  /**
+   * Instantiate.
+   * @param fs filesystem: will be invoked with statistics updates
+   * @param key key for the upload
+   * @param upload source of events
+   * @param progress optional callback for progress.
+   */
+  public ProgressableProgressListener(S3AFileSystem fs,
+      String key,
+      Upload upload,
+      Progressable progress) {
+    this.fs = fs;
+    this.key = key;
+    this.upload = upload;
+    this.progress = progress;
+    this.lastBytesTransferred = 0;
+  }
+
+  @Override
+  public void progressChanged(ProgressEvent progressEvent) {
+    if (progress != null) {
+      progress.progress();
+    }
+
+    // There are 3 http ops here, but this should be close enough for now
+    ProgressEventType pet = progressEvent.getEventType();
+    if (pet == TRANSFER_PART_STARTED_EVENT ||
+        pet == TRANSFER_COMPLETED_EVENT) {
+      fs.incrementWriteOperations();
+    }
+
+    long transferred = upload.getProgress().getBytesTransferred();
+    long delta = transferred - lastBytesTransferred;
+    fs.incrementPutProgressStatistics(key, delta);
+    lastBytesTransferred = transferred;
+  }
+
+  /**
+   * Method to invoke after upload has completed.
+   * This can handle race conditions in setup/teardown.
+   * @return the number of bytes which were transferred after the notification
+   */
+  public long uploadCompleted() {
+    long delta = upload.getProgress().getBytesTransferred() -
+        lastBytesTransferred;
+    if (delta > 0) {
+      LOG.debug("S3A write delta changed after finished: {} bytes", delta);
+      fs.incrementPutProgressStatistics(key, delta);
+    }
+    return delta;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
index 6a59f3f..5509d36 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
@@ -35,10 +35,8 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.util.Progressable;
 import org.slf4j.Logger;
 
@@ -54,6 +52,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.Statistic.*;
 
 /**
  * Upload files/parts asap directly from a memory buffer (instead of buffering
@@ -77,8 +76,6 @@ public class S3AFastOutputStream extends OutputStream {
   private final int multiPartThreshold;
   private final S3AFileSystem fs;
   private final CannedAccessControlList cannedACL;
-  private final FileSystem.Statistics statistics;
-  private final String serverSideEncryptionAlgorithm;
   private final ProgressListener progressListener;
   private final ListeningExecutorService executorService;
   private MultiPartUpload multiPartUpload;
@@ -98,28 +95,28 @@ public class S3AFastOutputStream extends OutputStream {
    * @param bucket S3 bucket name
    * @param key S3 key name
    * @param progress report progress in order to prevent timeouts
-   * @param statistics track FileSystem.Statistics on the performed operations
    * @param cannedACL used CannedAccessControlList
-   * @param serverSideEncryptionAlgorithm algorithm for server side encryption
    * @param partSize size of a single part in a multi-part upload (except
    * last part)
    * @param multiPartThreshold files at least this size use multi-part upload
    * @param threadPoolExecutor thread factory
    * @throws IOException on any problem
    */
-  public S3AFastOutputStream(AmazonS3Client client, S3AFileSystem fs,
-      String bucket, String key, Progressable progress,
-      FileSystem.Statistics statistics, CannedAccessControlList cannedACL,
-      String serverSideEncryptionAlgorithm, long partSize,
-      long multiPartThreshold, ExecutorService threadPoolExecutor)
+  public S3AFastOutputStream(AmazonS3Client client,
+      S3AFileSystem fs,
+      String bucket,
+      String key,
+      Progressable progress,
+      CannedAccessControlList cannedACL,
+      long partSize,
+      long multiPartThreshold,
+      ExecutorService threadPoolExecutor)
       throws IOException {
     this.bucket = bucket;
     this.key = key;
     this.client = client;
     this.fs = fs;
     this.cannedACL = cannedACL;
-    this.statistics = statistics;
-    this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
     //Ensure limit as ByteArrayOutputStream size cannot exceed Integer.MAX_VALUE
     if (partSize > Integer.MAX_VALUE) {
       this.partSize = Integer.MAX_VALUE;
@@ -246,16 +243,17 @@ public class S3AFastOutputStream extends OutputStream {
       if (multiPartUpload == null) {
         putObject();
       } else {
-        if (buffer.size() > 0) {
+        int size = buffer.size();
+        if (size > 0) {
+          fs.incrementPutStartStatistics(size);
           //send last part
           multiPartUpload.uploadPartAsync(new ByteArrayInputStream(buffer
-              .toByteArray()), buffer.size());
+              .toByteArray()), size);
         }
         final List<PartETag> partETags = multiPartUpload
             .waitForAllPartUploads();
         multiPartUpload.complete(partETags);
       }
-      statistics.incrementWriteOps(1);
       // This will delete unnecessary fake parent directories
       fs.finishedWrite(key);
       LOG.debug("Upload complete for bucket '{}' key '{}'", bucket, key);
@@ -265,18 +263,19 @@ public class S3AFastOutputStream extends OutputStream {
     }
   }
 
+  /**
+   * Create the default metadata for a multipart upload operation.
+   * @return the metadata to use/extend.
+   */
   private ObjectMetadata createDefaultMetadata() {
-    ObjectMetadata om = new ObjectMetadata();
-    if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
-      om.setSSEAlgorithm(serverSideEncryptionAlgorithm);
-    }
-    return om;
+    return fs.newObjectMetadata();
   }
 
   private MultiPartUpload initiateMultiPartUpload() throws IOException {
-    final ObjectMetadata om = createDefaultMetadata();
     final InitiateMultipartUploadRequest initiateMPURequest =
-        new InitiateMultipartUploadRequest(bucket, key, om);
+        new InitiateMultipartUploadRequest(bucket,
+            key,
+            createDefaultMetadata());
     initiateMPURequest.setCannedACL(cannedACL);
     try {
       return new MultiPartUpload(
@@ -290,15 +289,18 @@ public class S3AFastOutputStream extends OutputStream {
     LOG.debug("Executing regular upload for bucket '{}' key '{}'",
         bucket, key);
     final ObjectMetadata om = createDefaultMetadata();
-    om.setContentLength(buffer.size());
-    final PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key,
-        new ByteArrayInputStream(buffer.toByteArray()), om);
-    putObjectRequest.setCannedAcl(cannedACL);
+    final int size = buffer.size();
+    om.setContentLength(size);
+    final PutObjectRequest putObjectRequest =
+        fs.newPutObjectRequest(key,
+            om,
+            new ByteArrayInputStream(buffer.toByteArray()));
     putObjectRequest.setGeneralProgressListener(progressListener);
     ListenableFuture<PutObjectResult> putObjectResult =
         executorService.submit(new Callable<PutObjectResult>() {
           @Override
           public PutObjectResult call() throws Exception {
+            fs.incrementPutStartStatistics(size);
             return client.putObject(putObjectRequest);
           }
         });
@@ -306,7 +308,7 @@ public class S3AFastOutputStream extends OutputStream {
     try {
       putObjectResult.get();
     } catch (InterruptedException ie) {
-      LOG.warn("Interrupted object upload:" + ie, ie);
+      LOG.warn("Interrupted object upload: {}", ie, ie);
       Thread.currentThread().interrupt();
     } catch (ExecutionException ee) {
       throw extractException("regular upload", key, ee);
@@ -339,7 +341,7 @@ public class S3AFastOutputStream extends OutputStream {
             public PartETag call() throws Exception {
               LOG.debug("Uploading part {} for id '{}'", currentPartNumber,
                   uploadId);
-              return client.uploadPart(request).getPartETag();
+              return fs.uploadPart(request).getPartETag();
             }
           });
       partETagsFutures.add(partETagFuture);
@@ -349,7 +351,7 @@ public class S3AFastOutputStream extends OutputStream {
       try {
         return Futures.allAsList(partETagsFutures).get();
       } catch (InterruptedException ie) {
-        LOG.warn("Interrupted partUpload:" + ie, ie);
+        LOG.warn("Interrupted partUpload: {}", ie, ie);
         Thread.currentThread().interrupt();
         return null;
       } catch (ExecutionException ee) {
@@ -382,11 +384,12 @@ public class S3AFastOutputStream extends OutputStream {
     public void abort() {
       LOG.warn("Aborting multi-part upload with id '{}'", uploadId);
       try {
+        fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED);
         client.abortMultipartUpload(new AbortMultipartUploadRequest(bucket,
             key, uploadId));
       } catch (Exception e2) {
         LOG.warn("Unable to abort multipart upload, you may need to purge  " +
-            "uploaded parts: " + e2, e2);
+            "uploaded parts: {}", e2, e2);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
index 9ecca33..75a6500 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileStatus.java
@@ -93,4 +93,11 @@ public class S3AFileStatus extends FileStatus {
       return super.getModificationTime();
     }
   }
+
+  @Override
+  public String toString() {
+    return super.toString() +
+        String.format(" isEmptyDirectory=%s", isEmptyDirectory());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 28d9843..c028544 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -42,7 +42,6 @@ import com.amazonaws.services.s3.AmazonS3Client;
 import com.amazonaws.services.s3.S3ClientOptions;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.CannedAccessControlList;
-import com.amazonaws.services.s3.model.DeleteObjectRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsRequest;
 import com.amazonaws.services.s3.model.ListObjectsRequest;
 import com.amazonaws.services.s3.model.ObjectListing;
@@ -50,6 +49,8 @@ import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.CopyObjectRequest;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import com.amazonaws.services.s3.model.UploadPartResult;
 import com.amazonaws.services.s3.transfer.Copy;
 import com.amazonaws.services.s3.transfer.TransferManager;
 import com.amazonaws.services.s3.transfer.TransferManagerConfiguration;
@@ -68,8 +69,13 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.util.Progressable;
@@ -77,6 +83,7 @@ import org.apache.hadoop.util.VersionInfo;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.Statistic.*;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -115,6 +122,7 @@ public class S3AFileSystem extends FileSystem {
   private CannedAccessControlList cannedACL;
   private String serverSideEncryptionAlgorithm;
   private S3AInstrumentation instrumentation;
+  private S3AStorageStatistics storageStatistics;
   private long readAhead;
 
   // The maximum number of entries that can be deleted in any call to s3
@@ -182,6 +190,15 @@ public class S3AFileSystem extends FileSystem {
       enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
 
       readAhead = longOption(conf, READAHEAD_RANGE, DEFAULT_READAHEAD_RANGE, 0);
+      storageStatistics = (S3AStorageStatistics)
+          GlobalStorageStatistics.INSTANCE
+              .put(S3AStorageStatistics.NAME,
+                  new GlobalStorageStatistics.StorageStatisticsProvider() {
+                    @Override
+                    public StorageStatistics provide() {
+                      return new S3AStorageStatistics();
+                    }
+                  });
 
       int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS);
       if (maxThreads < 2) {
@@ -284,6 +301,14 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Get S3A Instrumentation. For test purposes.
+   * @return this instance's instrumentation.
+   */
+  public S3AInstrumentation getInstrumentation() {
+    return instrumentation;
+  }
+
+  /**
    * Initializes the User-Agent header to send in HTTP requests to the S3
    * back-end.  We always include the Hadoop version number.  The user also may
    * set an optional custom prefix to put in front of the Hadoop version number.
@@ -559,23 +584,26 @@ public class S3AFileSystem extends FileSystem {
     }
     instrumentation.fileCreated();
     if (getConf().getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD)) {
-      return new FSDataOutputStream(new S3AFastOutputStream(s3, this, bucket,
-          key, progress, statistics, cannedACL,
-          serverSideEncryptionAlgorithm, partSize, multiPartThreshold,
-          threadPoolExecutor), statistics);
+      return new FSDataOutputStream(
+          new S3AFastOutputStream(s3,
+              this,
+              bucket,
+              key,
+              progress,
+              cannedACL,
+              partSize,
+              multiPartThreshold,
+              threadPoolExecutor),
+          statistics);
     }
     // We pass null to FSDataOutputStream so it won't count writes that
     // are being buffered to a file
     return new FSDataOutputStream(
         new S3AOutputStream(getConf(),
-            transfers,
             this,
-            bucket,
             key,
-            progress,
-            cannedACL,
-            statistics,
-            serverSideEncryptionAlgorithm),
+            progress
+        ),
         null);
   }
 
@@ -631,6 +659,7 @@ public class S3AFileSystem extends FileSystem {
   private boolean innerRename(Path src, Path dst) throws IOException,
       AmazonClientException {
     LOG.debug("Rename path {} to {}", src, dst);
+    incrementStatistic(INVOCATION_RENAME);
 
     String srcKey = pathToKey(src);
     String dstKey = pathToKey(dst);
@@ -731,8 +760,7 @@ public class S3AFileSystem extends FileSystem {
       request.setPrefix(srcKey);
       request.setMaxKeys(maxKeys);
 
-      ObjectListing objects = s3.listObjects(request);
-      statistics.incrementReadOps(1);
+      ObjectListing objects = listObjects(request);
 
       while (true) {
         for (S3ObjectSummary summary : objects.getObjectSummaries()) {
@@ -746,8 +774,7 @@ public class S3AFileSystem extends FileSystem {
         }
 
         if (objects.isTruncated()) {
-          objects = s3.listNextBatchOfObjects(objects);
-          statistics.incrementReadOps(1);
+          objects = continueListObjects(objects);
         } else {
           if (!keysToDelete.isEmpty()) {
             removeKeys(keysToDelete, false);
@@ -776,17 +803,223 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Increment a statistic by 1.
+   * @param statistic The operation to increment
+   */
+  protected void incrementStatistic(Statistic statistic) {
+    incrementStatistic(statistic, 1);
+  }
+
+  /**
+   * Increment a statistic by a specific value.
+   * @param statistic The operation to increment
+   * @param count the count to increment
+   */
+  protected void incrementStatistic(Statistic statistic, long count) {
+    instrumentation.incrementCounter(statistic, count);
+    storageStatistics.incrementCounter(statistic, count);
+  }
+
+  /**
    * Request object metadata; increments counters in the process.
    * @param key key
    * @return the metadata
    */
-  private ObjectMetadata getObjectMetadata(String key) {
+  protected ObjectMetadata getObjectMetadata(String key) {
+    incrementStatistic(OBJECT_METADATA_REQUESTS);
     ObjectMetadata meta = s3.getObjectMetadata(bucket, key);
-    statistics.incrementReadOps(1);
+    incrementReadOperations();
     return meta;
   }
 
   /**
+   * Initiate a {@code listObjects} operation, incrementing metrics
+   * in the process.
+   * @param request request to initiate
+   * @return the results
+   */
+  protected ObjectListing listObjects(ListObjectsRequest request) {
+    incrementStatistic(OBJECT_LIST_REQUESTS);
+    incrementReadOperations();
+    return s3.listObjects(request);
+  }
+
+  /**
+   * List the next set of objects.
+   * @param objects paged result
+   * @return the next result object
+   */
+  protected ObjectListing continueListObjects(ObjectListing objects) {
+    incrementStatistic(OBJECT_LIST_REQUESTS);
+    incrementReadOperations();
+    return s3.listNextBatchOfObjects(objects);
+  }
+
+  /**
+   * Increment read operations.
+   */
+  public void incrementReadOperations() {
+    statistics.incrementReadOps(1);
+  }
+
+  /**
+   * Increment the write operation counter.
+   * This is somewhat inaccurate, as it appears to be invoked more
+   * often than needed in progress callbacks.
+   */
+  public void incrementWriteOperations() {
+    statistics.incrementWriteOps(1);
+  }
+
+  /**
+   * Delete an object.
+   * Increments the {@code OBJECT_DELETE_REQUESTS} and write
+   * operation statistics.
+   * @param key key to blob to delete.
+   */
+  private void deleteObject(String key) {
+    incrementWriteOperations();
+    incrementStatistic(OBJECT_DELETE_REQUESTS);
+    s3.deleteObject(bucket, key);
+  }
+
+  /**
+   * Perform a bulk object delete operation.
+   * Increments the {@code OBJECT_DELETE_REQUESTS} and write
+   * operation statistics.
+   * @param deleteRequest keys to delete on the s3-backend
+   */
+  private void deleteObjects(DeleteObjectsRequest deleteRequest) {
+    incrementWriteOperations();
+    incrementStatistic(OBJECT_DELETE_REQUESTS, 1);
+    s3.deleteObjects(deleteRequest);
+  }
+
+  /**
+   * Create a putObject request.
+   * Adds the ACL and metadata
+   * @param key key of object
+   * @param metadata metadata header
+   * @param srcfile source file
+   * @return the request
+   */
+  public PutObjectRequest newPutObjectRequest(String key,
+      ObjectMetadata metadata, File srcfile) {
+    PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key,
+        srcfile);
+    putObjectRequest.setCannedAcl(cannedACL);
+    putObjectRequest.setMetadata(metadata);
+    return putObjectRequest;
+  }
+
+  /**
+   * Create a {@link PutObjectRequest} request.
+   * The metadata is assumed to have been configured with the size of the
+   * operation.
+   * @param key key of object
+   * @param metadata metadata header
+   * @param inputStream source data.
+   * @return the request
+   */
+  PutObjectRequest newPutObjectRequest(String key,
+      ObjectMetadata metadata, InputStream inputStream) {
+    PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key,
+        inputStream, metadata);
+    putObjectRequest.setCannedAcl(cannedACL);
+    return putObjectRequest;
+  }
+
+  /**
+   * Create a new object metadata instance.
+   * Any standard metadata headers are added here, for example:
+   * encryption.
+   * @return a new metadata instance
+   */
+  public ObjectMetadata newObjectMetadata() {
+    final ObjectMetadata om = new ObjectMetadata();
+    if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
+      om.setSSEAlgorithm(serverSideEncryptionAlgorithm);
+    }
+    return om;
+  }
+
+  /**
+   * Create a new object metadata instance.
+   * Any standard metadata headers are added here, for example:
+   * encryption.
+   *
+   * @param length length of data to set in header.
+   * @return a new metadata instance
+   */
+  public ObjectMetadata newObjectMetadata(long length) {
+    final ObjectMetadata om = newObjectMetadata();
+    om.setContentLength(length);
+    return om;
+  }
+
+  /**
+   * PUT an object, incrementing the put requests and put bytes
+   * counters.
+   * It does not update the other counters,
+   * as existing code does that as progress callbacks come in.
+   * Byte length is calculated from the file length, or, if there is no
+   * file, from the content length of the header.
+   * @param putObjectRequest the request
+   * @return the upload initiated
+   */
+  public Upload putObject(PutObjectRequest putObjectRequest) {
+    long len;
+    if (putObjectRequest.getFile() != null) {
+      len = putObjectRequest.getFile().length();
+    } else {
+      len = putObjectRequest.getMetadata().getContentLength();
+    }
+    incrementPutStartStatistics(len);
+    return transfers.upload(putObjectRequest);
+  }
+
+  /**
+   * Upload part of a multi-partition file.
+   * Increments the write and put counters
+   * @param request request
+   * @return the result of the operation.
+   */
+  public UploadPartResult uploadPart(UploadPartRequest request) {
+    incrementPutStartStatistics(request.getPartSize());
+    return s3.uploadPart(request);
+  }
+
+  /**
+   * At the start of a put/multipart upload operation, update the
+   * relevant counters.
+   *
+   * @param bytes bytes in the request.
+   */
+  public void incrementPutStartStatistics(long bytes) {
+    LOG.debug("PUT start {} bytes", bytes);
+    incrementWriteOperations();
+    incrementStatistic(OBJECT_PUT_REQUESTS);
+    if (bytes > 0) {
+      incrementStatistic(OBJECT_PUT_BYTES, bytes);
+    }
+  }
+
+  /**
+   * Callback for use in progress callbacks from put/multipart upload events.
+   * Increments those statistics which are expected to be updated during
+   * the ongoing upload operation.
+   * @param key key to file that is being written (for logging)
+   * @param bytes bytes successfully uploaded.
+   */
+  public void incrementPutProgressStatistics(String key, long bytes) {
+    LOG.debug("PUT {}: {} bytes", key, bytes);
+    incrementWriteOperations();
+    if (bytes > 0) {
+      statistics.incrementBytesWritten(bytes);
+    }
+  }
+
+  /**
    * A helper method to delete a list of keys on a s3-backend.
    *
    * @param keysToDelete collection of keys to delete on the s3-backend
@@ -796,21 +1029,13 @@ public class S3AFileSystem extends FileSystem {
   private void removeKeys(List<DeleteObjectsRequest.KeyVersion> keysToDelete,
           boolean clearKeys) throws AmazonClientException {
     if (enableMultiObjectsDelete) {
-      DeleteObjectsRequest deleteRequest
-          = new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
-      s3.deleteObjects(deleteRequest);
+      deleteObjects(new DeleteObjectsRequest(bucket).withKeys(keysToDelete));
       instrumentation.fileDeleted(keysToDelete.size());
-      statistics.incrementWriteOps(1);
     } else {
-      int writeops = 0;
-
       for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
-        s3.deleteObject(
-            new DeleteObjectRequest(bucket, keyVersion.getKey()));
-        writeops++;
+        deleteObject(keyVersion.getKey());
       }
       instrumentation.fileDeleted(keysToDelete.size());
-      statistics.incrementWriteOps(writeops);
     }
     if (clearKeys) {
       keysToDelete.clear();
@@ -880,9 +1105,8 @@ public class S3AFileSystem extends FileSystem {
 
       if (status.isEmptyDirectory()) {
         LOG.debug("Deleting fake empty directory {}", key);
-        s3.deleteObject(bucket, key);
+        deleteObject(key);
         instrumentation.directoryDeleted();
-        statistics.incrementWriteOps(1);
       } else {
         LOG.debug("Getting objects for directory prefix {} to delete", key);
 
@@ -893,9 +1117,9 @@ public class S3AFileSystem extends FileSystem {
         //request.setDelimiter("/");
         request.setMaxKeys(maxKeys);
 
-        List<DeleteObjectsRequest.KeyVersion> keys = new ArrayList<>();
-        ObjectListing objects = s3.listObjects(request);
-        statistics.incrementReadOps(1);
+        ObjectListing objects = listObjects(request);
+        List<DeleteObjectsRequest.KeyVersion> keys =
+            new ArrayList<>(objects.getObjectSummaries().size());
         while (true) {
           for (S3ObjectSummary summary : objects.getObjectSummaries()) {
             keys.add(new DeleteObjectsRequest.KeyVersion(summary.getKey()));
@@ -907,8 +1131,7 @@ public class S3AFileSystem extends FileSystem {
           }
 
           if (objects.isTruncated()) {
-            objects = s3.listNextBatchOfObjects(objects);
-            statistics.incrementReadOps(1);
+            objects = continueListObjects(objects);
           } else {
             if (!keys.isEmpty()) {
               removeKeys(keys, false);
@@ -919,13 +1142,11 @@ public class S3AFileSystem extends FileSystem {
       }
     } else {
       LOG.debug("delete: Path is a file");
-      s3.deleteObject(bucket, key);
       instrumentation.fileDeleted(1);
-      statistics.incrementWriteOps(1);
+      deleteObject(key);
     }
 
     createFakeDirectoryIfNecessary(f.getParent());
-
     return true;
   }
 
@@ -934,7 +1155,7 @@ public class S3AFileSystem extends FileSystem {
     String key = pathToKey(f);
     if (!key.isEmpty() && !exists(f)) {
       LOG.debug("Creating new fake directory at {}", f);
-      createFakeDirectory(bucket, key);
+      createFakeDirectory(key);
     }
   }
 
@@ -970,6 +1191,7 @@ public class S3AFileSystem extends FileSystem {
       IOException, AmazonClientException {
     String key = pathToKey(f);
     LOG.debug("List status for path: {}", f);
+    incrementStatistic(INVOCATION_LIST_STATUS);
 
     final List<FileStatus> result = new ArrayList<FileStatus>();
     final FileStatus fileStatus =  getFileStatus(f);
@@ -987,8 +1209,7 @@ public class S3AFileSystem extends FileSystem {
 
       LOG.debug("listStatus: doing listObjects for directory {}", key);
 
-      ObjectListing objects = s3.listObjects(request);
-      statistics.incrementReadOps(1);
+      ObjectListing objects = listObjects(request);
 
       Path fQualified = f.makeQualified(uri, workingDir);
 
@@ -999,33 +1220,25 @@ public class S3AFileSystem extends FileSystem {
           if (keyPath.equals(fQualified) ||
               summary.getKey().endsWith(S3N_FOLDER_SUFFIX)) {
             LOG.debug("Ignoring: {}", keyPath);
-            continue;
-          }
-
-          if (objectRepresentsDirectory(summary.getKey(), summary.getSize())) {
-            result.add(new S3AFileStatus(true, true, keyPath));
-            LOG.debug("Adding: fd: {}", keyPath);
           } else {
-            result.add(new S3AFileStatus(summary.getSize(),
-                dateToLong(summary.getLastModified()), keyPath,
-                getDefaultBlockSize(fQualified)));
-            LOG.debug("Adding: fi: {}", keyPath);
+            S3AFileStatus status = createFileStatus(keyPath, summary,
+                getDefaultBlockSize(keyPath));
+            result.add(status);
+            LOG.debug("Adding: {}", status);
           }
         }
 
         for (String prefix : objects.getCommonPrefixes()) {
           Path keyPath = keyToPath(prefix).makeQualified(uri, workingDir);
-          if (keyPath.equals(f)) {
-            continue;
+          if (!keyPath.equals(f)) {
+            result.add(new S3AFileStatus(true, false, keyPath));
+            LOG.debug("Adding: rd: {}", keyPath);
           }
-          result.add(new S3AFileStatus(true, false, keyPath));
-          LOG.debug("Adding: rd: {}", keyPath);
         }
 
         if (objects.isTruncated()) {
           LOG.debug("listStatus: list truncated - getting next batch");
-          objects = s3.listNextBatchOfObjects(objects);
-          statistics.incrementReadOps(1);
+          objects = continueListObjects(objects);
         } else {
           break;
         }
@@ -1038,8 +1251,6 @@ public class S3AFileSystem extends FileSystem {
     return result.toArray(new FileStatus[result.size()]);
   }
 
-
-
   /**
    * Set the current working directory for the given file system. All relative
    * paths will be resolved relative to it.
@@ -1061,7 +1272,7 @@ public class S3AFileSystem extends FileSystem {
   /**
    *
    * Make the given path and all non-existent parents into
-   * directories. Has the semantics of Unix @{code 'mkdir -p'}.
+   * directories. Has the semantics of Unix {@code 'mkdir -p'}.
    * Existence of the directory hierarchy is not an error.
    * @param path path to create
    * @param permission to apply to f
@@ -1096,7 +1307,7 @@ public class S3AFileSystem extends FileSystem {
   private boolean innerMkdirs(Path f, FsPermission permission)
       throws IOException, FileAlreadyExistsException, AmazonClientException {
     LOG.debug("Making directory: {}", f);
-
+    incrementStatistic(INVOCATION_MKDIRS);
     try {
       FileStatus fileStatus = getFileStatus(f);
 
@@ -1125,7 +1336,7 @@ public class S3AFileSystem extends FileSystem {
       } while (fPart != null);
 
       String key = pathToKey(f);
-      createFakeDirectory(bucket, key);
+      createFakeDirectory(key);
       return true;
     }
   }
@@ -1139,12 +1350,12 @@ public class S3AFileSystem extends FileSystem {
    */
   public S3AFileStatus getFileStatus(Path f) throws IOException {
     String key = pathToKey(f);
+    incrementStatistic(INVOCATION_GET_FILE_STATUS);
     LOG.debug("Getting path status for {}  ({})", f , key);
 
     if (!key.isEmpty()) {
       try {
-        ObjectMetadata meta = s3.getObjectMetadata(bucket, key);
-        statistics.incrementReadOps(1);
+        ObjectMetadata meta = getObjectMetadata(key);
 
         if (objectRepresentsDirectory(key, meta.getContentLength())) {
           LOG.debug("Found exact file: fake directory");
@@ -1169,8 +1380,7 @@ public class S3AFileSystem extends FileSystem {
       if (!key.endsWith("/")) {
         String newKey = key + "/";
         try {
-          ObjectMetadata meta = s3.getObjectMetadata(bucket, newKey);
-          statistics.incrementReadOps(1);
+          ObjectMetadata meta = getObjectMetadata(newKey);
 
           if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
             LOG.debug("Found file (with /): fake directory");
@@ -1203,8 +1413,7 @@ public class S3AFileSystem extends FileSystem {
       request.setDelimiter("/");
       request.setMaxKeys(1);
 
-      ObjectListing objects = s3.listObjects(request);
-      statistics.incrementReadOps(1);
+      ObjectListing objects = listObjects(request);
 
       if (!objects.getCommonPrefixes().isEmpty()
           || !objects.getObjectSummaries().isEmpty()) {
@@ -1287,7 +1496,8 @@ public class S3AFileSystem extends FileSystem {
   private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite,
       Path src, Path dst)
       throws IOException, FileAlreadyExistsException, AmazonClientException {
-    String key = pathToKey(dst);
+    incrementStatistic(INVOCATION_COPY_FROM_LOCAL_FILE);
+    final String key = pathToKey(dst);
 
     if (!overwrite && exists(dst)) {
       throw new FileAlreadyExistsException(dst + " already exists");
@@ -1298,35 +1508,19 @@ public class S3AFileSystem extends FileSystem {
     LocalFileSystem local = getLocal(getConf());
     File srcfile = local.pathToFile(src);
 
-    final ObjectMetadata om = new ObjectMetadata();
-    if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
-      om.setSSEAlgorithm(serverSideEncryptionAlgorithm);
-    }
-    PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key, srcfile);
-    putObjectRequest.setCannedAcl(cannedACL);
-    putObjectRequest.setMetadata(om);
-
-    ProgressListener progressListener = new ProgressListener() {
-      public void progressChanged(ProgressEvent progressEvent) {
-        switch (progressEvent.getEventType()) {
-          case TRANSFER_PART_COMPLETED_EVENT:
-            statistics.incrementWriteOps(1);
-            break;
-          default:
-            break;
-        }
-      }
-    };
-
-    statistics.incrementWriteOps(1);
-    Upload up = transfers.upload(putObjectRequest);
-    up.addProgressListener(progressListener);
+    final ObjectMetadata om = newObjectMetadata();
+    PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, srcfile);
+    Upload up = putObject(putObjectRequest);
+    ProgressableProgressListener listener = new ProgressableProgressListener(
+        this, key, up, null);
+    up.addProgressListener(listener);
     try {
       up.waitForUploadResult();
     } catch (InterruptedException e) {
       throw new InterruptedIOException("Interrupted copying " + src
           + " to "  + dst + ", cancelling");
     }
+    listener.uploadCompleted();
 
     // This will delete unnecessary fake parent directories
     finishedWrite(key);
@@ -1375,7 +1569,7 @@ public class S3AFileSystem extends FileSystem {
     LOG.debug("copyFile {} -> {} ", srcKey, dstKey);
 
     try {
-      ObjectMetadata srcom = s3.getObjectMetadata(bucket, srcKey);
+      ObjectMetadata srcom = getObjectMetadata(srcKey);
       ObjectMetadata dstom = cloneObjectMetadata(srcom);
       if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
         dstom.setSSEAlgorithm(serverSideEncryptionAlgorithm);
@@ -1389,7 +1583,7 @@ public class S3AFileSystem extends FileSystem {
         public void progressChanged(ProgressEvent progressEvent) {
           switch (progressEvent.getEventType()) {
             case TRANSFER_PART_COMPLETED_EVENT:
-              statistics.incrementWriteOps(1);
+              incrementWriteOperations();
               break;
             default:
               break;
@@ -1401,7 +1595,7 @@ public class S3AFileSystem extends FileSystem {
       copy.addProgressListener(progressListener);
       try {
         copy.waitForCopyResult();
-        statistics.incrementWriteOps(1);
+        incrementWriteOperations();
         instrumentation.filesCopied(1, size);
       } catch (InterruptedException e) {
         throw new InterruptedIOException("Interrupted copying " + srcKey
@@ -1413,26 +1607,12 @@ public class S3AFileSystem extends FileSystem {
     }
   }
 
-  private boolean objectRepresentsDirectory(final String name, final long size) {
-    return !name.isEmpty()
-        && name.charAt(name.length() - 1) == '/'
-        && size == 0L;
-  }
-
-  // Handles null Dates that can be returned by AWS
-  private static long dateToLong(final Date date) {
-    if (date == null) {
-      return 0L;
-    }
-
-    return date.getTime();
-  }
-
   /**
    * Perform post-write actions.
    * @param key key written to
    */
   public void finishedWrite(String key) {
+    LOG.debug("Finished write to {}", key);
     deleteUnnecessaryFakeDirectories(keyToPath(key).getParent());
   }
 
@@ -1454,8 +1634,7 @@ public class S3AFileSystem extends FileSystem {
 
         if (status.isDirectory() && status.isEmptyDirectory()) {
           LOG.debug("Deleting fake directory {}/", key);
-          s3.deleteObject(bucket, key + "/");
-          statistics.incrementWriteOps(1);
+          deleteObject(key + "/");
         }
       } catch (IOException | AmazonClientException e) {
         LOG.debug("While deleting key {} ", key, e);
@@ -1471,18 +1650,20 @@ public class S3AFileSystem extends FileSystem {
   }
 
 
-  private void createFakeDirectory(final String bucketName, final String objectName)
-      throws AmazonClientException, AmazonServiceException {
+  private void createFakeDirectory(final String objectName)
+      throws AmazonClientException, AmazonServiceException,
+      InterruptedIOException {
     if (!objectName.endsWith("/")) {
-      createEmptyObject(bucketName, objectName + "/");
+      createEmptyObject(objectName + "/");
     } else {
-      createEmptyObject(bucketName, objectName);
+      createEmptyObject(objectName);
     }
   }
 
   // Used to create an empty file that represents an empty directory
-  private void createEmptyObject(final String bucketName, final String objectName)
-      throws AmazonClientException, AmazonServiceException {
+  private void createEmptyObject(final String objectName)
+      throws AmazonClientException, AmazonServiceException,
+      InterruptedIOException {
     final InputStream im = new InputStream() {
       @Override
       public int read() throws IOException {
@@ -1490,16 +1671,16 @@ public class S3AFileSystem extends FileSystem {
       }
     };
 
-    final ObjectMetadata om = new ObjectMetadata();
-    om.setContentLength(0L);
-    if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
-      om.setSSEAlgorithm(serverSideEncryptionAlgorithm);
+    PutObjectRequest putObjectRequest = newPutObjectRequest(objectName,
+        newObjectMetadata(0L),
+        im);
+    Upload upload = putObject(putObjectRequest);
+    try {
+      upload.waitForUploadResult();
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException("Interrupted creating " + objectName);
     }
-    PutObjectRequest putObjectRequest =
-        new PutObjectRequest(bucketName, objectName, im, om);
-    putObjectRequest.setCannedAcl(cannedACL);
-    s3.putObject(putObjectRequest);
-    statistics.incrementWriteOps(1);
+    incrementPutProgressStatistics(objectName, 0);
     instrumentation.directoryCreated();
   }
 
@@ -1514,10 +1695,7 @@ public class S3AFileSystem extends FileSystem {
     // This approach may be too brittle, especially if
     // in future there are new attributes added to ObjectMetadata
     // that we do not explicitly call to set here
-    ObjectMetadata ret = new ObjectMetadata();
-
-    // Non null attributes
-    ret.setContentLength(source.getContentLength());
+    ObjectMetadata ret = newObjectMetadata(source.getContentLength());
 
     // Possibly null attributes
     // Allowing nulls to pass breaks it during later use
@@ -1627,6 +1805,75 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Override superclass so as to add statistic collection.
+   * {@inheritDoc}
+   */
+  @Override
+  public FileStatus[] globStatus(Path pathPattern) throws IOException {
+    incrementStatistic(INVOCATION_GLOB_STATUS);
+    return super.globStatus(pathPattern);
+  }
+
+  /**
+   * Override superclass so as to add statistic collection.
+   * {@inheritDoc}
+   */
+  @Override
+  public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
+      throws IOException {
+    incrementStatistic(INVOCATION_GLOB_STATUS);
+    return super.globStatus(pathPattern, filter);
+  }
+
+  /**
+   * Override superclass so as to add statistic collection.
+   * {@inheritDoc}
+   */
+  @Override
+  public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
+      throws FileNotFoundException, IOException {
+    incrementStatistic(INVOCATION_LIST_LOCATED_STATUS);
+    return super.listLocatedStatus(f);
+  }
+
+  @Override
+  public RemoteIterator<LocatedFileStatus> listFiles(Path f,
+      boolean recursive) throws FileNotFoundException, IOException {
+    incrementStatistic(INVOCATION_LIST_FILES);
+    return super.listFiles(f, recursive);
+  }
+
+  /**
+   * Override superclass so as to add statistic collection.
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean exists(Path f) throws IOException {
+    incrementStatistic(INVOCATION_EXISTS);
+    return super.exists(f);
+  }
+
+  /**
+   * Override superclass so as to add statistic collection.
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean isDirectory(Path f) throws IOException {
+    incrementStatistic(INVOCATION_IS_DIRECTORY);
+    return super.isDirectory(f);
+  }
+
+  /**
+   * Override superclass so as to add statistic collection.
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean isFile(Path f) throws IOException {
+    incrementStatistic(INVOCATION_IS_FILE);
+    return super.isFile(f);
+  }
+
+  /**
    * Get a integer option >= the minimum allowed value.
    * @param conf configuration
    * @param key key to look up

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index 285f228..8892f0e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.metrics2.MetricStringBuilder;
@@ -26,49 +27,30 @@ import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.metrics2.lib.MutableMetric;
 
 import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
 
+import static org.apache.hadoop.fs.s3a.Statistic.*;
+
 /**
  * Instrumentation of S3a.
- * Derived from the {@code AzureFileSystemInstrumentation}
+ * Derived from the {@code AzureFileSystemInstrumentation}.
+ *
+ * Counters and metrics are generally addressed in code by their name or
+ * {@link Statistic} key. There <i>may</i> be some Statistics which do
+ * not have an entry here. To avoid attempts to access such counters failing,
+ * the operations to increment/query metric values are designed to handle
+ * lookup failures.
  */
 @Metrics(about = "Metrics for S3a", context = "S3AFileSystem")
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3AInstrumentation {
   public static final String CONTEXT = "S3AFileSystem";
-
-  public static final String STREAM_OPENED = "streamOpened";
-  public static final String STREAM_CLOSE_OPERATIONS = "streamCloseOperations";
-  public static final String STREAM_CLOSED = "streamClosed";
-  public static final String STREAM_ABORTED = "streamAborted";
-  public static final String STREAM_READ_EXCEPTIONS = "streamReadExceptions";
-  public static final String STREAM_SEEK_OPERATIONS = "streamSeekOperations";
-  public static final String STREAM_FORWARD_SEEK_OPERATIONS
-      = "streamForwardSeekOperations";
-  public static final String STREAM_BACKWARD_SEEK_OPERATIONS
-      = "streamBackwardSeekOperations";
-  public static final String STREAM_SEEK_BYTES_SKIPPED =
-      "streamBytesSkippedOnSeek";
-  public static final String STREAM_SEEK_BYTES_BACKWARDS =
-      "streamBytesBackwardsOnSeek";
-  public static final String STREAM_SEEK_BYTES_READ = "streamBytesRead";
-  public static final String STREAM_READ_OPERATIONS = "streamReadOperations";
-  public static final String STREAM_READ_FULLY_OPERATIONS
-      = "streamReadFullyOperations";
-  public static final String STREAM_READ_OPERATIONS_INCOMPLETE
-      = "streamReadOperationsIncomplete";
-  public static final String FILES_CREATED = "files_created";
-  public static final String FILES_COPIED = "files_copied";
-  public static final String FILES_COPIED_BYTES = "files_copied_bytes";
-  public static final String FILES_DELETED = "files_deleted";
-  public static final String DIRECTORIES_CREATED = "directories_created";
-  public static final String DIRECTORIES_DELETED = "directories_deleted";
-  public static final String IGNORED_ERRORS = "ignored_errors";
   private final MetricsRegistry registry =
       new MetricsRegistry("S3AFileSystem").setContext(CONTEXT);
   private final MutableCounterLong streamOpenOperations;
@@ -95,6 +77,27 @@ public class S3AInstrumentation {
   private final MutableCounterLong numberOfDirectoriesDeleted;
   private final Map<String, MutableCounterLong> streamMetrics = new HashMap<>();
 
+  private static final Statistic[] COUNTERS_TO_CREATE = {
+      INVOCATION_COPY_FROM_LOCAL_FILE,
+      INVOCATION_EXISTS,
+      INVOCATION_GET_FILE_STATUS,
+      INVOCATION_GLOB_STATUS,
+      INVOCATION_IS_DIRECTORY,
+      INVOCATION_IS_FILE,
+      INVOCATION_LIST_FILES,
+      INVOCATION_LIST_LOCATED_STATUS,
+      INVOCATION_LIST_STATUS,
+      INVOCATION_MKDIRS,
+      INVOCATION_RENAME,
+      OBJECT_COPY_REQUESTS,
+      OBJECT_DELETE_REQUESTS,
+      OBJECT_LIST_REQUESTS,
+      OBJECT_METADATA_REQUESTS,
+      OBJECT_MULTIPART_UPLOAD_ABORTED,
+      OBJECT_PUT_BYTES,
+      OBJECT_PUT_REQUESTS
+  };
+
   public S3AInstrumentation(URI name) {
     UUID fileSystemInstanceId = UUID.randomUUID();
     registry.tag("FileSystemId",
@@ -103,50 +106,35 @@ public class S3AInstrumentation {
     registry.tag("fsURI",
         "URI of this filesystem",
         name.toString());
-    streamOpenOperations = streamCounter(STREAM_OPENED,
-        "Total count of times an input stream to object store was opened");
-    streamCloseOperations = streamCounter(STREAM_CLOSE_OPERATIONS,
-        "Total count of times an attempt to close a data stream was made");
-    streamClosed = streamCounter(STREAM_CLOSED,
-        "Count of times the TCP stream was closed");
-    streamAborted = streamCounter(STREAM_ABORTED,
-        "Count of times the TCP stream was aborted");
-    streamSeekOperations = streamCounter(STREAM_SEEK_OPERATIONS,
-        "Number of seek operations invoked on input streams");
-    streamReadExceptions = streamCounter(STREAM_READ_EXCEPTIONS,
-        "Number of read exceptions caught and attempted to recovered from");
-    streamForwardSeekOperations = streamCounter(STREAM_FORWARD_SEEK_OPERATIONS,
-        "Number of executed seek operations which went forward in a stream");
-    streamBackwardSeekOperations = streamCounter(
-        STREAM_BACKWARD_SEEK_OPERATIONS,
-        "Number of executed seek operations which went backwards in a stream");
-    streamBytesSkippedOnSeek = streamCounter(STREAM_SEEK_BYTES_SKIPPED,
-        "Count of bytes skipped during forward seek operations");
-    streamBytesBackwardsOnSeek = streamCounter(STREAM_SEEK_BYTES_BACKWARDS,
-        "Count of bytes moved backwards during seek operations");
-    streamBytesRead = streamCounter(STREAM_SEEK_BYTES_READ,
-        "Count of bytes read during seek() in stream operations");
-    streamReadOperations = streamCounter(STREAM_READ_OPERATIONS,
-        "Count of read() operations in streams");
-    streamReadFullyOperations = streamCounter(STREAM_READ_FULLY_OPERATIONS,
-        "Count of readFully() operations in streams");
-    streamReadsIncomplete = streamCounter(STREAM_READ_OPERATIONS_INCOMPLETE,
-        "Count of incomplete read() operations in streams");
-
-    numberOfFilesCreated = counter(FILES_CREATED,
-            "Total number of files created through the object store.");
-    numberOfFilesCopied = counter(FILES_COPIED,
-            "Total number of files copied within the object store.");
-    bytesOfFilesCopied = counter(FILES_COPIED_BYTES,
-            "Total number of bytes copied within the object store.");
-    numberOfFilesDeleted = counter(FILES_DELETED,
-            "Total number of files deleted through from the object store.");
-    numberOfDirectoriesCreated = counter(DIRECTORIES_CREATED,
-        "Total number of directories created through the object store.");
-    numberOfDirectoriesDeleted = counter(DIRECTORIES_DELETED,
-        "Total number of directories deleted through the object store.");
-    ignoredErrors = counter(IGNORED_ERRORS,
-        "Total number of errors caught and ingored.");
+    streamOpenOperations = streamCounter(STREAM_OPENED);
+    streamCloseOperations = streamCounter(STREAM_CLOSE_OPERATIONS);
+    streamClosed = streamCounter(STREAM_CLOSED);
+    streamAborted = streamCounter(STREAM_ABORTED);
+    streamSeekOperations = streamCounter(STREAM_SEEK_OPERATIONS);
+    streamReadExceptions = streamCounter(STREAM_READ_EXCEPTIONS);
+    streamForwardSeekOperations =
+        streamCounter(STREAM_FORWARD_SEEK_OPERATIONS);
+    streamBackwardSeekOperations =
+        streamCounter(STREAM_BACKWARD_SEEK_OPERATIONS);
+    streamBytesSkippedOnSeek = streamCounter(STREAM_SEEK_BYTES_SKIPPED);
+    streamBytesBackwardsOnSeek =
+        streamCounter(STREAM_SEEK_BYTES_BACKWARDS);
+    streamBytesRead = streamCounter(STREAM_SEEK_BYTES_READ);
+    streamReadOperations = streamCounter(STREAM_READ_OPERATIONS);
+    streamReadFullyOperations =
+        streamCounter(STREAM_READ_FULLY_OPERATIONS);
+    streamReadsIncomplete =
+        streamCounter(STREAM_READ_OPERATIONS_INCOMPLETE);
+    numberOfFilesCreated = counter(FILES_CREATED);
+    numberOfFilesCopied = counter(FILES_COPIED);
+    bytesOfFilesCopied = counter(FILES_COPIED_BYTES);
+    numberOfFilesDeleted = counter(FILES_DELETED);
+    numberOfDirectoriesCreated = counter(DIRECTORIES_CREATED);
+    numberOfDirectoriesDeleted = counter(DIRECTORIES_DELETED);
+    ignoredErrors = counter(IGNORED_ERRORS);
+    for (Statistic statistic : COUNTERS_TO_CREATE) {
+      counter(statistic);
+    }
   }
 
   /**
@@ -174,6 +162,25 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Create a counter in the registry.
+   * @param op statistic to count
+   * @return a new counter
+   */
+  protected final MutableCounterLong counter(Statistic op) {
+    return counter(op.getSymbol(), op.getDescription());
+  }
+
+  /**
+   * Create a counter in the stream map: these are unregistered in the public
+   * metrics.
+   * @param op statistic to count
+   * @return a new counter
+   */
+  protected final MutableCounterLong streamCounter(Statistic op) {
+    return streamCounter(op.getSymbol(), op.getDescription());
+  }
+
+  /**
    * Create a gauge in the registry.
    * @param name name gauge name
    * @param desc description
@@ -216,6 +223,58 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Get the value of a counter.
+   * @param statistic the operation
+   * @return its value, or 0 if not found.
+   */
+  public long getCounterValue(Statistic statistic) {
+    return getCounterValue(statistic.getSymbol());
+  }
+
+  /**
+   * Get the value of a counter.
+   * If the counter is null, return 0.
+   * @param name the name of the counter
+   * @return its value.
+   */
+  public long getCounterValue(String name) {
+    MutableCounterLong counter = lookupCounter(name);
+    return counter == null ? 0 : counter.value();
+  }
+
+  /**
+   * Lookup a counter by name. Return null if it is not known.
+   * @param name counter name
+   * @return the counter
+   */
+  private MutableCounterLong lookupCounter(String name) {
+    MutableMetric metric = lookupMetric(name);
+    if (metric == null) {
+      return null;
+    }
+    Preconditions.checkNotNull(metric, "not found: " + name);
+    if (!(metric instanceof MutableCounterLong)) {
+      throw new IllegalStateException("Metric " + name
+          + " is not a MutableCounterLong: " + metric);
+    }
+    return (MutableCounterLong) metric;
+  }
+
+  /**
+   * Look up a metric from both the registered set and the lighter weight
+   * stream entries.
+   * @param name metric name
+   * @return the metric or null
+   */
+  public MutableMetric lookupMetric(String name) {
+    MutableMetric metric = getRegistry().get(name);
+    if (metric == null) {
+      metric = streamMetrics.get(name);
+    }
+    return metric;
+  }
+
+  /**
    * Indicate that S3A created a file.
    */
   public void fileCreated() {
@@ -263,6 +322,19 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Increment a specific counter.
+   * No-op if not defined.
+   * @param op operation
+   * @param count increment value
+   */
+  public void incrementCounter(Statistic op, long count) {
+    MutableCounterLong counter = lookupCounter(op.getSymbol());
+    if (counter != null) {
+      counter.incr(count);
+    }
+  }
+
+  /**
    * Create a stream input statistics instance.
    * @return the new instance
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
index 593e9e8..23ba682 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
@@ -19,19 +19,11 @@
 package org.apache.hadoop.fs.s3a;
 
 import com.amazonaws.AmazonClientException;
-import com.amazonaws.event.ProgressEvent;
-import com.amazonaws.event.ProgressEventType;
-import com.amazonaws.event.ProgressListener;
-import com.amazonaws.services.s3.model.CannedAccessControlList;
 import com.amazonaws.services.s3.model.ObjectMetadata;
-import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.transfer.TransferManager;
 import com.amazonaws.services.s3.transfer.Upload;
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.util.Progressable;
 
@@ -44,8 +36,6 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.io.OutputStream;
 
-import static com.amazonaws.event.ProgressEventType.TRANSFER_COMPLETED_EVENT;
-import static com.amazonaws.event.ProgressEventType.TRANSFER_PART_STARTED_EVENT;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 
@@ -59,32 +49,20 @@ public class S3AOutputStream extends OutputStream {
   private File backupFile;
   private boolean closed;
   private String key;
-  private String bucket;
-  private TransferManager transfers;
   private Progressable progress;
   private long partSize;
   private long partSizeThreshold;
   private S3AFileSystem fs;
-  private CannedAccessControlList cannedACL;
-  private FileSystem.Statistics statistics;
   private LocalDirAllocator lDirAlloc;
-  private String serverSideEncryptionAlgorithm;
 
   public static final Logger LOG = S3AFileSystem.LOG;
 
-  public S3AOutputStream(Configuration conf, TransferManager transfers,
-      S3AFileSystem fs, String bucket, String key, Progressable progress,
-      CannedAccessControlList cannedACL, FileSystem.Statistics statistics,
-      String serverSideEncryptionAlgorithm)
+  public S3AOutputStream(Configuration conf,
+      S3AFileSystem fs, String key, Progressable progress)
       throws IOException {
-    this.bucket = bucket;
     this.key = key;
-    this.transfers = transfers;
     this.progress = progress;
     this.fs = fs;
-    this.cannedACL = cannedACL;
-    this.statistics = statistics;
-    this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
 
     partSize = fs.getPartitionSize();
     partSizeThreshold = fs.getMultiPartThreshold();
@@ -124,30 +102,18 @@ public class S3AOutputStream extends OutputStream {
 
 
     try {
-      final ObjectMetadata om = new ObjectMetadata();
-      if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
-        om.setSSEAlgorithm(serverSideEncryptionAlgorithm);
-      }
-      PutObjectRequest putObjectRequest =
-          new PutObjectRequest(bucket, key, backupFile);
-      putObjectRequest.setCannedAcl(cannedACL);
-      putObjectRequest.setMetadata(om);
-
-      Upload upload = transfers.upload(putObjectRequest);
-
-      ProgressableProgressListener listener = 
-          new ProgressableProgressListener(upload, progress, statistics);
+      final ObjectMetadata om = fs.newObjectMetadata();
+      Upload upload = fs.putObject(
+          fs.newPutObjectRequest(
+              key,
+              om,
+              backupFile));
+      ProgressableProgressListener listener =
+          new ProgressableProgressListener(fs, key, upload, progress);
       upload.addProgressListener(listener);
 
       upload.waitForUploadResult();
-
-      long delta = upload.getProgress().getBytesTransferred() -
-          listener.getLastBytesTransferred();
-      if (statistics != null && delta != 0) {
-        LOG.debug("S3A write delta changed after finished: {} bytes", delta);
-        statistics.incrementBytesWritten(delta);
-      }
-
+      listener.uploadCompleted();
       // This will delete unnecessary fake parent directories
       fs.finishedWrite(key);
     } catch (InterruptedException e) {
@@ -175,46 +141,4 @@ public class S3AOutputStream extends OutputStream {
     backupStream.write(b, off, len);
   }
 
-  /**
-   * Listener to progress from AWS regarding transfers.
-   */
-  public static class ProgressableProgressListener implements ProgressListener {
-    private Progressable progress;
-    private FileSystem.Statistics statistics;
-    private long lastBytesTransferred;
-    private Upload upload;
-
-    public ProgressableProgressListener(Upload upload, Progressable progress, 
-        FileSystem.Statistics statistics) {
-      this.upload = upload;
-      this.progress = progress;
-      this.statistics = statistics;
-      this.lastBytesTransferred = 0;
-    }
-
-    public void progressChanged(ProgressEvent progressEvent) {
-      if (progress != null) {
-        progress.progress();
-      }
-
-      // There are 3 http ops here, but this should be close enough for now
-      ProgressEventType pet = progressEvent.getEventType();
-      if (pet == TRANSFER_PART_STARTED_EVENT ||
-          pet == TRANSFER_COMPLETED_EVENT) {
-        statistics.incrementWriteOps(1);
-      }
-
-      long transferred = upload.getProgress().getBytesTransferred();
-      long delta = transferred - lastBytesTransferred;
-      if (statistics != null && delta != 0) {
-        statistics.incrementBytesWritten(delta);
-      }
-
-      lastBytesTransferred = transferred;
-    }
-
-    public long getLastBytesTransferred() {
-      return lastBytesTransferred;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
new file mode 100644
index 0000000..f69159a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStorageStatistics.java
@@ -0,0 +1,104 @@
+/*
+ * 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.fs.s3a;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageStatistics;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Storage statistics for S3A.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class S3AStorageStatistics extends StorageStatistics {
+  private static final Logger LOG = S3AFileSystem.LOG;
+
+  public static final String NAME = "S3AStorageStatistics";
+  private final Map<Statistic, AtomicLong> opsCount =
+      new EnumMap<>(Statistic.class);
+
+  public S3AStorageStatistics() {
+    super(NAME);
+    for (Statistic opType : Statistic.values()) {
+      opsCount.put(opType, new AtomicLong(0));
+    }
+  }
+
+  /**
+   * Increment a specific counter.
+   * @param op operation
+   * @param count increment value
+   * @return the new value
+   */
+  public long incrementCounter(Statistic op, long count) {
+    long updated = opsCount.get(op).addAndGet(count);
+    LOG.debug("{} += {}  ->  {}", op, count, updated);
+    return updated;
+  }
+
+  private class LongIterator implements Iterator<LongStatistic> {
+    private Iterator<Map.Entry<Statistic, AtomicLong>> iterator =
+        Collections.unmodifiableSet(opsCount.entrySet()).iterator();
+
+    @Override
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    @Override
+    public LongStatistic next() {
+      if (!iterator.hasNext()) {
+        throw new NoSuchElementException();
+      }
+      final Map.Entry<Statistic, AtomicLong> entry = iterator.next();
+      return new LongStatistic(entry.getKey().name(), entry.getValue().get());
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public Iterator<LongStatistic> getLongStatistics() {
+    return new LongIterator();
+  }
+
+  @Override
+  public Long getLong(String key) {
+    final Statistic type = Statistic.fromSymbol(key);
+    return type == null ? null : opsCount.get(type).get();
+  }
+
+  @Override
+  public boolean isTracked(String key) {
+    return Statistic.fromSymbol(key) == null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 12d14e2..062fca4 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
+import com.amazonaws.services.s3.model.S3ObjectSummary;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
@@ -29,6 +30,7 @@ import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.file.AccessDeniedException;
+import java.util.Date;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
 
@@ -186,4 +188,50 @@ public final class S3AUtils {
     }
     return builder.toString();
   }
+
+  /**
+   * Create a files status instance from a listing.
+   * @param keyPath path to entry
+   * @param summary summary from AWS
+   * @param blockSize block size to declare.
+   * @return a status entry
+   */
+  public static S3AFileStatus createFileStatus(Path keyPath,
+      S3ObjectSummary summary,
+      long blockSize) {
+    if (objectRepresentsDirectory(summary.getKey(), summary.getSize())) {
+      return new S3AFileStatus(true, true, keyPath);
+    } else {
+      return new S3AFileStatus(summary.getSize(),
+          dateToLong(summary.getLastModified()), keyPath,
+          blockSize);
+    }
+  }
+
+  /**
+   * Predicate: does the object represent a directory?.
+   * @param name object name
+   * @param size object size
+   * @return true if it meets the criteria for being an object
+   */
+  public static boolean objectRepresentsDirectory(final String name,
+      final long size) {
+    return !name.isEmpty()
+        && name.charAt(name.length() - 1) == '/'
+        && size == 0L;
+  }
+
+  /**
+   * Date to long conversion.
+   * Handles null Dates that can be returned by AWS by returning 0
+   * @param date date from AWS query
+   * @return timestamp of the object
+   */
+  public static long dateToLong(final Date date) {
+    if (date == null) {
+      return 0L;
+    }
+
+    return date.getTime();
+  }
 }


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


[04/47] hadoop git commit: HDFS-9476. TestDFSUpgradeFromImage#testUpgradeFromRel1BBWImage occasionally fail. Contributed by Masatake Iwasaki.

Posted by ae...@apache.org.
HDFS-9476. TestDFSUpgradeFromImage#testUpgradeFromRel1BBWImage occasionally fail. Contributed by Masatake Iwasaki.


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

Branch: refs/heads/HDFS-1312
Commit: 69555fca066815053dd9168ebe15868a5c02cdcd
Parents: 16b1cc7
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Jun 2 18:52:47 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Jun 2 18:52:47 2016 +0900

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/TestDFSUpgradeFromImage.java | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/69555fca/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index 1ba36f3..60bea7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -173,7 +173,7 @@ public class TestDFSUpgradeFromImage {
   private static FSInputStream dfsOpenFileWithRetries(DistributedFileSystem dfs,
       String pathName) throws IOException {
     IOException exc = null;
-    for (int tries = 0; tries < 10; tries++) {
+    for (int tries = 0; tries < 30; tries++) {
       try {
         return dfs.dfs.open(pathName);
       } catch (IOException e) {
@@ -184,6 +184,7 @@ public class TestDFSUpgradeFromImage {
         throw exc;
       }
       try {
+        LOG.info("Open failed. " + tries + " times. Retrying.");
         Thread.sleep(1000);
       } catch (InterruptedException ignored) {}
     }
@@ -570,8 +571,17 @@ public class TestDFSUpgradeFromImage {
     String pathStr = path.toString();
     HdfsFileStatus status = dfs.getFileInfo(pathStr);
     if (!status.isDir()) {
-      dfs.recoverLease(pathStr);
-      return;
+      for (int retries = 10; retries > 0; retries--) {
+        if (dfs.recoverLease(pathStr)) {
+          return;
+        } else {
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException ignored) {
+          }
+        }
+      }
+      throw new IOException("Failed to recover lease of " + path);
     }
     byte prev[] = HdfsFileStatus.EMPTY_NAME;
     DirectoryListing dirList;


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


[07/47] hadoop git commit: YARN-5180. Allow ResourceRequest to specify an enforceExecutionType flag. (asuresh)

Posted by ae...@apache.org.
YARN-5180. Allow ResourceRequest to specify an enforceExecutionType flag. (asuresh)


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

Branch: refs/heads/HDFS-1312
Commit: dc26601d8fe27a4223a50601bf7522cc42e8e2f3
Parents: aadb77e
Author: Arun Suresh <as...@apache.org>
Authored: Thu Jun 2 05:18:01 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Thu Jun 2 09:01:02 2016 -0700

----------------------------------------------------------------------
 .../v2/app/rm/RMContainerRequestor.java         |   4 +-
 .../yarn/api/records/ExecutionTypeRequest.java  | 124 +++++++++++++++++++
 .../yarn/api/records/ResourceRequest.java       |  34 ++---
 .../src/main/proto/yarn_protos.proto            |   7 +-
 .../api/impl/TestDistributedScheduling.java     |   9 +-
 .../impl/pb/ExecutionTypeRequestPBImpl.java     |  93 ++++++++++++++
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  15 +++
 .../records/impl/pb/ResourceRequestPBImpl.java  |  52 ++++----
 .../hadoop/yarn/api/TestPBImplRecords.java      |   2 +
 .../nodemanager/scheduler/LocalScheduler.java   |   3 +-
 .../scheduler/TestLocalScheduler.java           |  11 +-
 .../TestDistributedSchedulingService.java       |  17 ++-
 12 files changed, 323 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
index 7030712..f4579ab 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -462,7 +463,8 @@ public abstract class RMContainerRequestor extends RMCommunicator {
       remoteRequest.setCapability(capability);
       remoteRequest.setNumContainers(0);
       remoteRequest.setNodeLabelExpression(nodeLabelExpression);
-      remoteRequest.setExecutionType(executionType);
+      remoteRequest.setExecutionTypeRequest(
+          ExecutionTypeRequest.newInstance(executionType, true));
       reqMap.put(capability, remoteRequest);
     }
     remoteRequest.setNumContainers(remoteRequest.getNumContainers() + 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ExecutionTypeRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ExecutionTypeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ExecutionTypeRequest.java
new file mode 100644
index 0000000..f553a44
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ExecutionTypeRequest.java
@@ -0,0 +1,124 @@
+/**
+ * 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.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * An object of this class represents a specification of the execution
+ * guarantee of the Containers associated with a ResourceRequest. It consists
+ * of an <code>ExecutionType</code> as well as flag that explicitly asks the
+ * configuredScheduler to return Containers of exactly the Execution Type
+ * requested.
+ */
+@Public
+@Evolving
+public abstract class ExecutionTypeRequest {
+
+  @Public
+  @Evolving
+  public static ExecutionTypeRequest newInstance() {
+    return newInstance(ExecutionType.GUARANTEED, false);
+  }
+
+  @Public
+  @Evolving
+  public static ExecutionTypeRequest newInstance(ExecutionType execType,
+      boolean ensureExecutionType) {
+    ExecutionTypeRequest executionTypeRequest =
+        Records.newRecord(ExecutionTypeRequest.class);
+    executionTypeRequest.setExecutionType(execType);
+    executionTypeRequest.setEnforceExecutionType(ensureExecutionType);
+    return executionTypeRequest;
+  }
+
+  /**
+   * Set the <code>ExecutionType</code> of the requested container.
+   *
+   * @param execType
+   *          ExecutionType of the requested container
+   */
+  @Public
+  public abstract void setExecutionType(ExecutionType execType);
+
+  /**
+   * Get <code>ExecutionType</code>.
+   *
+   * @return <code>ExecutionType</code>.
+   */
+  @Public
+  public abstract ExecutionType getExecutionType();
+
+  /**
+   * Set to true to explicitly ask that the Scheduling Authority return
+   * Containers of exactly the Execution Type requested.
+   * @param enforceExecutionType whether ExecutionType request should be
+   *                            strictly honored.
+   */
+  @Public
+  public abstract void setEnforceExecutionType(boolean enforceExecutionType);
+
+
+  /**
+   * Get whether Scheduling Authority should return Containers of exactly the
+   * Execution Type requested for this <code>ResourceRequest</code>.
+   * Defaults to false.
+   * @return whether ExecutionType request should be strictly honored
+   */
+  @Public
+  public abstract boolean getEnforceExecutionType();
+
+  @Override
+  public int hashCode() {
+    final int prime = 2153;
+    int result = 2459;
+    ExecutionType executionType = getExecutionType();
+    boolean ensureExecutionType = getEnforceExecutionType();
+    result = prime * result + ((executionType == null) ? 0 :
+        executionType.hashCode());
+    result = prime * result + (ensureExecutionType ? 0 : 1);
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    ExecutionTypeRequest other = (ExecutionTypeRequest) obj;
+    ExecutionType executionType = getExecutionType();
+    if (executionType == null) {
+      if (other.getExecutionType() != null) {
+        return false;
+      }
+    } else if (executionType != other.getExecutionType()) {
+      return false;
+    }
+    boolean enforceExecutionType = getEnforceExecutionType();
+    return enforceExecutionType == other.getEnforceExecutionType();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
index 8c1fd8d..fbe7e58 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
@@ -80,14 +80,14 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
       Resource capability, int numContainers, boolean relaxLocality,
       String labelExpression) {
     return newInstance(priority, hostName, capability, numContainers,
-        relaxLocality, labelExpression, ExecutionType.GUARANTEED);
+        relaxLocality, labelExpression, ExecutionTypeRequest.newInstance());
   }
 
   @Public
-  @Stable
+  @Evolving
   public static ResourceRequest newInstance(Priority priority, String hostName,
       Resource capability, int numContainers, boolean relaxLocality, String
-      labelExpression, ExecutionType execType) {
+      labelExpression, ExecutionTypeRequest executionTypeRequest) {
     ResourceRequest request = Records.newRecord(ResourceRequest.class);
     request.setPriority(priority);
     request.setResourceName(hostName);
@@ -95,7 +95,7 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
     request.setNumContainers(numContainers);
     request.setRelaxLocality(relaxLocality);
     request.setNodeLabelExpression(labelExpression);
-    request.setExecutionType(execType);
+    request.setExecutionTypeRequest(executionTypeRequest);
     return request;
   }
 
@@ -233,14 +233,16 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
   public abstract boolean getRelaxLocality();
 
   /**
-   * Set the <code>ExecutionType</code> of the requested container.
+   * Set the <code>ExecutionTypeRequest</code> of the requested container.
    *
-   * @param execType
-   *          ExecutionType of the requested container
+   * @param execSpec
+   *          ExecutionTypeRequest of the requested container
    */
   @Public
-  @Stable
-  public abstract void setExecutionType(ExecutionType execType);
+  @Evolving
+  public void setExecutionTypeRequest(ExecutionTypeRequest execSpec) {
+    throw new UnsupportedOperationException();
+  }
 
   /**
    * Get whether locality relaxation is enabled with this
@@ -250,8 +252,10 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
    * <code>ResourceRequest</code>.
    */
   @Public
-  @Stable
-  public abstract ExecutionType getExecutionType();
+  @Evolving
+  public ExecutionTypeRequest getExecutionTypeRequest() {
+    throw new UnsupportedOperationException();
+  }
 
   /**
    * <p>For a request at a network hierarchy level, set whether locality can be relaxed
@@ -353,12 +357,12 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
         return false;
     } else if (!priority.equals(other.getPriority()))
       return false;
-    ExecutionType executionType = getExecutionType();
-    if (executionType == null) {
-      if (other.getExecutionType() != null) {
+    ExecutionTypeRequest execTypeRequest = getExecutionTypeRequest();
+    if (execTypeRequest == null) {
+      if (other.getExecutionTypeRequest() != null) {
         return false;
       }
-    } else if (executionType != other.getExecutionType()) {
+    } else if (!execTypeRequest.equals(other.getExecutionTypeRequest())) {
       return false;
     }
     if (getNodeLabelExpression() == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 42b5410..ca33b28 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -305,7 +305,12 @@ message ResourceRequestProto {
   optional int32 num_containers = 4;
   optional bool relax_locality = 5 [default = true];
   optional string node_label_expression = 6;
-  optional ExecutionTypeProto executionType = 7 [default = GUARANTEED];
+  optional ExecutionTypeRequestProto execution_type_request = 7;
+}
+
+message ExecutionTypeRequestProto {
+  optional ExecutionTypeProto execution_type = 1 [default = GUARANTEED];
+  optional bool enforce_execution_type = 2 [default = false];
 }
 
 enum AMCommandProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
index b4dcf66..6d93eb3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -129,7 +130,9 @@ public class TestDistributedScheduling extends TestAMRMProxy {
           ResourceRequest newRR = ResourceRequest.newInstance(rr
                   .getPriority(), rr.getResourceName(),
               rr.getCapability(), rr.getNumContainers(), rr.getRelaxLocality(),
-              rr.getNodeLabelExpression(), ExecutionType.OPPORTUNISTIC);
+              rr.getNodeLabelExpression(),
+              ExecutionTypeRequest.newInstance(
+                  ExecutionType.OPPORTUNISTIC, true));
           newAskList.add(newRR);
         }
       }
@@ -235,7 +238,9 @@ public class TestDistributedScheduling extends TestAMRMProxy {
           ResourceRequest newRR = ResourceRequest.newInstance(rr
               .getPriority(), rr.getResourceName(),
               rr.getCapability(), rr.getNumContainers(), rr.getRelaxLocality(),
-              rr.getNodeLabelExpression(), ExecutionType.OPPORTUNISTIC);
+              rr.getNodeLabelExpression(),
+              ExecutionTypeRequest.newInstance(
+                  ExecutionType.OPPORTUNISTIC, true));
           newAskList.add(newRR);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ExecutionTypeRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ExecutionTypeRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ExecutionTypeRequestPBImpl.java
new file mode 100644
index 0000000..0037dd3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ExecutionTypeRequestPBImpl.java
@@ -0,0 +1,93 @@
+/**
+ * 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.yarn.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProtoOrBuilder;
+
+/**
+ * Implementation of <code>ExecutionTypeRequest</code>.
+ */
+public class ExecutionTypeRequestPBImpl extends ExecutionTypeRequest {
+  private ExecutionTypeRequestProto proto =
+      ExecutionTypeRequestProto.getDefaultInstance();
+  private ExecutionTypeRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public ExecutionTypeRequestPBImpl() {
+    builder = ExecutionTypeRequestProto.newBuilder();
+  }
+
+  public ExecutionTypeRequestPBImpl(ExecutionTypeRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ExecutionTypeRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  public ExecutionTypeRequestProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ExecutionType getExecutionType() {
+    ExecutionTypeRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasExecutionType()) {
+      return null;
+    }
+    return ProtoUtils.convertFromProtoFormat(p.getExecutionType());
+  }
+
+  @Override
+  public void setExecutionType(ExecutionType execType) {
+    maybeInitBuilder();
+    if (execType == null) {
+      builder.clearExecutionType();
+      return;
+    }
+    builder.setExecutionType(ProtoUtils.convertToProtoFormat(execType));
+  }
+
+  @Override
+  public void setEnforceExecutionType(boolean enforceExecutionType) {
+    maybeInitBuilder();
+    builder.setEnforceExecutionType(enforceExecutionType);
+  }
+
+  @Override
+  public boolean getEnforceExecutionType() {
+    ExecutionTypeRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getEnforceExecutionType();
+  }
+
+  @Override
+  public String toString() {
+    return "{Execution Type: " + getExecutionType()
+        + ", Enforce Execution Type: " + getEnforceExecutionType() + "}";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index 236df90..1a0f30a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
@@ -60,6 +61,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryPolicyProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerTypeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 
@@ -324,4 +326,17 @@ public class ProtoUtils {
       ContainerRetryPolicyProto e) {
     return ContainerRetryPolicy.valueOf(e.name());
   }
+
+  /*
+   * ExecutionTypeRequest
+   */
+  public static ExecutionTypeRequestProto convertToProtoFormat(
+      ExecutionTypeRequest e) {
+    return ((ExecutionTypeRequestPBImpl)e).getProto();
+  }
+
+  public static ExecutionTypeRequest convertFromProtoFormat(
+      ExecutionTypeRequestProto e) {
+    return new ExecutionTypeRequestPBImpl(e);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
index 53ae2cd..fd56f4f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -39,6 +39,7 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
   
   private Priority priority = null;
   private Resource capability = null;
+  private ExecutionTypeRequest executionTypeRequest = null;
   
   
   public ResourceRequestPBImpl() {
@@ -64,6 +65,10 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
     if (this.capability != null) {
       builder.setCapability(convertToProtoFormat(this.capability));
     }
+    if (this.executionTypeRequest != null) {
+      builder.setExecutionTypeRequest(
+          ProtoUtils.convertToProtoFormat(this.executionTypeRequest));
+    }
   }
 
   private void mergeLocalToProto() {
@@ -102,6 +107,29 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
       builder.clearPriority();
     this.priority = priority;
   }
+
+
+  public ExecutionTypeRequest getExecutionTypeRequest() {
+    ResourceRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.executionTypeRequest != null) {
+      return this.executionTypeRequest;
+    }
+    if (!p.hasExecutionTypeRequest()) {
+      return null;
+    }
+    this.executionTypeRequest =
+        ProtoUtils.convertFromProtoFormat(p.getExecutionTypeRequest());
+    return this.executionTypeRequest;
+  }
+
+  public void setExecutionTypeRequest(ExecutionTypeRequest execSpec) {
+    maybeInitBuilder();
+    if (execSpec == null) {
+      builder.clearExecutionTypeRequest();
+    }
+    this.executionTypeRequest = execSpec;
+  }
+
   @Override
   public String getResourceName() {
     ResourceRequestProtoOrBuilder p = viaProto ? proto : builder;
@@ -186,7 +214,7 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
         + ", # Containers: " + getNumContainers()
         + ", Location: " + getResourceName()
         + ", Relax Locality: " + getRelaxLocality()
-        + ", Node Label Expression: " + getNodeLabelExpression() + "}";
+        + ", Execution Spec: " + getExecutionTypeRequest() + "}";
   }
 
   @Override
@@ -207,24 +235,4 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
     }
     builder.setNodeLabelExpression(nodeLabelExpression);
   }
-
-  @Override
-  public ExecutionType getExecutionType() {
-    ResourceRequestProtoOrBuilder p = viaProto ? proto : builder;
-    if (!p.hasExecutionType()) {
-      return null;
-    }
-    return ProtoUtils.convertFromProtoFormat(p.getExecutionType());
-  }
-
-  @Override
-  public void setExecutionType(ExecutionType execType) {
-    maybeInitBuilder();
-    if (execType == null) {
-      builder.clearExecutionType();
-      return;
-    }
-    builder.setExecutionType(ProtoUtils.convertToProtoFormat(execType));
-  }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 14f61b7..91d65b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -123,6 +123,7 @@ import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -463,6 +464,7 @@ public class TestPBImplRecords {
         "http", "localhost", 8080, "file0"));
     typeValueCache.put(SerializedException.class,
         SerializedException.newInstance(new IOException("exception for test")));
+    generateByNewInstance(ExecutionTypeRequest.class);
     generateByNewInstance(LogAggregationContext.class);
     generateByNewInstance(ApplicationId.class);
     generateByNewInstance(ApplicationAttemptId.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java
index fca814b..8e2ceb0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java
@@ -214,7 +214,8 @@ public final class LocalScheduler extends AbstractRequestInterceptor {
     PartitionedResourceRequests partitionedRequests =
         new PartitionedResourceRequests();
     for (ResourceRequest rr : askList) {
-      if (rr.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+      if (rr.getExecutionTypeRequest().getExecutionType() ==
+          ExecutionType.OPPORTUNISTIC) {
         partitionedRequests.getOpportunistic().add(rr);
       } else {
         partitionedRequests.getGuaranteed().add(rr);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java
index e987e79..a1d39f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java
@@ -19,8 +19,8 @@
 package org.apache.hadoop.yarn.server.nodemanager.scheduler;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DistSchedAllocateResponse;
@@ -138,13 +138,15 @@ public class TestLocalScheduler {
 
     AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
     ResourceRequest guaranteedReq = Records.newRecord(ResourceRequest.class);
-    guaranteedReq.setExecutionType(ExecutionType.GUARANTEED);
+    guaranteedReq.setExecutionTypeRequest(
+        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED, true));
     guaranteedReq.setNumContainers(5);
     guaranteedReq.setCapability(Resource.newInstance(2048, 2));
     guaranteedReq.setRelaxLocality(true);
     guaranteedReq.setResourceName("*");
     ResourceRequest opportunisticReq = Records.newRecord(ResourceRequest.class);
-    opportunisticReq.setExecutionType(ExecutionType.OPPORTUNISTIC);
+    opportunisticReq.setExecutionTypeRequest(
+        ExecutionTypeRequest.newInstance(ExecutionType.OPPORTUNISTIC, true));
     opportunisticReq.setNumContainers(4);
     opportunisticReq.setCapability(Resource.newInstance(1024, 4));
     opportunisticReq.setPriority(Priority.newInstance(100));
@@ -167,7 +169,8 @@ public class TestLocalScheduler {
     // New Allocate request
     allocateRequest = Records.newRecord(AllocateRequest.class);
     opportunisticReq = Records.newRecord(ResourceRequest.class);
-    opportunisticReq.setExecutionType(ExecutionType.OPPORTUNISTIC);
+    opportunisticReq.setExecutionTypeRequest(
+        ExecutionTypeRequest.newInstance(ExecutionType.OPPORTUNISTIC, true));
     opportunisticReq.setNumContainers(6);
     opportunisticReq.setCapability(Resource.newInstance(512, 3));
     opportunisticReq.setPriority(Priority.newInstance(100));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc26601d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java
index 5d5ab78..7d2ed33 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java
@@ -39,8 +39,11 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -104,6 +107,13 @@ public class TestDistributedSchedulingService {
         ContainerId.newContainerId(
             ApplicationAttemptId.newInstance(
                 ApplicationId.newInstance(12345, 1), 2), 3));
+    AllocateRequest allReq =
+        (AllocateRequestPBImpl)factory.newRecordInstance(AllocateRequest.class);
+    allReq.setAskList(Arrays.asList(
+        ResourceRequest.newInstance(Priority.UNDEFINED, "a",
+            Resource.newInstance(1, 2), 1, true, "exp",
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true))));
     DistributedSchedulingService service = createService(factory, rmContext, c);
     Server server = service.getServer(rpc, conf, addr, null);
     server.start();
@@ -168,8 +178,7 @@ public class TestDistributedSchedulingService {
     DistSchedAllocateResponse dsAllocResp =
         new DistSchedAllocateResponsePBImpl(
             dsProxy.allocateForDistributedScheduling(null,
-                ((AllocateRequestPBImpl)factory
-                    .newRecordInstance(AllocateRequest.class)).getProto()));
+                ((AllocateRequestPBImpl)allReq).getProto()));
     Assert.assertEquals(
         "h1", dsAllocResp.getNodesForScheduling().get(0).getHost());
 
@@ -235,6 +244,10 @@ public class TestDistributedSchedulingService {
       @Override
       public DistSchedAllocateResponse allocateForDistributedScheduling(
           AllocateRequest request) throws YarnException, IOException {
+        List<ResourceRequest> askList = request.getAskList();
+        Assert.assertEquals(1, askList.size());
+        Assert.assertTrue(askList.get(0)
+            .getExecutionTypeRequest().getEnforceExecutionType());
         DistSchedAllocateResponse resp =
             factory.newRecordInstance(DistSchedAllocateResponse.class);
         resp.setNodesForScheduling(


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


[37/47] hadoop git commit: Revert "Revert "HDFS-10430. Reuse FileSystem#access in TestAsyncDFS. Contributed by Xiaobing Zhou.""

Posted by ae...@apache.org.
Revert "Revert "HDFS-10430. Reuse FileSystem#access in TestAsyncDFS. Contributed by Xiaobing Zhou.""

This reverts commit 8cf47d8589badfc07ef4bca3328a420c7c68abbd.


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

Branch: refs/heads/HDFS-1312
Commit: 7e7b1ae03759da0becfef677e1d5f7a2ed9041c3
Parents: db41e6d
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 6 16:31:38 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 6 16:31:38 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    | 36 +-------------------
 1 file changed, 1 insertion(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e7b1ae0/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
index ddcf492..c7615a9 100644
--- 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
@@ -34,7 +34,6 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -46,19 +45,16 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
 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.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
@@ -445,7 +441,7 @@ public class TestAsyncDFS {
     for (int i = 0; i < NUM_TESTS; i++) {
       assertTrue(fs.exists(dsts[i]));
       FsPermission fsPerm = new FsPermission(permissions[i]);
-      checkAccessPermissions(fs.getFileStatus(dsts[i]), fsPerm.getUserAction());
+      fs.access(dsts[i], fsPerm.getUserAction());
     }
 
     // test setOwner
@@ -474,34 +470,4 @@ public class TestAsyncDFS {
       assertTrue("group2".equals(fs.getFileStatus(dsts[i]).getGroup()));
     }
   }
-
-  static void checkAccessPermissions(FileStatus stat, FsAction mode)
-      throws IOException {
-    checkAccessPermissions(UserGroupInformation.getCurrentUser(), stat, mode);
-  }
-
-  static void checkAccessPermissions(final UserGroupInformation ugi,
-      FileStatus stat, FsAction mode) throws IOException {
-    FsPermission perm = stat.getPermission();
-    String user = ugi.getShortUserName();
-    List<String> groups = Arrays.asList(ugi.getGroupNames());
-
-    if (user.equals(stat.getOwner())) {
-      if (perm.getUserAction().implies(mode)) {
-        return;
-      }
-    } else if (groups.contains(stat.getGroup())) {
-      if (perm.getGroupAction().implies(mode)) {
-        return;
-      }
-    } else {
-      if (perm.getOtherAction().implies(mode)) {
-        return;
-      }
-    }
-    throw new AccessControlException(String.format(
-        "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat
-            .getPath(), stat.getOwner(), stat.getGroup(),
-        stat.isDirectory() ? "d" : "-", perm));
-  }
 }


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


[03/47] hadoop git commit: HADOOP-13131. Add tests to verify that S3A supports SSE-S3 encryption. Contributed by Steve Loughran.

Posted by ae...@apache.org.
HADOOP-13131. Add tests to verify that S3A supports SSE-S3 encryption. Contributed by Steve Loughran.


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

Branch: refs/heads/HDFS-1312
Commit: 16b1cc7af9bd63b65ef50e1056f275a7baf111a2
Parents: 0bc05e4
Author: Chris Nauroth <cn...@apache.org>
Authored: Wed Jun 1 14:49:22 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Wed Jun 1 14:49:22 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/s3a/Constants.java     |  11 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  24 ++++-
 .../src/site/markdown/tools/hadoop-aws/index.md |  22 ++++
 .../hadoop/fs/s3a/AbstractS3ATestBase.java      | 103 ++++++++++++++++++
 .../apache/hadoop/fs/s3a/S3ATestConstants.java  |  76 ++++++++++++++
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  |  73 +++++++++++--
 .../apache/hadoop/fs/s3a/TestS3AEncryption.java | 104 +++++++++++++++++++
 .../TestS3AEncryptionAlgorithmPropagation.java  |  82 +++++++++++++++
 .../s3a/TestS3AEncryptionFastOutputStream.java  |  35 +++++++
 .../fs/s3a/TestS3AFileSystemContract.java       |   1 -
 .../hadoop/fs/s3a/scale/S3AScaleTestBase.java   |  38 +------
 11 files changed, 518 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index be12398..eb859ac 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -127,8 +127,15 @@ public final class Constants {
   public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 14400;
 
   // s3 server-side encryption
-  public static final String SERVER_SIDE_ENCRYPTION_ALGORITHM = 
-    "fs.s3a.server-side-encryption-algorithm";
+  public static final String SERVER_SIDE_ENCRYPTION_ALGORITHM =
+      "fs.s3a.server-side-encryption-algorithm";
+
+  /**
+   * The standard encryption algorithm AWS supports.
+   * Different implementations may support others (or none).
+   */
+  public static final String SERVER_SIDE_ENCRYPTION_AES256 =
+      "AES256";
 
   //override signature algorithm used for signing requests
   public static final String SIGNING_ALGORITHM = "fs.s3a.signing-algorithm";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 581518a..28d9843 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -765,6 +765,28 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
+   * Low-level call to get at the object metadata.
+   * @param path path to the object
+   * @return metadata
+   * @throws IOException IO and object access problems.
+   */
+  @VisibleForTesting
+  public ObjectMetadata getObjectMetadata(Path path) throws IOException {
+    return getObjectMetadata(pathToKey(path));
+  }
+
+  /**
+   * Request object metadata; increments counters in the process.
+   * @param key key
+   * @return the metadata
+   */
+  private ObjectMetadata getObjectMetadata(String key) {
+    ObjectMetadata meta = s3.getObjectMetadata(bucket, key);
+    statistics.incrementReadOps(1);
+    return meta;
+  }
+
+  /**
    * A helper method to delete a list of keys on a s3-backend.
    *
    * @param keysToDelete collection of keys to delete on the s3-backend
@@ -1597,7 +1619,7 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
-   * Get the threshold for multipart files
+   * Get the threshold for multipart files.
    * @return the value as set during initialization
    */
   public long getMultiPartThreshold() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index a81aff9..4f5a077 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -768,3 +768,25 @@ By default, the `parallel-tests` profile runs 4 test suites concurrently.  This
 can be tuned by passing the `testsThreadCount` argument.
 
     mvn -Pparallel-tests -DtestsThreadCount=8 clean test
+
+### Testing against non AWS S3 endpoints.
+
+The S3A filesystem is designed to work with storage endpoints which implement
+the S3 protocols to the extent that the amazon S3 SDK is capable of talking
+to it. We encourage testing against other filesystems and submissions of patches
+which address issues. In particular, we encourage testing of Hadoop release
+candidates, as these third-party endpoints get even less testing than the
+S3 endpoint itself.
+
+
+**Disabling the encryption tests**
+
+If the endpoint doesn't support server-side-encryption, these will fail
+
+      <property>
+        <name>test.fs.s3a.encryption.enabled</name>
+        <value>false</value>
+      </property>
+
+Encryption is only used for those specific test suites with `Encryption` in
+their classname.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
new file mode 100644
index 0000000..b7973b3
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
@@ -0,0 +1,103 @@
+/*
+ * 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.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+
+/**
+ * An extension of the contract test base set up for S3A tests.
+ */
+public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
+    implements S3ATestConstants {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new S3AContract(conf);
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    IOUtils.closeStream(getFileSystem());
+  }
+
+  @Rule
+  public TestName methodName = new TestName();
+
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit-" + methodName.getMethodName());
+  }
+
+  protected Configuration getConfiguration() {
+    return getContract().getConf();
+  }
+
+  /**
+   * Get the filesystem as an S3A filesystem.
+   * @return the typecast FS
+   */
+  @Override
+  public S3AFileSystem getFileSystem() {
+    return (S3AFileSystem) super.getFileSystem();
+  }
+
+  /**
+   * Write a file, read it back, validate the dataset. Overwrites the file
+   * if it is present
+   * @param name filename (will have the test path prepended to it)
+   * @param len length of file
+   * @return the full path to the file
+   * @throws IOException any IO problem
+   */
+  protected Path writeThenReadFile(String name, int len) throws IOException {
+    Path path = path(name);
+    byte[] data = dataset(len, 'a', 'z');
+    writeDataset(getFileSystem(), path, data, data.length, 1024 * 1024, true);
+    ContractTestUtils.verifyFileContents(getFileSystem(), path, data);
+    return path;
+  }
+
+  /**
+   * Assert that an exception failed with a specific status code.
+   * @param e exception
+   * @param code expected status code
+   * @throws AWSS3IOException rethrown if the status code does not match.
+   */
+  protected void assertStatusCode(AWSS3IOException e, int code)
+      throws AWSS3IOException {
+    if (e.getStatusCode() != code) {
+      throw e;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
new file mode 100644
index 0000000..6a4e68c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
@@ -0,0 +1,76 @@
+/*
+ * 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.fs.s3a;
+
+/**
+ * Constants for S3A Testing.
+ */
+public interface S3ATestConstants {
+
+  /**
+   * Prefix for any cross-filesystem scale test options.
+   */
+  String SCALE_TEST = "scale.test.";
+
+  /**
+   * Prefix for s3a-specific scale tests.
+   */
+  String S3A_SCALE_TEST = "fs.s3a.scale.test.";
+
+  /**
+   * Prefix for FS s3a tests.
+   */
+  String TEST_FS_S3A = "test.fs.s3a.";
+
+  /**
+   * Name of the test filesystem.
+   */
+  String TEST_FS_S3A_NAME = TEST_FS_S3A + "name";
+
+  /**
+   * The number of operations to perform: {@value}.
+   */
+  String KEY_OPERATION_COUNT = SCALE_TEST + "operation.count";
+
+  /**
+   * The readahead buffer: {@value}.
+   */
+  String KEY_READ_BUFFER_SIZE = S3A_SCALE_TEST + "read.buffer.size";
+  int DEFAULT_READ_BUFFER_SIZE = 16384;
+
+  /**
+   * Key for a multi MB test file: {@value}.
+   */
+  String KEY_CSVTEST_FILE = S3A_SCALE_TEST + "csvfile";
+
+  /**
+   * Default path for the multi MB test file: {@value}.
+   */
+  String DEFAULT_CSVTEST_FILE = "s3a://landsat-pds/scene_list.gz";
+
+  /**
+   * The default number of operations to perform: {@value}.
+   */
+  long DEFAULT_OPERATION_COUNT = 2005;
+
+  /**
+   * Run the encryption tests?
+   */
+  String KEY_ENCRYPTION_TESTS = TEST_FS_S3A + "encryption.enabled";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index 44bdc02..a4f9b99 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -27,11 +27,30 @@ import java.io.IOException;
 import java.net.URI;
 import java.util.concurrent.Callable;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Utilities for the S3A tests.
+ */
 public class S3ATestUtils {
 
-  public static S3AFileSystem createTestFileSystem(Configuration conf) throws
-      IOException {
-    String fsname = conf.getTrimmed(TestS3AFileSystemContract.TEST_FS_S3A_NAME, "");
+  /**
+   * Create the test filesystem.
+   *
+   * If the test.fs.s3a.name property is not set, this will
+   * trigger a JUnit failure.
+   *
+   * Multipart purging is enabled.
+   * @param conf configuration
+   * @return the FS
+   * @throws IOException IO Problems
+   * @throws AssumptionViolatedException if the FS is not named
+   */
+  public static S3AFileSystem createTestFileSystem(Configuration conf)
+      throws IOException {
+    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
 
 
     boolean liveTest = !StringUtils.isEmpty(fsname);
@@ -44,19 +63,31 @@ public class S3ATestUtils {
       // This doesn't work with our JUnit 3 style test cases, so instead we'll
       // make this whole class not run by default
       throw new AssumptionViolatedException(
-          "No test filesystem in " + TestS3AFileSystemContract.TEST_FS_S3A_NAME);
+          "No test filesystem in " + TEST_FS_S3A_NAME);
     }
     S3AFileSystem fs1 = new S3AFileSystem();
     //enable purging in tests
-    conf.setBoolean(Constants.PURGE_EXISTING_MULTIPART, true);
-    conf.setInt(Constants.PURGE_EXISTING_MULTIPART_AGE, 0);
+    conf.setBoolean(PURGE_EXISTING_MULTIPART, true);
+    conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 0);
     fs1.initialize(testURI, conf);
     return fs1;
   }
 
-  public static FileContext createTestFileContext(Configuration conf) throws
-      IOException {
-    String fsname = conf.getTrimmed(TestS3AFileSystemContract.TEST_FS_S3A_NAME, "");
+  /**
+   * Create a file context for tests.
+   *
+   * If the test.fs.s3a.name property is not set, this will
+   * trigger a JUnit failure.
+   *
+   * Multipart purging is enabled.
+   * @param conf configuration
+   * @return the FS
+   * @throws IOException IO Problems
+   * @throws AssumptionViolatedException if the FS is not named
+   */
+  public static FileContext createTestFileContext(Configuration conf)
+      throws IOException {
+    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
 
     boolean liveTest = !StringUtils.isEmpty(fsname);
     URI testURI = null;
@@ -67,8 +98,8 @@ public class S3ATestUtils {
     if (!liveTest) {
       // This doesn't work with our JUnit 3 style test cases, so instead we'll
       // make this whole class not run by default
-      throw new AssumptionViolatedException(
-          "No test filesystem in " + TestS3AFileSystemContract.TEST_FS_S3A_NAME);
+      throw new AssumptionViolatedException("No test filesystem in "
+          + TEST_FS_S3A_NAME);
     }
     FileContext fc = FileContext.getFileContext(testURI,conf);
     return fc;
@@ -139,4 +170,24 @@ public class S3ATestUtils {
     return ex;
   }
 
+  /**
+   * Turn off FS Caching: use if a filesystem with different options from
+   * the default is required.
+   * @param conf configuration to patch
+   */
+  public static void disableFilesystemCaching(Configuration conf) {
+    conf.setBoolean("fs.s3a.impl.disable.cache", true);
+  }
+
+  /**
+   * Skip a test if encryption tests are disabled.
+   * @param configuration configuration to probe
+   */
+  public static void skipIfEncryptionTestsDisabled(
+      Configuration configuration) {
+    if (!configuration.getBoolean(KEY_ENCRYPTION_TESTS, true)) {
+      skip("Skipping encryption tests");
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryption.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryption.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryption.java
new file mode 100644
index 0000000..43a26ff
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryption.java
@@ -0,0 +1,104 @@
+/*
+ * 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.fs.s3a;
+
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
+/**
+ * Test whether or not encryption works by turning it on. Some checks
+ * are made for different file sizes as there have been reports that the
+ * file length may be rounded up to match word boundaries.
+ */
+public class TestS3AEncryption extends AbstractS3ATestBase {
+  private static final String AES256 = Constants.SERVER_SIDE_ENCRYPTION_AES256;
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    S3ATestUtils.disableFilesystemCaching(conf);
+    conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
+        AES256);
+    return conf;
+  }
+
+  private static final int[] SIZES = {
+      0, 1, 2, 3, 4, 5, 254, 255, 256, 257, 2 ^ 10 - 3, 2 ^ 11 - 2, 2 ^ 12 - 1
+  };
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    IOUtils.closeStream(getFileSystem());
+  }
+
+  @Test
+  public void testEncryption() throws Throwable {
+    for (int size: SIZES) {
+      validateEncryptionForFilesize(size);
+    }
+  }
+
+  @Test
+  public void testEncryptionOverRename() throws Throwable {
+    skipIfEncryptionTestsDisabled(getConfiguration());
+    Path src = path(createFilename(1024));
+    byte[] data = dataset(1024, 'a', 'z');
+    S3AFileSystem fs = getFileSystem();
+    writeDataset(fs, src, data, data.length, 1024 * 1024, true);
+    ContractTestUtils.verifyFileContents(fs, src, data);
+    Path dest = path(src.getName() + "-copy");
+    fs.rename(src, dest);
+    ContractTestUtils.verifyFileContents(fs, dest, data);
+    assertEncrypted(dest);
+  }
+
+  protected void validateEncryptionForFilesize(int len) throws IOException {
+    skipIfEncryptionTestsDisabled(getConfiguration());
+    describe("Create an encrypted file of size " + len);
+    String src = createFilename(len);
+    Path path = writeThenReadFile(src, len);
+    assertEncrypted(path);
+    rm(getFileSystem(), path, false, false);
+  }
+
+  private String createFilename(int len) {
+    return String.format("%s-%04x", methodName.getMethodName(), len);
+  }
+
+  /**
+   * Assert that at path references an encrypted blob.
+   * @param path path
+   * @throws IOException on a failure
+   */
+  private void assertEncrypted(Path path) throws IOException {
+    ObjectMetadata md = getFileSystem().getObjectMetadata(path);
+    assertEquals(AES256, md.getSSEAlgorithm());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionAlgorithmPropagation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionAlgorithmPropagation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionAlgorithmPropagation.java
new file mode 100644
index 0000000..04ca830
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionAlgorithmPropagation.java
@@ -0,0 +1,82 @@
+/*
+ * 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.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+
+/**
+ * Test whether or not encryption settings propagate by choosing an invalid
+ * one. We expect the write to fail with a 400 bad request error
+ */
+public class TestS3AEncryptionAlgorithmPropagation extends AbstractS3ATestBase {
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    S3ATestUtils.disableFilesystemCaching(conf);
+    conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
+        "DES");
+    return conf;
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    IOUtils.closeStream(getFileSystem());
+  }
+
+  @Test
+  public void testEncrypt0() throws Throwable {
+    writeThenReadFileToFailure(0);
+  }
+
+  @Test
+  public void testEncrypt256() throws Throwable {
+    writeThenReadFileToFailure(256);
+  }
+
+  /**
+   * Make this a no-op so test setup doesn't fail.
+   * @param path path path
+   * @throws IOException on any failure
+   */
+  @Override
+  protected void mkdirs(Path path) throws IOException {
+
+  }
+
+  protected void writeThenReadFileToFailure(int len) throws IOException {
+    skipIfEncryptionTestsDisabled(getConfiguration());
+    describe("Create an encrypted file of size " + len);
+    try {
+      writeThenReadFile(methodName.getMethodName() + '-' + len, len);
+      fail("Expected an exception about an illegal encryption algorithm");
+    } catch (AWSS3IOException e) {
+      assertStatusCode(e, 400);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionFastOutputStream.java
new file mode 100644
index 0000000..1fa8486
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEncryptionFastOutputStream.java
@@ -0,0 +1,35 @@
+/*
+ * 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.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Run the encryption tests against the Fast output stream.
+ * This verifies that both file writing paths can encrypt their data.
+ */
+public class TestS3AEncryptionFastOutputStream extends TestS3AEncryption {
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setBoolean(Constants.FAST_UPLOAD, true);
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java
index 5c88358..6315a80 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystemContract.java
@@ -37,7 +37,6 @@ public class TestS3AFileSystemContract extends FileSystemContractBaseTest {
 
   protected static final Logger LOG =
       LoggerFactory.getLogger(TestS3AFileSystemContract.class);
-  public static final String TEST_FS_S3A_NAME = "test.fs.s3a.name";
 
   @Override
   public void setUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/16b1cc7a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
index 20ef6ec..d65f693 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.fs.s3a.S3AInputStream;
 import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.S3ATestConstants;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -45,11 +46,7 @@ import static org.junit.Assume.assumeTrue;
  * Base class for scale tests; here is where the common scale configuration
  * keys are defined.
  */
-public class S3AScaleTestBase extends Assert {
-
-  public static final String SCALE_TEST = "scale.test.";
-
-  public static final String S3A_SCALE_TEST = "fs.s3a.scale.test.";
+public class S3AScaleTestBase extends Assert implements S3ATestConstants {
 
   @Rule
   public TestName methodName = new TestName();
@@ -59,37 +56,6 @@ public class S3AScaleTestBase extends Assert {
     Thread.currentThread().setName("JUnit");
   }
 
-  /**
-   * The number of operations to perform: {@value}.
-   */
-  public static final String KEY_OPERATION_COUNT =
-      SCALE_TEST + "operation.count";
-
-  /**
-   * The readahead buffer: {@value}.
-   */
-  public static final String KEY_READ_BUFFER_SIZE =
-      S3A_SCALE_TEST + "read.buffer.size";
-
-  public static final int DEFAULT_READ_BUFFER_SIZE = 16384;
-
-  /**
-   * Key for a multi MB test file: {@value}.
-   */
-  public static final String KEY_CSVTEST_FILE =
-      S3A_SCALE_TEST + "csvfile";
-
-  /**
-   * Default path for the multi MB test file: {@value}.
-   */
-  public static final String DEFAULT_CSVTEST_FILE
-      = "s3a://landsat-pds/scene_list.gz";
-
-  /**
-   * The default number of operations to perform: {@value}.
-   */
-  public static final long DEFAULT_OPERATION_COUNT = 2005;
-
   protected S3AFileSystem fs;
 
   protected static final Logger LOG =


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


[13/47] hadoop git commit: YARN-1815. Work preserving recovery of Unmanged AMs. Contributed by Subru Krishnan

Posted by ae...@apache.org.
YARN-1815. Work preserving recovery of Unmanged AMs. Contributed by Subru Krishnan


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

Branch: refs/heads/HDFS-1312
Commit: 097baaaebae021c47bb7d69aa1ff1a2440df5166
Parents: c58a59f
Author: Jian He <ji...@apache.org>
Authored: Fri Jun 3 10:49:30 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Fri Jun 3 10:49:30 2016 -0700

----------------------------------------------------------------------
 .../rmapp/attempt/RMAppAttemptImpl.java         | 28 +++---
 .../scheduler/AbstractYarnScheduler.java        |  8 --
 .../yarn/server/resourcemanager/MockRM.java     | 14 +++
 .../TestWorkPreservingRMRestart.java            | 92 ++++++++++++++++++++
 .../attempt/TestRMAppAttemptTransitions.java    |  3 +-
 5 files changed, 122 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/097baaae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 1e2a293..75090fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -354,8 +354,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
               RMAppAttemptState.FAILED))
 
        // Transitions from RUNNING State
-      .addTransition(RMAppAttemptState.RUNNING,
-          EnumSet.of(RMAppAttemptState.FINAL_SAVING, RMAppAttemptState.FINISHED),
+      .addTransition(RMAppAttemptState.RUNNING, RMAppAttemptState.FINAL_SAVING,
           RMAppAttemptEventType.UNREGISTERED, new AMUnregisteredTransition())
       .addTransition(RMAppAttemptState.RUNNING, RMAppAttemptState.RUNNING,
           RMAppAttemptEventType.STATUS_UPDATE, new StatusUpdateTransition())
@@ -1714,25 +1713,26 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
-  private static final class AMUnregisteredTransition implements
-      MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {
+  private static final class AMUnregisteredTransition extends BaseTransition {
 
     @Override
-    public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
+    public void transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
       // Tell the app
       if (appAttempt.getSubmissionContext().getUnmanagedAM()) {
+        // YARN-1815: Saving the attempt final state so that we do not recover
+        // the finished Unmanaged AM post RM failover
         // Unmanaged AMs have no container to wait for, so they skip
         // the FINISHING state and go straight to FINISHED.
-        appAttempt.updateInfoOnAMUnregister(event);
-        new FinalTransition(RMAppAttemptState.FINISHED).transition(
-            appAttempt, event);
-        return RMAppAttemptState.FINISHED;
+        appAttempt.rememberTargetTransitionsAndStoreState(event,
+            new AMFinishedAfterFinalSavingTransition(event),
+            RMAppAttemptState.FINISHED, RMAppAttemptState.FINISHED);
+      } else {
+        // Saving the attempt final state
+        appAttempt.rememberTargetTransitionsAndStoreState(event,
+            new FinalStateSavedAfterAMUnregisterTransition(),
+            RMAppAttemptState.FINISHING, RMAppAttemptState.FINISHED);
       }
-      // Saving the attempt final state
-      appAttempt.rememberTargetTransitionsAndStoreState(event,
-        new FinalStateSavedAfterAMUnregisterTransition(),
-        RMAppAttemptState.FINISHING, RMAppAttemptState.FINISHED);
       ApplicationId applicationId =
           appAttempt.getAppAttemptId().getApplicationId();
 
@@ -1743,7 +1743,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // AppAttempt to App after this point of time is AM/AppAttempt Finished.
       appAttempt.eventHandler.handle(new RMAppEvent(applicationId,
         RMAppEventType.ATTEMPT_UNREGISTERED));
-      return RMAppAttemptState.FINAL_SAVING;
+      return;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/097baaae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 8f03de2..354dcb2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -354,14 +354,6 @@ public abstract class AbstractYarnScheduler
         continue;
       }
 
-      // Unmanaged AM recovery is addressed in YARN-1815
-      if (rmApp.getApplicationSubmissionContext().getUnmanagedAM()) {
-        LOG.info("Skip recovering container " + container + " for unmanaged AM."
-            + rmApp.getApplicationId());
-        killOrphanContainerOnNode(nm, container);
-        continue;
-      }
-
       SchedulerApplication<T> schedulerApp = applications.get(appId);
       if (schedulerApp == null) {
         LOG.info("Skip recovering container  " + container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/097baaae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 982724e..dc749be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -947,6 +947,20 @@ public class MockRM extends ResourceManager {
     return am;
   }
 
+  public static MockAM launchUAM(RMApp app, MockRM rm, MockNM nm)
+      throws Exception {
+    // UAMs go directly to LAUNCHED state
+    rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);
+    RMAppAttempt attempt = app.getCurrentAppAttempt();
+    waitForSchedulerAppAttemptAdded(attempt.getAppAttemptId(), rm);
+    System.out.println("Launch AM " + attempt.getAppAttemptId());
+    nm.nodeHeartbeat(true);
+    MockAM am = new MockAM(rm.getRMContext(), rm.masterService,
+        attempt.getAppAttemptId());
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+    return am;
+  }
+
   public static RMAppAttempt waitForAttemptScheduled(RMApp app, MockRM rm)
       throws Exception {
     rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/097baaae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
index 458e8c3..7e5915b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
@@ -1434,4 +1434,96 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // check that attempt state is recovered correctly.
     assertEquals(RMAppAttemptState.FINISHED, recoveredApp1.getCurrentAppAttempt().getState());
   }
+
+  @Test(timeout = 600000)
+  public void testUAMRecoveryOnRMWorkPreservingRestart() throws Exception {
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+
+    // start RM
+    rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
+    nm1.registerNode();
+
+    // create app and launch the UAM
+    RMApp app0 = rm1.submitApp(200, true);
+    MockAM am0 = MockRM.launchUAM(app0, rm1, nm1);
+    am0.registerAppAttempt();
+
+    // Allocate containers to UAM
+    int numContainers = 2;
+    am0.allocate("127.0.0.1", 1000, numContainers,
+        new ArrayList<ContainerId>());
+    nm1.nodeHeartbeat(true);
+    List<Container> conts = am0.allocate(new ArrayList<ResourceRequest>(),
+        new ArrayList<ContainerId>()).getAllocatedContainers();
+    Assert.assertTrue(conts.isEmpty());
+    while (conts.size() == 0) {
+      nm1.nodeHeartbeat(true);
+      conts.addAll(am0.allocate(new ArrayList<ResourceRequest>(),
+          new ArrayList<ContainerId>()).getAllocatedContainers());
+      Thread.sleep(500);
+    }
+    Assert.assertFalse(conts.isEmpty());
+
+    // start new RM
+    rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    rm2.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
+    rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.LAUNCHED);
+
+    // recover app
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+    RMApp recoveredApp =
+        rm2.getRMContext().getRMApps().get(app0.getApplicationId());
+    NMContainerStatus container1 = TestRMRestart
+        .createNMContainerStatus(am0.getApplicationAttemptId(), 1,
+            ContainerState.RUNNING);
+    NMContainerStatus container2 = TestRMRestart
+        .createNMContainerStatus(am0.getApplicationAttemptId(), 2,
+            ContainerState.RUNNING);
+    nm1.registerNode(Arrays.asList(container1, container2), null);
+
+    // Wait for RM to settle down on recovering containers;
+    waitForNumContainersToRecover(2, rm2, am0.getApplicationAttemptId());
+
+    // retry registerApplicationMaster() after RM restart.
+    am0.setAMRMProtocol(rm2.getApplicationMasterService(), rm2.getRMContext());
+    am0.registerAppAttempt(true);
+
+    // Check if UAM is correctly recovered on restart
+    rm2.waitForState(app0.getApplicationId(), RMAppState.RUNNING);
+    rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.RUNNING);
+
+    // Check if containers allocated to UAM are recovered
+    Map<ApplicationId, SchedulerApplication> schedulerApps =
+        ((AbstractYarnScheduler) rm2.getResourceScheduler())
+            .getSchedulerApplications();
+    SchedulerApplication schedulerApp =
+        schedulerApps.get(recoveredApp.getApplicationId());
+    SchedulerApplicationAttempt schedulerAttempt =
+        schedulerApp.getCurrentAppAttempt();
+    Assert.assertEquals(numContainers,
+        schedulerAttempt.getLiveContainers().size());
+
+    // Check if UAM is able to heart beat
+    Assert.assertNotNull(am0.doHeartbeat());
+
+    // Complete the UAM
+    am0.unregisterAppAttempt(false);
+    rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.FINISHED);
+    rm2.waitForState(app0.getApplicationId(), RMAppState.FINISHED);
+    Assert.assertEquals(FinalApplicationStatus.SUCCEEDED,
+        recoveredApp.getFinalApplicationStatus());
+
+    // Restart RM once more to check UAM is not re-run
+    MockRM rm3 = new MockRM(conf, memStore);
+    rm3.start();
+    recoveredApp = rm3.getRMContext().getRMApps().get(app0.getApplicationId());
+    Assert.assertEquals(RMAppState.FINISHED, recoveredApp.getState());
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/097baaae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
index 93f17e7..3143b94 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
@@ -596,8 +596,8 @@ public class TestRMAppAttemptTransitions {
     } else {
       assertEquals(getProxyUrl(applicationAttempt),
           applicationAttempt.getTrackingUrl());
-      verifyAttemptFinalStateSaved();
     }
+    verifyAttemptFinalStateSaved();
     assertEquals(finishedContainerCount, applicationAttempt
         .getJustFinishedContainers().size());
     Assert.assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
@@ -735,6 +735,7 @@ public class TestRMAppAttemptTransitions {
     applicationAttempt.handle(new RMAppAttemptUnregistrationEvent(
         applicationAttempt.getAppAttemptId(), url, finalStatus,
         diagnostics));
+    sendAttemptUpdateSavedEvent(applicationAttempt);
     testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1,
         true);
     assertFalse(transferStateFromPreviousAttempt);


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


[27/47] hadoop git commit: Revert "HADOOP-13168. Support Future.get with timeout in ipc async calls."

Posted by ae...@apache.org.
Revert "HADOOP-13168. Support Future.get with timeout in ipc async calls."

This reverts commit 42c22f7e3d6e88bf1115f617f6e803288886d1ac.


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

Branch: refs/heads/HDFS-1312
Commit: e4450d47f19131818e1c040b6bd8d85ae8250475
Parents: b82c74b
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 18:09:16 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 18:09:16 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/Client.java | 119 ++++++++++--------
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |  62 +++++-----
 .../apache/hadoop/util/concurrent/AsyncGet.java |  60 ---------
 .../hadoop/util/concurrent/AsyncGetFuture.java  |  73 -----------
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     | 124 ++++++++-----------
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |  24 +++-
 .../ClientNamenodeProtocolTranslatorPB.java     |  33 +++--
 7 files changed, 185 insertions(+), 310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4450d47/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index d1d5b17..9be4649 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -18,10 +18,46 @@
 
 package org.apache.hadoop.ipc;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.google.protobuf.CodedOutputStream;
+import static org.apache.hadoop.ipc.RpcConstants.*;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import javax.net.SocketFactory;
+import javax.security.sasl.Sasl;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -57,25 +93,14 @@ import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.util.concurrent.AsyncGet;
-import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.Tracer;
 
-import javax.net.SocketFactory;
-import javax.security.sasl.Sasl;
-import java.io.*;
-import java.net.*;
-import java.security.PrivilegedExceptionAction;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.apache.hadoop.ipc.RpcConstants.CONNECTION_CONTEXT_CALL_ID;
-import static org.apache.hadoop.ipc.RpcConstants.PING_CALL_ID;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.protobuf.CodedOutputStream;
 
 /** A client for an IPC service.  IPC calls take a single {@link Writable} as a
  * parameter, and return a {@link Writable} as their value.  A service runs on
@@ -94,8 +119,8 @@ public class Client implements AutoCloseable {
 
   private static final ThreadLocal<Integer> callId = new ThreadLocal<Integer>();
   private static final ThreadLocal<Integer> retryCount = new ThreadLocal<Integer>();
-  private static final ThreadLocal<Future<?>> ASYNC_RPC_RESPONSE
-      = new ThreadLocal<>();
+  private static final ThreadLocal<Future<?>>
+      RETURN_RPC_RESPONSE = new ThreadLocal<>();
   private static final ThreadLocal<Boolean> asynchronousMode =
       new ThreadLocal<Boolean>() {
         @Override
@@ -106,8 +131,8 @@ public class Client implements AutoCloseable {
 
   @SuppressWarnings("unchecked")
   @Unstable
-  public static <T> Future<T> getAsyncRpcResponse() {
-    return (Future<T>) ASYNC_RPC_RESPONSE.get();
+  public static <T> Future<T> getReturnRpcResponse() {
+    return (Future<T>) RETURN_RPC_RESPONSE.get();
   }
 
   /** Set call id and retry count for the next call. */
@@ -354,11 +379,6 @@ public class Client implements AutoCloseable {
       }
     }
 
-    @Override
-    public String toString() {
-      return getClass().getSimpleName() + id;
-    }
-
     /** Indicate when the call is complete and the
      * value or error are available.  Notifies by default.  */
     protected synchronized void callComplete() {
@@ -1393,32 +1413,27 @@ public class Client implements AutoCloseable {
     }
 
     if (isAsynchronousMode()) {
-      final AsyncGet<Writable, IOException> asyncGet
-          = new AsyncGet<Writable, IOException>() {
+      Future<Writable> returnFuture = new AbstractFuture<Writable>() {
+        private final AtomicBoolean callled = new AtomicBoolean(false);
         @Override
-        public Writable get(long timeout, TimeUnit unit)
-            throws IOException, TimeoutException{
-          boolean done = true;
-          try {
-            final Writable w = getRpcResponse(call, connection, timeout, unit);
-            if (w == null) {
-              done = false;
-              throw new TimeoutException(call + " timed out "
-                  + timeout + " " + unit);
-            }
-            return w;
-          } finally {
-            if (done) {
+        public Writable get() throws InterruptedException, ExecutionException {
+          if (callled.compareAndSet(false, true)) {
+            try {
+              set(getRpcResponse(call, connection));
+            } catch (IOException ie) {
+              setException(ie);
+            } finally {
               releaseAsyncCall();
             }
           }
+          return super.get();
         }
       };
 
-      ASYNC_RPC_RESPONSE.set(new AsyncGetFuture<>(asyncGet));
+      RETURN_RPC_RESPONSE.set(returnFuture);
       return null;
     } else {
-      return getRpcResponse(call, connection, -1, null);
+      return getRpcResponse(call, connection);
     }
   }
 
@@ -1454,18 +1469,12 @@ public class Client implements AutoCloseable {
     return asyncCallCounter.get();
   }
 
-  /** @return the rpc response or, in case of timeout, null. */
-  private Writable getRpcResponse(final Call call, final Connection connection,
-      final long timeout, final TimeUnit unit) throws IOException {
+  private Writable getRpcResponse(final Call call, final Connection connection)
+      throws IOException {
     synchronized (call) {
       while (!call.done) {
         try {
-          final long waitTimeout = AsyncGet.Util.asyncGetTimeout2WaitTimeout(
-              timeout, unit);
-          call.wait(waitTimeout); // wait for the result
-          if (waitTimeout > 0 && !call.done) {
-            return null;
-          }
+          call.wait();                           // wait for the result
         } catch (InterruptedException ie) {
           Thread.currentThread().interrupt();
           throw new InterruptedIOException("Call interrupted");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4450d47/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 0f43fc6..8fcdb78 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -18,9 +18,21 @@
 
 package org.apache.hadoop.ipc;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.*;
-import com.google.protobuf.Descriptors.MethodDescriptor;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.net.SocketFactory;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -40,23 +52,17 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.util.concurrent.AsyncGet;
 import org.apache.htrace.core.TraceScope;
 import org.apache.htrace.core.Tracer;
 
-import javax.net.SocketFactory;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.net.InetSocketAddress;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.GeneratedMessage;
+import com.google.protobuf.Message;
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.TextFormat;
 
 /**
  * RPC Engine for for protobuf based RPCs.
@@ -64,8 +70,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 @InterfaceStability.Evolving
 public class ProtobufRpcEngine implements RpcEngine {
   public static final Log LOG = LogFactory.getLog(ProtobufRpcEngine.class);
-  private static final ThreadLocal<AsyncGet<Message, Exception>>
-      ASYNC_RETURN_MESSAGE = new ThreadLocal<>();
+  private static final ThreadLocal<Callable<?>>
+      RETURN_MESSAGE_CALLBACK = new ThreadLocal<>();
 
   static { // Register the rpcRequest deserializer for WritableRpcEngine 
     org.apache.hadoop.ipc.Server.registerProtocolEngine(
@@ -75,9 +81,10 @@ public class ProtobufRpcEngine implements RpcEngine {
 
   private static final ClientCache CLIENTS = new ClientCache();
 
+  @SuppressWarnings("unchecked")
   @Unstable
-  public static AsyncGet<Message, Exception> getAsyncReturnMessage() {
-    return ASYNC_RETURN_MESSAGE.get();
+  public static <T> Callable<T> getReturnMessageCallback() {
+    return (Callable<T>) RETURN_MESSAGE_CALLBACK.get();
   }
 
   public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
@@ -256,17 +263,14 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
       
       if (Client.isAsynchronousMode()) {
-        final Future<RpcResponseWrapper> frrw = Client.getAsyncRpcResponse();
-        final AsyncGet<Message, Exception> asyncGet
-            = new AsyncGet<Message, Exception>() {
+        final Future<RpcResponseWrapper> frrw = Client.getReturnRpcResponse();
+        Callable<Message> callback = new Callable<Message>() {
           @Override
-          public Message get(long timeout, TimeUnit unit) throws Exception {
-            final RpcResponseWrapper rrw = timeout < 0?
-                frrw.get(): frrw.get(timeout, unit);
-            return getReturnMessage(method, rrw);
+          public Message call() throws Exception {
+            return getReturnMessage(method, frrw.get());
           }
         };
-        ASYNC_RETURN_MESSAGE.set(asyncGet);
+        RETURN_MESSAGE_CALLBACK.set(callback);
         return null;
       } else {
         return getReturnMessage(method, val);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4450d47/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
deleted file mode 100644
index 5eac869..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
+++ /dev/null
@@ -1,60 +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.util.concurrent;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-/**
- * This interface defines an asynchronous {@link #get(long, TimeUnit)} method.
- *
- * When the return value is still being computed, invoking
- * {@link #get(long, TimeUnit)} will result in a {@link TimeoutException}.
- * The method should be invoked again and again
- * until the underlying computation is completed.
- *
- * @param <R> The type of the return value.
- * @param <E> The exception type that the underlying implementation may throw.
- */
-public interface AsyncGet<R, E extends Throwable> {
-  /**
-   * Get the result.
-   *
-   * @param timeout The maximum time period to wait.
-   *                When timeout == 0, it does not wait at all.
-   *                When timeout < 0, it waits indefinitely.
-   * @param unit The unit of the timeout value
-   * @return the result, which is possibly null.
-   * @throws E an exception thrown by the underlying implementation.
-   * @throws TimeoutException if it cannot return after the given time period.
-   * @throws InterruptedException if the thread is interrupted.
-   */
-  R get(long timeout, TimeUnit unit)
-      throws E, TimeoutException, InterruptedException;
-
-  /** Utility */
-  class Util {
-    /**
-     * @return {@link Object#wait(long)} timeout converted
-     *         from {@link #get(long, TimeUnit)} timeout.
-     */
-    public static long asyncGetTimeout2WaitTimeout(long timeout, TimeUnit unit){
-      return timeout < 0? 0: timeout == 0? 1:unit.toMillis(timeout);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4450d47/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGetFuture.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGetFuture.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGetFuture.java
deleted file mode 100644
index d687867..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGetFuture.java
+++ /dev/null
@@ -1,73 +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.util.concurrent;
-
-import com.google.common.util.concurrent.AbstractFuture;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/** A {@link Future} implemented using an {@link AsyncGet} object. */
-public class AsyncGetFuture<T, E extends Throwable> extends AbstractFuture<T> {
-  public static final Log LOG = LogFactory.getLog(AsyncGetFuture.class);
-
-  private final AtomicBoolean called = new AtomicBoolean(false);
-  private final AsyncGet<T, E> asyncGet;
-
-  public AsyncGetFuture(AsyncGet<T, E> asyncGet) {
-    this.asyncGet = asyncGet;
-  }
-
-  private void callAsyncGet(long timeout, TimeUnit unit) {
-    if (!isCancelled() && called.compareAndSet(false, true)) {
-      try {
-        set(asyncGet.get(timeout, unit));
-      } catch (TimeoutException te) {
-        LOG.trace("TRACE", te);
-        called.compareAndSet(true, false);
-      } catch (Throwable e) {
-        LOG.trace("TRACE", e);
-        setException(e);
-      }
-    }
-  }
-
-  @Override
-  public T get() throws InterruptedException, ExecutionException {
-    callAsyncGet(-1, TimeUnit.MILLISECONDS);
-    return super.get();
-  }
-
-  @Override
-  public T get(long timeout, TimeUnit unit)
-      throws InterruptedException, TimeoutException, ExecutionException {
-    callAsyncGet(timeout, unit);
-    return super.get(0, TimeUnit.MILLISECONDS);
-  }
-
-  @Override
-  public boolean isDone() {
-    callAsyncGet(0, TimeUnit.MILLISECONDS);
-    return super.isDone();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4450d47/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
index 0ad191b..8ee3a2c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
@@ -18,6 +18,20 @@
 
 package org.apache.hadoop.ipc;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+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;
@@ -34,17 +48,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.*;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
 public class TestAsyncIPC {
 
   private static Configuration conf;
@@ -84,50 +87,26 @@ public class TestAsyncIPC {
         try {
           final long param = TestIPC.RANDOM.nextLong();
           TestIPC.call(client, param, server, conf);
-          returnFutures.put(i, Client.getAsyncRpcResponse());
+          Future<LongWritable> returnFuture = Client.getReturnRpcResponse();
+          returnFutures.put(i, returnFuture);
           expectedValues.put(i, param);
         } catch (Exception e) {
+          LOG.fatal("Caught: " + StringUtils.stringifyException(e));
           failed = true;
-          throw new RuntimeException(e);
         }
       }
     }
 
-    void assertReturnValues() throws InterruptedException, ExecutionException {
+    public void waitForReturnValues() throws InterruptedException,
+        ExecutionException {
       for (int i = 0; i < count; i++) {
         LongWritable value = returnFutures.get(i).get();
-        Assert.assertEquals("call" + i + " failed.",
-            expectedValues.get(i).longValue(), value.get());
-      }
-      Assert.assertFalse(failed);
-    }
-
-    void assertReturnValues(long timeout, TimeUnit unit)
-        throws InterruptedException, ExecutionException {
-      final boolean[] checked = new boolean[count];
-      for(boolean done = false; !done;) {
-        done = true;
-        for (int i = 0; i < count; i++) {
-          if (checked[i]) {
-            continue;
-          } else {
-            done = false;
-          }
-
-          final LongWritable value;
-          try {
-            value = returnFutures.get(i).get(timeout, unit);
-          } catch (TimeoutException e) {
-            LOG.info("call" + i + " caught ", e);
-            continue;
-          }
-
-          Assert.assertEquals("call" + i + " failed.",
-              expectedValues.get(i).longValue(), value.get());
-          checked[i] = true;
+        if (expectedValues.get(i) != value.get()) {
+          LOG.fatal(String.format("Call-%d failed!", i));
+          failed = true;
+          break;
         }
       }
-      Assert.assertFalse(failed);
     }
   }
 
@@ -204,7 +183,8 @@ public class TestAsyncIPC {
 
     private void doCall(final int idx, final long param) throws IOException {
       TestIPC.call(client, param, server, conf);
-      returnFutures.put(idx, Client.getAsyncRpcResponse());
+      Future<LongWritable> returnFuture = Client.getReturnRpcResponse();
+      returnFutures.put(idx, returnFuture);
       expectedValues.put(idx, param);
     }
 
@@ -253,7 +233,10 @@ public class TestAsyncIPC {
     }
     for (int i = 0; i < callerCount; i++) {
       callers[i].join();
-      callers[i].assertReturnValues();
+      callers[i].waitForReturnValues();
+      String msg = String.format("Expected not failed for caller-%d: %s.", i,
+          callers[i]);
+      assertFalse(msg, callers[i].failed);
     }
     for (int i = 0; i < clientCount; i++) {
       clients[i].stop();
@@ -275,37 +258,25 @@ public class TestAsyncIPC {
     try {
       AsyncCaller caller = new AsyncCaller(client, addr, callCount);
       caller.run();
-      caller.assertReturnValues();
-      caller.assertReturnValues();
-      caller.assertReturnValues();
-      Assert.assertEquals(asyncCallCount, client.getAsyncCallCount());
-    } finally {
-      client.stop();
-      server.stop();
-    }
-  }
 
-  @Test(timeout = 60000)
-  public void testFutureGetWithTimeout() throws IOException,
-      InterruptedException, ExecutionException {
-//    GenericTestUtils.setLogLevel(AsyncGetFuture.LOG, Level.ALL);
-    final Server server = new TestIPC.TestServer(10, true, conf);
-    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    server.start();
+      caller.waitForReturnValues();
+      String msg = String.format(
+          "First time, expected not failed for caller: %s.", caller);
+      assertFalse(msg, caller.failed);
 
-    final Client client = new Client(LongWritable.class, conf);
+      caller.waitForReturnValues();
+      assertTrue(asyncCallCount == client.getAsyncCallCount());
+      msg = String.format("Second time, expected not failed for caller: %s.",
+          caller);
+      assertFalse(msg, caller.failed);
 
-    try {
-      final AsyncCaller caller = new AsyncCaller(client, addr, 10);
-      caller.run();
-      caller.assertReturnValues(10, TimeUnit.MILLISECONDS);
+      assertTrue(asyncCallCount == client.getAsyncCallCount());
     } finally {
       client.stop();
       server.stop();
     }
   }
 
-
   public void internalTestAsyncCallLimit(int handlerCount, boolean handlerSleep,
       int clientCount, int callerCount, int callCount) throws IOException,
       InterruptedException, ExecutionException {
@@ -396,7 +367,9 @@ public class TestAsyncIPC {
       server.start();
       final AsyncCaller caller = new AsyncCaller(client, addr, 4);
       caller.run();
-      caller.assertReturnValues();
+      caller.waitForReturnValues();
+      String msg = String.format("Expected not failed for caller: %s.", caller);
+      assertFalse(msg, caller.failed);
     } finally {
       client.stop();
       server.stop();
@@ -433,7 +406,9 @@ public class TestAsyncIPC {
       server.start();
       final AsyncCaller caller = new AsyncCaller(client, addr, 10);
       caller.run();
-      caller.assertReturnValues();
+      caller.waitForReturnValues();
+      String msg = String.format("Expected not failed for caller: %s.", caller);
+      assertFalse(msg, caller.failed);
     } finally {
       client.stop();
       server.stop();
@@ -468,7 +443,9 @@ public class TestAsyncIPC {
       server.start();
       final AsyncCaller caller = new AsyncCaller(client, addr, 10);
       caller.run();
-      caller.assertReturnValues();
+      caller.waitForReturnValues();
+      String msg = String.format("Expected not failed for caller: %s.", caller);
+      assertFalse(msg, caller.failed);
     } finally {
       client.stop();
       server.stop();
@@ -512,7 +489,10 @@ public class TestAsyncIPC {
       }
       for (int i = 0; i < callerCount; ++i) {
         callers[i].join();
-        callers[i].assertReturnValues();
+        callers[i].waitForReturnValues();
+        String msg = String.format("Expected not failed for caller-%d: %s.", i,
+            callers[i]);
+        assertFalse(msg, callers[i].failed);
       }
     } finally {
       client.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4450d47/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 6bfd71d..4fe0861 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,20 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 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.FsPermission;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
-import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.hadoop.ipc.Client;
 
+import com.google.common.util.concurrent.AbstractFuture;
+
 /****************************************************************
  * Implementation of the asynchronous distributed file system.
  * This instance of this class is the way end-user code interacts
@@ -48,8 +52,22 @@ public class AsyncDistributedFileSystem {
   }
 
   static <T> Future<T> getReturnValue() {
-    return new AsyncGetFuture<>(
-        ClientNamenodeProtocolTranslatorPB.getAsyncReturnValue());
+    final Callable<T> returnValueCallback = ClientNamenodeProtocolTranslatorPB
+        .getReturnValueCallback();
+    Future<T> returnFuture = new AbstractFuture<T>() {
+      private final AtomicBoolean called = new AtomicBoolean(false);
+      public T get() throws InterruptedException, ExecutionException {
+        if (called.compareAndSet(false, true)) {
+          try {
+            set(returnValueCallback.call());
+          } catch (Exception e) {
+            setException(e);
+          }
+        }
+        return super.get();
+      }
+    };
+    return returnFuture;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4450d47/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 939c1ac..faa925c 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
@@ -24,8 +24,7 @@ import java.util.EnumSet;
 import java.util.List;
 
 import com.google.common.collect.Lists;
-
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.Callable;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -199,7 +198,6 @@ import org.apache.hadoop.security.token.Token;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Message;
 import com.google.protobuf.ServiceException;
-import org.apache.hadoop.util.concurrent.AsyncGet;
 
 /**
  * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
@@ -211,8 +209,8 @@ import org.apache.hadoop.util.concurrent.AsyncGet;
 public class ClientNamenodeProtocolTranslatorPB implements
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
-  private static final ThreadLocal<AsyncGet<?, Exception>>
-      ASYNC_RETURN_VALUE = new ThreadLocal<>();
+  private static final ThreadLocal<Callable<?>>
+      RETURN_VALUE_CALLBACK = new ThreadLocal<>();
 
   static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
       GetServerDefaultsRequestProto.newBuilder().build();
@@ -248,8 +246,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @SuppressWarnings("unchecked")
   @Unstable
-  public static <T> AsyncGet<T, Exception> getAsyncReturnValue() {
-    return (AsyncGet<T, Exception>) ASYNC_RETURN_VALUE.get();
+  public static <T> Callable<T> getReturnValueCallback() {
+    return (Callable<T>) RETURN_VALUE_CALLBACK.get();
   }
 
   @Override
@@ -371,7 +369,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       if (Client.isAsynchronousMode()) {
         rpcProxy.setPermission(null, req);
-        setAsyncReturnValue();
+        setReturnValueCallback();
       } else {
         rpcProxy.setPermission(null, req);
       }
@@ -380,18 +378,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
-  private void setAsyncReturnValue() {
-    final AsyncGet<Message, Exception> asyncReturnMessage
-        = ProtobufRpcEngine.getAsyncReturnMessage();
-    final AsyncGet<Void, Exception> asyncGet
-        = new AsyncGet<Void, Exception>() {
+  private void setReturnValueCallback() {
+    final Callable<Message> returnMessageCallback = ProtobufRpcEngine
+        .getReturnMessageCallback();
+    Callable<Void> callBack = new Callable<Void>() {
       @Override
-      public Void get(long timeout, TimeUnit unit) throws Exception {
-        asyncReturnMessage.get(timeout, unit);
+      public Void call() throws Exception {
+        returnMessageCallback.call();
         return null;
       }
     };
-    ASYNC_RETURN_VALUE.set(asyncGet);
+    RETURN_VALUE_CALLBACK.set(callBack);
   }
 
   @Override
@@ -406,7 +403,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       if (Client.isAsynchronousMode()) {
         rpcProxy.setOwner(null, req.build());
-        setAsyncReturnValue();
+        setReturnValueCallback();
       } else {
         rpcProxy.setOwner(null, req.build());
       }
@@ -539,7 +536,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       if (Client.isAsynchronousMode()) {
         rpcProxy.rename2(null, req);
-        setAsyncReturnValue();
+        setReturnValueCallback();
       } else {
         rpcProxy.rename2(null, req);
       }


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


[35/47] hadoop git commit: Revert "Revert "HDFS-10390. Implement asynchronous setAcl/getAclStatus for DistributedFileSystem. Contributed by Xiaobing Zhou""

Posted by ae...@apache.org.
Revert "Revert "HDFS-10390. Implement asynchronous setAcl/getAclStatus for DistributedFileSystem.  Contributed by Xiaobing Zhou""

This reverts commit b82c74b9102ba95eae776501ed4484be9edd8c96.


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

Branch: refs/heads/HDFS-1312
Commit: b3d81f38da5d3d913e7b7ed498198c899c1e68b7
Parents: 574dcd3
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 6 16:31:30 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 6 16:31:30 2016 +0800

----------------------------------------------------------------------
 .../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, 411 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3d81f38/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 6bfd71d..29bac2a 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,12 +19,16 @@
 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;
@@ -83,6 +87,7 @@ 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);
@@ -111,6 +116,7 @@ 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);
@@ -142,6 +148,7 @@ 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);
@@ -152,4 +159,56 @@ 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/b3d81f38/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 0ae4d70..66ee42f 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,4 +2529,7 @@ public class DistributedFileSystem extends FileSystem {
     return statistics;
   }
 
+  DFSOpsCountStatistics getDFSOpsCountStatistics() {
+    return storageStatistics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3d81f38/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 939c1ac..2373da7 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,6 +72,7 @@ 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;
@@ -163,7 +164,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.EncryptionZonesProtos;
+import org.apache.hadoop.hdfs.protocol.proto.*;
 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;
@@ -1346,7 +1347,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec))
         .build();
     try {
-      rpcProxy.setAcl(null, req);
+      if (Client.isAsynchronousMode()) {
+        rpcProxy.setAcl(null, req);
+        setAsyncReturnValue();
+      } else {
+        rpcProxy.setAcl(null, req);
+      }
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1357,7 +1363,25 @@ public class ClientNamenodeProtocolTranslatorPB implements
     GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
+      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));
+      }
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3d81f38/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
new file mode 100644
index 0000000..67262dd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFS.java
@@ -0,0 +1,310 @@
+/**
+ * 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/b3d81f38/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 7539fbd..03c8151 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) {
-      checkPermissionDenied(e, src, user1);
+      TestAsyncDFS.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) {
-      checkPermissionDenied(e, src, user1);
+      TestAsyncDFS.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) {
-      checkPermissionDenied(e, src, user1);
+      TestAsyncDFS.checkPermissionDenied(e, src, user1);
       assertTrue("Permission denied messages must carry the name of the path",
           e.getMessage().contains(src.getName()));
     } finally {
@@ -551,13 +551,4 @@ 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/b3d81f38/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 002f7c0..216147a 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,17 +1637,23 @@ 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
    */
-  private static void assertAclFeature(Path pathToCheck,
-      boolean expectAclFeature) throws IOException {
-    AclFeature aclFeature = getAclFeature(pathToCheck, cluster);
+  public static void assertAclFeature(final MiniDFSCluster miniCluster,
+      Path pathToCheck, boolean expectAclFeature) throws IOException {
+    AclFeature aclFeature = getAclFeature(pathToCheck, miniCluster);
     if (expectAclFeature) {
       assertNotNull(aclFeature);
       // Intentionally capturing a reference to the entries, not using nested


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


[31/47] hadoop git commit: Revert "Revert "HDFS-10224. Implement asynchronous rename for DistributedFileSystem. Contributed by Xiaobing Zhou""

Posted by ae...@apache.org.
Revert "Revert "HDFS-10224. Implement asynchronous rename for DistributedFileSystem.  Contributed by Xiaobing Zhou""

This reverts commit 106234d873c60fa52cd0d812fb1cdc0c6b998a6d.


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

Branch: refs/heads/HDFS-1312
Commit: eded3d109e4c5225d8c5cd3c2d82e7ac93841263
Parents: 106234d
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 6 16:28:21 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 6 16:28:21 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileSystem.java   |   1 -
 .../main/java/org/apache/hadoop/ipc/Client.java |  11 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |  34 ++-
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     |   2 +-
 .../hadoop/hdfs/AsyncDistributedFileSystem.java | 110 ++++++++
 .../hadoop/hdfs/DistributedFileSystem.java      |  22 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  45 +++-
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 258 +++++++++++++++++++
 8 files changed, 463 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eded3d10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 0ecd8b7..9e13a7a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -1252,7 +1252,6 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Renames Path src to Path dst
    * <ul>
-   * <li
    * <li>Fails if src is a file and dst is a directory.
    * <li>Fails if src is a directory and dst is a file.
    * <li>Fails if the parent of dst does not exist or is a file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eded3d10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index f206861..d59aeb89 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -119,7 +119,8 @@ public class Client implements AutoCloseable {
 
   private static final ThreadLocal<Integer> callId = new ThreadLocal<Integer>();
   private static final ThreadLocal<Integer> retryCount = new ThreadLocal<Integer>();
-  private static final ThreadLocal<Future<?>> returnValue = new ThreadLocal<>();
+  private static final ThreadLocal<Future<?>>
+      RETURN_RPC_RESPONSE = new ThreadLocal<>();
   private static final ThreadLocal<Boolean> asynchronousMode =
       new ThreadLocal<Boolean>() {
         @Override
@@ -130,8 +131,8 @@ public class Client implements AutoCloseable {
 
   @SuppressWarnings("unchecked")
   @Unstable
-  public static <T> Future<T> getReturnValue() {
-    return (Future<T>) returnValue.get();
+  public static <T> Future<T> getReturnRpcResponse() {
+    return (Future<T>) RETURN_RPC_RESPONSE.get();
   }
 
   /** Set call id and retry count for the next call. */
@@ -1396,7 +1397,7 @@ public class Client implements AutoCloseable {
         }
       };
 
-      returnValue.set(returnFuture);
+      RETURN_RPC_RESPONSE.set(returnFuture);
       return null;
     } else {
       return getRpcResponse(call, connection);
@@ -1410,7 +1411,7 @@ public class Client implements AutoCloseable {
    *          synchronous mode.
    */
   @Unstable
-  static boolean isAsynchronousMode() {
+  public static boolean isAsynchronousMode() {
     return asynchronousMode.get();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eded3d10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 071e2e8..8fcdb78 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -26,7 +26,9 @@ import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.net.InetSocketAddress;
 import java.util.Map;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.net.SocketFactory;
@@ -35,6 +37,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataOutputOutputStream;
 import org.apache.hadoop.io.Writable;
@@ -67,7 +70,9 @@ import com.google.protobuf.TextFormat;
 @InterfaceStability.Evolving
 public class ProtobufRpcEngine implements RpcEngine {
   public static final Log LOG = LogFactory.getLog(ProtobufRpcEngine.class);
-  
+  private static final ThreadLocal<Callable<?>>
+      RETURN_MESSAGE_CALLBACK = new ThreadLocal<>();
+
   static { // Register the rpcRequest deserializer for WritableRpcEngine 
     org.apache.hadoop.ipc.Server.registerProtocolEngine(
         RPC.RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWrapper.class,
@@ -76,6 +81,12 @@ public class ProtobufRpcEngine implements RpcEngine {
 
   private static final ClientCache CLIENTS = new ClientCache();
 
+  @SuppressWarnings("unchecked")
+  @Unstable
+  public static <T> Callable<T> getReturnMessageCallback() {
+    return (Callable<T>) RETURN_MESSAGE_CALLBACK.get();
+  }
+
   public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
       InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
       SocketFactory factory, int rpcTimeout) throws IOException {
@@ -189,7 +200,7 @@ public class ProtobufRpcEngine implements RpcEngine {
      * the server.
      */
     @Override
-    public Object invoke(Object proxy, Method method, Object[] args)
+    public Object invoke(Object proxy, final Method method, Object[] args)
         throws ServiceException {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
@@ -251,6 +262,23 @@ public class ProtobufRpcEngine implements RpcEngine {
         LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
       }
       
+      if (Client.isAsynchronousMode()) {
+        final Future<RpcResponseWrapper> frrw = Client.getReturnRpcResponse();
+        Callable<Message> callback = new Callable<Message>() {
+          @Override
+          public Message call() throws Exception {
+            return getReturnMessage(method, frrw.get());
+          }
+        };
+        RETURN_MESSAGE_CALLBACK.set(callback);
+        return null;
+      } else {
+        return getReturnMessage(method, val);
+      }
+    }
+
+    private Message getReturnMessage(final Method method,
+        final RpcResponseWrapper rrw) throws ServiceException {
       Message prototype = null;
       try {
         prototype = getReturnProtoType(method);
@@ -260,7 +288,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       Message returnMessage;
       try {
         returnMessage = prototype.newBuilderForType()
-            .mergeFrom(val.theResponseRead).build();
+            .mergeFrom(rrw.theResponseRead).build();
 
         if (LOG.isTraceEnabled()) {
           LOG.trace(Thread.currentThread().getId() + ": Response <- " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eded3d10/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
index de4395e..6cf75c7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
@@ -84,7 +84,7 @@ public class TestAsyncIPC {
         try {
           final long param = TestIPC.RANDOM.nextLong();
           TestIPC.call(client, param, server, conf);
-          Future<LongWritable> returnFuture = Client.getReturnValue();
+          Future<LongWritable> returnFuture = Client.getReturnRpcResponse();
           returnFutures.put(i, returnFuture);
           expectedValues.put(i, param);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eded3d10/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
new file mode 100644
index 0000000..37899aa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
@@ -0,0 +1,110 @@
+/**
+ * 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 java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+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.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.ipc.Client;
+
+import com.google.common.util.concurrent.AbstractFuture;
+
+/****************************************************************
+ * Implementation of the asynchronous distributed file system.
+ * This instance of this class is the way end-user code interacts
+ * with a Hadoop DistributedFileSystem in an asynchronous manner.
+ *
+ *****************************************************************/
+@Unstable
+public class AsyncDistributedFileSystem {
+
+  private final DistributedFileSystem dfs;
+
+  AsyncDistributedFileSystem(final DistributedFileSystem dfs) {
+    this.dfs = dfs;
+  }
+
+  static <T> Future<T> getReturnValue() {
+    final Callable<T> returnValueCallback = ClientNamenodeProtocolTranslatorPB
+        .getReturnValueCallback();
+    Future<T> returnFuture = new AbstractFuture<T>() {
+      public T get() throws InterruptedException, ExecutionException {
+        try {
+          set(returnValueCallback.call());
+        } catch (Exception e) {
+          setException(e);
+        }
+        return super.get();
+      }
+    };
+    return returnFuture;
+  }
+
+  /**
+   * Renames Path src to Path dst
+   * <ul>
+   * <li>Fails if src is a file and dst is a directory.
+   * <li>Fails if src is a directory and dst is a file.
+   * <li>Fails if the parent of dst does not exist or is a file.
+   * </ul>
+   * <p>
+   * If OVERWRITE option is not passed as an argument, rename fails if the dst
+   * already exists.
+   * <p>
+   * If OVERWRITE option is passed as an argument, rename overwrites the dst if
+   * it is a file or an empty directory. Rename fails if dst is a non-empty
+   * directory.
+   * <p>
+   * Note that atomicity of rename is dependent on the file system
+   * implementation. Please refer to the file system documentation for details.
+   * This default implementation is non atomic.
+   *
+   * @param src
+   *          path to be renamed
+   * @param dst
+   *          new path after rename
+   * @throws IOException
+   *           on failure
+   * @return an instance of Future, #get of which is invoked to wait for
+   *         asynchronous call being finished.
+   */
+  public Future<Void> rename(Path src, Path dst,
+      final Options.Rename... options) throws IOException {
+    dfs.getFsStatistics().incrementWriteOps(1);
+
+    final Path absSrc = dfs.fixRelativePart(src);
+    final Path absDst = dfs.fixRelativePart(dst);
+
+    final boolean isAsync = Client.isAsynchronousMode();
+    Client.setAsynchronousMode(true);
+    try {
+      dfs.getClient().rename(dfs.getPathName(absSrc), dfs.getPathName(absDst),
+          options);
+      return getReturnValue();
+    } finally {
+      Client.setAsynchronousMode(isAsync);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eded3d10/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 5e54edd..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
@@ -31,6 +31,7 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
@@ -204,7 +205,7 @@ public class DistributedFileSystem extends FileSystem {
    * @return path component of {file}
    * @throws IllegalArgumentException if URI does not belong to this DFS
    */
-  private String getPathName(Path file) {
+  String getPathName(Path file) {
     checkPath(file);
     String result = file.toUri().getPath();
     if (!DFSUtilClient.isValidName(result)) {
@@ -2509,4 +2510,23 @@ public class DistributedFileSystem extends FileSystem {
     }
     return ret;
   }
+
+  private final AsyncDistributedFileSystem adfs =
+      new AsyncDistributedFileSystem(this);
+
+  /** @return an {@link AsyncDistributedFileSystem} object. */
+  @Unstable
+  public AsyncDistributedFileSystem getAsyncDistributedFileSystem() {
+    return adfs;
+  }
+
+  @Override
+  protected Path fixRelativePart(Path p) {
+    return super.fixRelativePart(p);
+  }
+
+  Statistics getFsStatistics() {
+    return statistics;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eded3d10/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 513a5e3..f4074b6 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
@@ -24,11 +24,14 @@ import java.util.EnumSet;
 import java.util.List;
 
 import com.google.common.collect.Lists;
+import java.util.concurrent.Callable;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -55,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
@@ -135,7 +139,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Recove
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
@@ -153,13 +156,15 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPer
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
+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.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
-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;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
@@ -177,8 +182,9 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.RPC;
@@ -190,12 +196,9 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenReque
 import org.apache.hadoop.security.token.Token;
 
 import com.google.protobuf.ByteString;
+import com.google.protobuf.Message;
 import com.google.protobuf.ServiceException;
 
-import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
-import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
-    .EncryptionZoneProto;
-
 /**
  * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
  * while translating from the parameter types used in ClientProtocol to the
@@ -206,6 +209,8 @@ import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
 public class ClientNamenodeProtocolTranslatorPB implements
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
+  private static final ThreadLocal<Callable<?>>
+      RETURN_VALUE_CALLBACK = new ThreadLocal<>();
 
   static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
       GetServerDefaultsRequestProto.newBuilder().build();
@@ -239,6 +244,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
     rpcProxy = proxy;
   }
 
+  @SuppressWarnings("unchecked")
+  @Unstable
+  public static <T> Callable<T> getReturnValueCallback() {
+    return (Callable<T>) RETURN_VALUE_CALLBACK.get();
+  }
+
   @Override
   public void close() {
     RPC.stopProxy(rpcProxy);
@@ -475,6 +486,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     RenameRequestProto req = RenameRequestProto.newBuilder()
         .setSrc(src)
         .setDst(dst).build();
+
     try {
       return rpcProxy.rename(null, req).getResult();
     } catch (ServiceException e) {
@@ -499,7 +511,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
         setDst(dst).setOverwriteDest(overwrite).
         build();
     try {
-      rpcProxy.rename2(null, req);
+      if (Client.isAsynchronousMode()) {
+        rpcProxy.rename2(null, req);
+
+        final Callable<Message> returnMessageCallback = ProtobufRpcEngine
+            .getReturnMessageCallback();
+        Callable<Void> callBack = new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            returnMessageCallback.call();
+            return null;
+          }
+        };
+        RETURN_VALUE_CALLBACK.set(callBack);
+      } else {
+        rpcProxy.rename2(null, req);
+      }
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eded3d10/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
new file mode 100644
index 0000000..9322e1a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
@@ -0,0 +1,258 @@
+/**
+ * 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.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestAsyncDFSRename {
+  final Path asyncRenameDir = new Path("/test/async_rename/");
+  public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
+  final private static Configuration CONF = new HdfsConfiguration();
+
+  final private static String GROUP1_NAME = "group1";
+  final private static String GROUP2_NAME = "group2";
+  final private static String USER1_NAME = "user1";
+  private static final UserGroupInformation USER1;
+
+  private MiniDFSCluster gCluster;
+
+  static {
+    // explicitly turn on permission checking
+    CONF.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+
+    // create fake mapping for the groups
+    Map<String, String[]> u2g_map = new HashMap<String, String[]>(1);
+    u2g_map.put(USER1_NAME, new String[] { GROUP1_NAME, GROUP2_NAME });
+    DFSTestUtil.updateConfWithFakeGroupMapping(CONF, u2g_map);
+
+    // Initiate all four users
+    USER1 = UserGroupInformation.createUserForTesting(USER1_NAME, new String[] {
+        GROUP1_NAME, GROUP2_NAME });
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    gCluster = new MiniDFSCluster.Builder(CONF).numDataNodes(3).build();
+    gCluster.waitActive();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (gCluster != null) {
+      gCluster.shutdown();
+      gCluster = null;
+    }
+  }
+
+  static int countLease(MiniDFSCluster cluster) {
+    return TestDFSRename.countLease(cluster);
+  }
+
+  void list(DistributedFileSystem dfs, String name) throws IOException {
+    FileSystem.LOG.info("\n\n" + name);
+    for (FileStatus s : dfs.listStatus(asyncRenameDir)) {
+      FileSystem.LOG.info("" + s.getPath());
+    }
+  }
+
+  static void createFile(DistributedFileSystem dfs, Path f) throws IOException {
+    DataOutputStream a_out = dfs.create(f);
+    a_out.writeBytes("something");
+    a_out.close();
+  }
+
+  /**
+   * Check the blocks of dst file are cleaned after rename with overwrite
+   * Restart NN to check the rename successfully
+   */
+  @Test
+  public void testAsyncRenameWithOverwrite() throws Exception {
+    final short replFactor = 2;
+    final long blockSize = 512;
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        replFactor).build();
+    cluster.waitActive();
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
+
+    try {
+
+      long fileLen = blockSize * 3;
+      String src = "/foo/src";
+      String dst = "/foo/dst";
+      String src2 = "/foo/src2";
+      String dst2 = "/foo/dst2";
+      Path srcPath = new Path(src);
+      Path dstPath = new Path(dst);
+      Path srcPath2 = new Path(src2);
+      Path dstPath2 = new Path(dst2);
+
+      DFSTestUtil.createFile(dfs, srcPath, fileLen, replFactor, 1);
+      DFSTestUtil.createFile(dfs, dstPath, fileLen, replFactor, 1);
+      DFSTestUtil.createFile(dfs, srcPath2, fileLen, replFactor, 1);
+      DFSTestUtil.createFile(dfs, dstPath2, fileLen, replFactor, 1);
+
+      LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(
+          cluster.getNameNode(), dst, 0, fileLen);
+      LocatedBlocks lbs2 = NameNodeAdapter.getBlockLocations(
+          cluster.getNameNode(), dst2, 0, fileLen);
+      BlockManager bm = NameNodeAdapter.getNamesystem(cluster.getNameNode())
+          .getBlockManager();
+      assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock()
+          .getLocalBlock()) != null);
+      assertTrue(bm.getStoredBlock(lbs2.getLocatedBlocks().get(0).getBlock()
+          .getLocalBlock()) != null);
+
+      Future<Void> retVal1 = adfs.rename(srcPath, dstPath, Rename.OVERWRITE);
+      Future<Void> retVal2 = adfs.rename(srcPath2, dstPath2, Rename.OVERWRITE);
+      retVal1.get();
+      retVal2.get();
+
+      assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock()
+          .getLocalBlock()) == null);
+      assertTrue(bm.getStoredBlock(lbs2.getLocatedBlocks().get(0).getBlock()
+          .getLocalBlock()) == null);
+
+      // Restart NN and check the rename successfully
+      cluster.restartNameNodes();
+      assertFalse(dfs.exists(srcPath));
+      assertTrue(dfs.exists(dstPath));
+      assertFalse(dfs.exists(srcPath2));
+      assertTrue(dfs.exists(dstPath2));
+    } finally {
+      if (dfs != null) {
+        dfs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testConcurrentAsyncRenameWithOverwrite() throws Exception {
+    final short replFactor = 2;
+    final long blockSize = 512;
+    final Path renameDir = new Path(
+        "/test/concurrent_reanme_with_overwrite_dir/");
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
+        .build();
+    cluster.waitActive();
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
+    int count = 1000;
+
+    try {
+      long fileLen = blockSize * 3;
+      assertTrue(dfs.mkdirs(renameDir));
+
+      Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
+
+      // concurrently invoking many rename
+      for (int i = 0; i < count; i++) {
+        Path src = new Path(renameDir, "src" + i);
+        Path dst = new Path(renameDir, "dst" + i);
+        DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
+        DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
+        Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+        returnFutures.put(i, returnFuture);
+      }
+
+      // wait for completing the calls
+      for (int i = 0; i < count; i++) {
+        returnFutures.get(i).get();
+      }
+
+      // Restart NN and check the rename successfully
+      cluster.restartNameNodes();
+
+      // very the src dir should not exist, dst should
+      for (int i = 0; i < count; i++) {
+        Path src = new Path(renameDir, "src" + i);
+        Path dst = new Path(renameDir, "dst" + i);
+        assertFalse(dfs.exists(src));
+        assertTrue(dfs.exists(dst));
+      }
+    } finally {
+      dfs.delete(renameDir, true);
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testAsyncRenameWithException() throws Exception {
+    FileSystem rootFs = FileSystem.get(CONF);
+    final Path renameDir = new Path("/test/async_rename_exception/");
+    final Path src = new Path(renameDir, "src");
+    final Path dst = new Path(renameDir, "dst");
+    rootFs.mkdirs(src);
+
+    AsyncDistributedFileSystem adfs = USER1
+        .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
+          @Override
+          public AsyncDistributedFileSystem run() throws Exception {
+            return gCluster.getFileSystem().getAsyncDistributedFileSystem();
+          }
+        });
+
+    try {
+      Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+      returnFuture.get();
+    } catch (ExecutionException e) {
+      checkPermissionDenied(e, src);
+    }
+  }
+
+  private void checkPermissionDenied(final Exception e, final Path dir) {
+    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(USER1_NAME));
+    assertTrue("Permission denied messages must carry the path parent", e
+        .getMessage().contains(dir.getParent().toUri().getPath()));
+  }
+}
\ No newline at end of file


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


[19/47] hadoop git commit: HADOOP-13155. Implement TokenRenewer to renew and cancel delegation tokens in KMS. Contributed by Xiao Chen.

Posted by ae...@apache.org.
HADOOP-13155. Implement TokenRenewer to renew and cancel delegation tokens in KMS. Contributed by Xiao Chen.


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

Branch: refs/heads/HDFS-1312
Commit: 713cb71820ad94a5436f35824d07aa12fcba5cc6
Parents: d82bc85
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 16:48:54 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 16:48:54 2016 -0700

----------------------------------------------------------------------
 .../KeyProviderDelegationTokenExtension.java    |  30 +++-
 .../crypto/key/kms/KMSClientProvider.java       | 158 +++++++++++++++++--
 .../key/kms/LoadBalancingKMSClientProvider.java |  21 +++
 .../java/org/apache/hadoop/util/KMSUtil.java    |  76 +++++++++
 ...rg.apache.hadoop.security.token.TokenRenewer |  14 ++
 .../key/kms/server/KMSAuthenticationFilter.java |   2 +-
 .../hadoop/crypto/key/kms/server/TestKMS.java   | 127 +++++++++++++--
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  38 ++---
 8 files changed, 410 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/713cb718/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java
index 2f237c6..9212cbc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java
@@ -34,7 +34,7 @@ public class KeyProviderDelegationTokenExtension extends
       new DefaultDelegationTokenExtension();
 
   /**
-   * DelegationTokenExtension is a type of Extension that exposes methods to 
+   * DelegationTokenExtension is a type of Extension that exposes methods
    * needed to work with Delegation Tokens.
    */  
   public interface DelegationTokenExtension extends 
@@ -49,8 +49,23 @@ public class KeyProviderDelegationTokenExtension extends
      * @return list of new delegation tokens
      * @throws IOException thrown if IOException if an IO error occurs.
      */
-    public Token<?>[] addDelegationTokens(final String renewer, 
+    Token<?>[] addDelegationTokens(final String renewer,
         Credentials credentials) throws IOException;
+
+    /**
+     * Renews the given token.
+     * @param token The token to be renewed.
+     * @return The token's lifetime after renewal, or 0 if it can't be renewed.
+     * @throws IOException
+     */
+    long renewDelegationToken(final Token<?> token) throws IOException;
+
+    /**
+     * Cancels the given token.
+     * @param token The token to be cancelled.
+     * @throws IOException
+     */
+    Void cancelDelegationToken(final Token<?> token) throws IOException;
   }
   
   /**
@@ -65,7 +80,16 @@ public class KeyProviderDelegationTokenExtension extends
         Credentials credentials) {
       return null;
     }
-    
+
+    @Override
+    public long renewDelegationToken(final Token<?> token) throws IOException {
+      return 0;
+    }
+
+    @Override
+    public Void cancelDelegationToken(final Token<?> token) throws IOException {
+      return null;
+    }
   }
 
   private KeyProviderDelegationTokenExtension(KeyProvider keyProvider,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/713cb718/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
index 32ef09c..f4103b4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
@@ -38,8 +38,11 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenRenewer;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
 import org.apache.hadoop.util.HttpExceptionUtils;
+import org.apache.hadoop.util.KMSUtil;
 import org.apache.http.client.utils.URIBuilder;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
@@ -94,7 +97,8 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
 
   private static final String ANONYMOUS_REQUESTS_DISALLOWED = "Anonymous requests are disallowed";
 
-  public static final String TOKEN_KIND = "kms-dt";
+  public static final String TOKEN_KIND_STR = "kms-dt";
+  public static final Text TOKEN_KIND = new Text(TOKEN_KIND_STR);
 
   public static final String SCHEME_NAME = "kms";
 
@@ -146,6 +150,54 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     }
   }
 
+  /**
+   * The KMS implementation of {@link TokenRenewer}.
+   */
+  public static class KMSTokenRenewer extends TokenRenewer {
+    private static final Logger LOG =
+        LoggerFactory.getLogger(KMSTokenRenewer.class);
+
+    @Override
+    public boolean handleKind(Text kind) {
+      return kind.equals(TOKEN_KIND);
+    }
+
+    @Override
+    public boolean isManaged(Token<?> token) throws IOException {
+      return true;
+    }
+
+    @Override
+    public long renew(Token<?> token, Configuration conf) throws IOException {
+      LOG.debug("Renewing delegation token {}", token);
+      KeyProvider keyProvider = KMSUtil.createKeyProvider(conf,
+          KeyProviderFactory.KEY_PROVIDER_PATH);
+      if (!(keyProvider instanceof
+          KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
+        LOG.warn("keyProvider {} cannot renew dt.", keyProvider == null ?
+            "null" : keyProvider.getClass());
+        return 0;
+      }
+      return ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
+          keyProvider).renewDelegationToken(token);
+    }
+
+    @Override
+    public void cancel(Token<?> token, Configuration conf) throws IOException {
+      LOG.debug("Canceling delegation token {}", token);
+      KeyProvider keyProvider = KMSUtil.createKeyProvider(conf,
+          KeyProviderFactory.KEY_PROVIDER_PATH);
+      if (!(keyProvider instanceof
+          KeyProviderDelegationTokenExtension.DelegationTokenExtension)) {
+        LOG.warn("keyProvider {} cannot cancel dt.", keyProvider == null ?
+            "null" : keyProvider.getClass());
+        return;
+      }
+      ((KeyProviderDelegationTokenExtension.DelegationTokenExtension)
+          keyProvider).cancelDelegationToken(token);
+    }
+  }
+
   public static class KMSEncryptedKeyVersion extends EncryptedKeyVersion {
     public KMSEncryptedKeyVersion(String keyName, String keyVersionName,
         byte[] iv, String encryptedVersionName, byte[] keyMaterial) {
@@ -854,6 +906,100 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
   }
 
   @Override
+  public long renewDelegationToken(final Token<?> dToken) throws IOException {
+    try {
+      final String doAsUser = getDoAsUser();
+      final DelegationTokenAuthenticatedURL.Token token =
+          generateDelegationToken(dToken);
+      final URL url = createURL(null, null, null, null);
+      LOG.debug("Renewing delegation token {} with url:{}, as:{}",
+          token, url, doAsUser);
+      final DelegationTokenAuthenticatedURL authUrl =
+          new DelegationTokenAuthenticatedURL(configurator);
+      return actualUgi.doAs(
+          new PrivilegedExceptionAction<Long>() {
+            @Override
+            public Long run() throws Exception {
+              return authUrl.renewDelegationToken(url, token, doAsUser);
+            }
+          }
+      );
+    } catch (Exception ex) {
+      if (ex instanceof IOException) {
+        throw (IOException) ex;
+      } else {
+        throw new IOException(ex);
+      }
+    }
+  }
+
+  @Override
+  public Void cancelDelegationToken(final Token<?> dToken) throws IOException {
+    try {
+      final String doAsUser = getDoAsUser();
+      final DelegationTokenAuthenticatedURL.Token token =
+          generateDelegationToken(dToken);
+      return actualUgi.doAs(
+          new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws Exception {
+              final URL url = createURL(null, null, null, null);
+              LOG.debug("Cancelling delegation token {} with url:{}, as:{}",
+                  dToken, url, doAsUser);
+              final DelegationTokenAuthenticatedURL authUrl =
+                  new DelegationTokenAuthenticatedURL(configurator);
+              authUrl.cancelDelegationToken(url, token, doAsUser);
+              return null;
+            }
+          }
+      );
+    } catch (Exception ex) {
+      if (ex instanceof IOException) {
+        throw (IOException) ex;
+      } else {
+        throw new IOException(ex);
+      }
+    }
+  }
+
+  /**
+   * Get the doAs user name.
+   *
+   * 'actualUGI' is the UGI of the user creating the client
+   * It is possible that the creator of the KMSClientProvier
+   * calls this method on behalf of a proxyUser (the doAsUser).
+   * In which case this call has to be made as the proxy user.
+   *
+   * @return the doAs user name.
+   * @throws IOException
+   */
+  private String getDoAsUser() throws IOException {
+    UserGroupInformation currentUgi = UserGroupInformation.getCurrentUser();
+    return (currentUgi.getAuthenticationMethod() ==
+        UserGroupInformation.AuthenticationMethod.PROXY)
+        ? currentUgi.getShortUserName() : null;
+  }
+
+  /**
+   * Generate a DelegationTokenAuthenticatedURL.Token from the given generic
+   * typed delegation token.
+   *
+   * @param dToken The delegation token.
+   * @return The DelegationTokenAuthenticatedURL.Token, with its delegation
+   *         token set to the delegation token passed in.
+   */
+  private DelegationTokenAuthenticatedURL.Token generateDelegationToken(
+      final Token<?> dToken) {
+    DelegationTokenAuthenticatedURL.Token token =
+        new DelegationTokenAuthenticatedURL.Token();
+    Token<AbstractDelegationTokenIdentifier> dt =
+        new Token<>(dToken.getIdentifier(), dToken.getPassword(),
+            dToken.getKind(), dToken.getService());
+    token.setDelegationToken(dt);
+    return token;
+  }
+
+  @Override
   public Token<?>[] addDelegationTokens(final String renewer,
       Credentials credentials) throws IOException {
     Token<?>[] tokens = null;
@@ -864,15 +1010,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       final DelegationTokenAuthenticatedURL authUrl =
           new DelegationTokenAuthenticatedURL(configurator);
       try {
-        // 'actualUGI' is the UGI of the user creating the client 
-        // It is possible that the creator of the KMSClientProvier
-        // calls this method on behalf of a proxyUser (the doAsUser).
-        // In which case this call has to be made as the proxy user.
-        UserGroupInformation currentUgi = UserGroupInformation.getCurrentUser();
-        final String doAsUser = (currentUgi.getAuthenticationMethod() ==
-            UserGroupInformation.AuthenticationMethod.PROXY)
-                                ? currentUgi.getShortUserName() : null;
-
+        final String doAsUser = getDoAsUser();
         token = actualUgi.doAs(new PrivilegedExceptionAction<Token<?>>() {
           @Override
           public Token<?> run() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/713cb718/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
index 83bc7b5..6a9bd62 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
@@ -134,6 +134,27 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
     }, nextIdx());
   }
 
+  @Override
+  public long renewDelegationToken(final Token<?> token) throws IOException {
+    return doOp(new ProviderCallable<Long>() {
+      @Override
+      public Long call(KMSClientProvider provider) throws IOException {
+        return provider.renewDelegationToken(token);
+      }
+    }, nextIdx());
+  }
+
+  @Override
+  public Void cancelDelegationToken(final Token<?> token) throws IOException {
+    return doOp(new ProviderCallable<Void>() {
+      @Override
+      public Void call(KMSClientProvider provider) throws IOException {
+        provider.cancelDelegationToken(token);
+        return null;
+      }
+    }, nextIdx());
+  }
+
   // This request is sent to all providers in the load-balancing group
   @Override
   public void warmUpEncryptedKeys(String... keyNames) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/713cb718/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
new file mode 100644
index 0000000..bd6b460
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/KMSUtil.java
@@ -0,0 +1,76 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Utils for KMS.
+ */
+@InterfaceAudience.Private
+public final class KMSUtil {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(KMSUtil.class);
+
+  private KMSUtil() { /* Hidden constructor */ }
+
+  /**
+   * Creates a new KeyProvider from the given Configuration
+   * and configuration key name.
+   *
+   * @param conf Configuration
+   * @param configKeyName The configuration key name
+   * @return new KeyProvider, or null if no provider was found.
+   * @throws IOException if the KeyProvider is improperly specified in
+   *                             the Configuration
+   */
+  public static KeyProvider createKeyProvider(final Configuration conf,
+      final String configKeyName) throws IOException {
+    LOG.debug("Creating key provider with config key {}", configKeyName);
+    final String providerUriStr = conf.getTrimmed(configKeyName, "");
+    // No provider set in conf
+    if (providerUriStr.isEmpty()) {
+      return null;
+    }
+    final URI providerUri;
+    try {
+      providerUri = new URI(providerUriStr);
+    } catch (URISyntaxException e) {
+      throw new IOException(e);
+    }
+    KeyProvider keyProvider = KeyProviderFactory.get(providerUri, conf);
+    if (keyProvider == null) {
+      throw new IOException("Could not instantiate KeyProvider from " +
+          configKeyName + " setting of '" + providerUriStr + "'");
+    }
+    if (keyProvider.isTransient()) {
+      throw new IOException("KeyProvider " + keyProvider.toString()
+          + " was found but it is a transient provider.");
+    }
+    return keyProvider;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/713cb718/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer b/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
new file mode 100644
index 0000000..56320fb
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
@@ -0,0 +1,14 @@
+#
+#   Licensed 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.
+#
+org.apache.hadoop.crypto.key.kms.KMSClientProvider$KMSTokenRenewer
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/713cb718/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
index 79652f3..60f1918 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
@@ -72,7 +72,7 @@ public class KMSAuthenticationFilter
           KerberosDelegationTokenAuthenticationHandler.class.getName());
     }
     props.setProperty(DelegationTokenAuthenticationHandler.TOKEN_KIND,
-        KMSClientProvider.TOKEN_KIND);
+        KMSClientProvider.TOKEN_KIND_STR);
     return props;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/713cb718/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
index a452a80..db34aa9 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.crypto.key.kms.server;
 
 import org.apache.curator.test.TestingServer;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.kms.server.KeyAuthorizationKeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.crypto.key.KeyProvider.Options;
@@ -31,11 +31,14 @@ import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.Authenticator;
+import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.token.Token;
@@ -45,11 +48,12 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.security.auth.kerberos.KerberosPrincipal;
 import javax.security.auth.login.AppConfigurationEntry;
 
 import java.io.File;
@@ -72,9 +76,17 @@ import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
 public class TestKMS {
   private static final Logger LOG = LoggerFactory.getLogger(TestKMS.class);
 
+  @Rule
+  public final Timeout testTimeout = new Timeout(180000);
+
   @Before
   public void cleanUp() {
     // resetting kerberos security
@@ -649,20 +661,6 @@ public class TestKMS {
         Assert.assertEquals("d", meta.getDescription());
         Assert.assertEquals(attributes, meta.getAttributes());
 
-        // test delegation token retrieval
-        KeyProviderDelegationTokenExtension kpdte =
-            KeyProviderDelegationTokenExtension.
-                createKeyProviderDelegationTokenExtension(kp);
-        Credentials credentials = new Credentials();
-        kpdte.addDelegationTokens("foo", credentials);
-        Assert.assertEquals(1, credentials.getAllTokens().size());
-        InetSocketAddress kmsAddr = new InetSocketAddress(getKMSUrl().getHost(),
-            getKMSUrl().getPort());
-
-        Assert.assertEquals(new Text("kms-dt"), credentials.getToken(
-            SecurityUtil.buildTokenService(kmsAddr)).getKind());
-
-
         // test rollover draining
         KeyProviderCryptoExtension kpce = KeyProviderCryptoExtension.
             createKeyProviderCryptoExtension(kp);
@@ -1746,6 +1744,101 @@ public class TestKMS {
   }
 
   @Test
+  public void testDelegationTokensOpsSimple() throws Exception {
+    final Configuration conf = new Configuration();
+    final Authenticator mock = mock(PseudoAuthenticator.class);
+    testDelegationTokensOps(conf, mock);
+  }
+
+  @Test
+  public void testDelegationTokensOpsKerberized() throws Exception {
+    final Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    final Authenticator mock = mock(KerberosAuthenticator.class);
+    testDelegationTokensOps(conf, mock);
+  }
+
+  private void testDelegationTokensOps(Configuration conf,
+      final Authenticator mockAuthenticator) throws Exception {
+    UserGroupInformation.setConfiguration(conf);
+    File confDir = getTestDir();
+    conf = createBaseKMSConf(confDir);
+    writeConf(confDir, conf);
+    doNothing().when(mockAuthenticator).authenticate(any(URL.class),
+        any(AuthenticatedURL.Token.class));
+
+    runServer(null, null, confDir, new KMSCallable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        Configuration conf = new Configuration();
+        URI uri = createKMSUri(getKMSUrl());
+        KeyProvider kp = createProvider(uri, conf);
+        conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
+            createKMSUri(getKMSUrl()).toString());
+
+        // test delegation token retrieval
+        KeyProviderDelegationTokenExtension kpdte =
+            KeyProviderDelegationTokenExtension.
+                createKeyProviderDelegationTokenExtension(kp);
+        Credentials credentials = new Credentials();
+        final Token<?>[] tokens = kpdte.addDelegationTokens(
+            UserGroupInformation.getCurrentUser().getUserName(), credentials);
+        Assert.assertEquals(1, credentials.getAllTokens().size());
+        InetSocketAddress kmsAddr = new InetSocketAddress(getKMSUrl().getHost(),
+            getKMSUrl().getPort());
+        Assert.assertEquals(KMSClientProvider.TOKEN_KIND,
+            credentials.getToken(SecurityUtil.buildTokenService(kmsAddr)).
+                getKind());
+
+        // After this point, we're supposed to use the delegation token to auth.
+        doThrow(new IOException("Authenticator should not fall back"))
+            .when(mockAuthenticator).authenticate(any(URL.class),
+            any(AuthenticatedURL.Token.class));
+
+        // test delegation token renewal
+        boolean renewed = false;
+        for (Token<?> token : tokens) {
+          if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) {
+            LOG.info("Skipping token {}", token);
+            continue;
+          }
+          LOG.info("Got dt for " + uri + "; " + token);
+          long tokenLife = token.renew(conf);
+          LOG.info("Renewed token of kind {}, new lifetime:{}",
+              token.getKind(), tokenLife);
+          Thread.sleep(100);
+          long newTokenLife = token.renew(conf);
+          LOG.info("Renewed token of kind {}, new lifetime:{}",
+              token.getKind(), newTokenLife);
+          Assert.assertTrue(newTokenLife > tokenLife);
+          renewed = true;
+        }
+        Assert.assertTrue(renewed);
+
+        // test delegation token cancellation
+        for (Token<?> token : tokens) {
+          if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) {
+            LOG.info("Skipping token {}", token);
+            continue;
+          }
+          LOG.info("Got dt for " + uri + "; " + token);
+          token.cancel(conf);
+          LOG.info("Cancelled token of kind {}", token.getKind());
+          doNothing().when(mockAuthenticator).
+              authenticate(any(URL.class), any(AuthenticatedURL.Token.class));
+          try {
+            token.renew(conf);
+            Assert.fail("should not be able to renew a canceled token");
+          } catch (Exception e) {
+            LOG.info("Expected exception when trying to renew token", e);
+          }
+        }
+        return null;
+      }
+    });
+  }
+
+  @Test
   public void testKMSWithZKSigner() throws Exception {
     doKMSWithZK(true, false);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/713cb718/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index b9f4dce2..7ce93fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -23,7 +23,6 @@ import com.google.common.primitives.SignedBytes;
 import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
@@ -53,6 +52,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.KMSUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -70,7 +70,6 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.nio.channels.SocketChannel;
 import java.text.SimpleDateFormat;
 import java.util.Collection;
@@ -514,6 +513,17 @@ public class DFSUtilClient {
     return new ReconfigurationProtocolTranslatorPB(addr, ticket, conf, factory);
   }
 
+  private static String keyProviderUriKeyName =
+      HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI;
+
+  /**
+   * Set the key provider uri configuration key name for creating key providers.
+   * @param keyName The configuration key name.
+   */
+  public static void setKeyProviderUriKeyName(final String keyName) {
+    keyProviderUriKeyName = keyName;
+  }
+
   /**
    * Creates a new KeyProvider from the given Configuration.
    *
@@ -524,29 +534,7 @@ public class DFSUtilClient {
    */
   public static KeyProvider createKeyProvider(
       final Configuration conf) throws IOException {
-    final String providerUriStr =
-        conf.getTrimmed(HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
-    // No provider set in conf
-    if (providerUriStr.isEmpty()) {
-      return null;
-    }
-    final URI providerUri;
-    try {
-      providerUri = new URI(providerUriStr);
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    }
-    KeyProvider keyProvider = KeyProviderFactory.get(providerUri, conf);
-    if (keyProvider == null) {
-      throw new IOException("Could not instantiate KeyProvider from " +
-          HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI + " setting of '"
-          + providerUriStr + "'");
-    }
-    if (keyProvider.isTransient()) {
-      throw new IOException("KeyProvider " + keyProvider.toString()
-          + " was found but it is a transient provider.");
-    }
-    return keyProvider;
+    return KMSUtil.createKeyProvider(conf, keyProviderUriKeyName);
   }
 
   public static Peer peerFromSocket(Socket socket)


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


[05/47] hadoop git commit: HADOOP-13232. Typo in exception in ValueQueue.java. Contributed by Jiayi Zhou.

Posted by ae...@apache.org.
HADOOP-13232. Typo in exception in ValueQueue.java. Contributed by Jiayi Zhou.


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

Branch: refs/heads/HDFS-1312
Commit: 99675e00df65e84dcc02082b795f1f93c90b794e
Parents: 69555fc
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Jun 2 19:14:53 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Jun 2 19:14:53 2016 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99675e00/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
index f38a6b3..4aeeeef 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
@@ -307,7 +307,7 @@ public class ValueQueue <E> {
         ekvs.add(val);
       }
     } catch (Exception e) {
-      throw new IOException("Exeption while contacting value generator ", e);
+      throw new IOException("Exception while contacting value generator ", e);
     }
     return ekvs;
   }


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


[33/47] hadoop git commit: Revert "Revert "HDFS-10346. Implement asynchronous setPermission/setOwner for DistributedFileSystem. Contributed by Xiaobing Zhou""

Posted by ae...@apache.org.
Revert "Revert "HDFS-10346. Implement asynchronous setPermission/setOwner for DistributedFileSystem.  Contributed by  Xiaobing Zhou""

This reverts commit f23d5dfc60a017187ae57f3667ac0e688877c2dd.


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

Branch: refs/heads/HDFS-1312
Commit: cba9a0188970cb33dcb95e9c49168ac4a83446d9
Parents: aa20fa1
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 6 16:29:38 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 6 16:29:38 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |  59 ++++
 .../ClientNamenodeProtocolTranslatorPB.java     |  39 ++-
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 267 +++++++++++++++++--
 .../apache/hadoop/hdfs/TestDFSPermission.java   |  29 +-
 4 files changed, 351 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cba9a018/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 356ae3f..4fe0861 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
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 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.FsPermission;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
 import org.apache.hadoop.ipc.Client;
 
@@ -37,6 +38,9 @@ import com.google.common.util.concurrent.AbstractFuture;
  * This instance of this class is the way end-user code interacts
  * with a Hadoop DistributedFileSystem in an asynchronous manner.
  *
+ * This class is unstable, so no guarantee is provided as to reliability,
+ * stability or compatibility across any level of release granularity.
+ *
  *****************************************************************/
 @Unstable
 public class AsyncDistributedFileSystem {
@@ -111,4 +115,59 @@ public class AsyncDistributedFileSystem {
       Client.setAsynchronousMode(isAsync);
     }
   }
+
+  /**
+   * Set permission of a path.
+   *
+   * @param p
+   *          the path the permission is set to
+   * @param permission
+   *          the permission that is set to a path.
+   * @return an instance of Future, #get of which is invoked to wait for
+   *         asynchronous call being finished.
+   */
+  public Future<Void> setPermission(Path p, final FsPermission permission)
+      throws IOException {
+    dfs.getFsStatistics().incrementWriteOps(1);
+    final Path absPath = dfs.fixRelativePart(p);
+    final boolean isAsync = Client.isAsynchronousMode();
+    Client.setAsynchronousMode(true);
+    try {
+      dfs.getClient().setPermission(dfs.getPathName(absPath), permission);
+      return getReturnValue();
+    } finally {
+      Client.setAsynchronousMode(isAsync);
+    }
+  }
+
+  /**
+   * Set owner of a path (i.e. a file or a directory). The parameters username
+   * and groupname cannot both be null.
+   *
+   * @param p
+   *          The path
+   * @param username
+   *          If it is null, the original username remains unchanged.
+   * @param groupname
+   *          If it is null, the original groupname remains unchanged.
+   * @return an instance of Future, #get of which is invoked to wait for
+   *         asynchronous call being finished.
+   */
+  public Future<Void> setOwner(Path p, String username, String groupname)
+      throws IOException {
+    if (username == null && groupname == null) {
+      throw new IOException("username == null && groupname == null");
+    }
+
+    dfs.getFsStatistics().incrementWriteOps(1);
+    final Path absPath = dfs.fixRelativePart(p);
+    final boolean isAsync = Client.isAsynchronousMode();
+    Client.setAsynchronousMode(true);
+    try {
+      dfs.getClient().setOwner(dfs.getPathName(absPath), username, groupname);
+      return getReturnValue();
+    } finally {
+      Client.setAsynchronousMode(isAsync);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cba9a018/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 f4074b6..faa925c 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
@@ -367,12 +367,30 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setPermission(PBHelperClient.convert(permission))
         .build();
     try {
-      rpcProxy.setPermission(null, req);
+      if (Client.isAsynchronousMode()) {
+        rpcProxy.setPermission(null, req);
+        setReturnValueCallback();
+      } else {
+        rpcProxy.setPermission(null, req);
+      }
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
+  private void setReturnValueCallback() {
+    final Callable<Message> returnMessageCallback = ProtobufRpcEngine
+        .getReturnMessageCallback();
+    Callable<Void> callBack = new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        returnMessageCallback.call();
+        return null;
+      }
+    };
+    RETURN_VALUE_CALLBACK.set(callBack);
+  }
+
   @Override
   public void setOwner(String src, String username, String groupname)
       throws IOException {
@@ -383,7 +401,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
     if (groupname != null)
       req.setGroupname(groupname);
     try {
-      rpcProxy.setOwner(null, req.build());
+      if (Client.isAsynchronousMode()) {
+        rpcProxy.setOwner(null, req.build());
+        setReturnValueCallback();
+      } else {
+        rpcProxy.setOwner(null, req.build());
+      }
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -513,17 +536,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       if (Client.isAsynchronousMode()) {
         rpcProxy.rename2(null, req);
-
-        final Callable<Message> returnMessageCallback = ProtobufRpcEngine
-            .getReturnMessageCallback();
-        Callable<Void> callBack = new Callable<Void>() {
-          @Override
-          public Void call() throws Exception {
-            returnMessageCallback.call();
-            return null;
-          }
-        };
-        RETURN_VALUE_CALLBACK.set(callBack);
+        setReturnValueCallback();
       } else {
         rpcProxy.rename2(null, req);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cba9a018/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 d129299..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
@@ -22,8 +22,11 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
@@ -31,18 +34,30 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 public class TestAsyncDFSRename {
   public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
+  private final long seed = Time.now();
+  private final Random r = new Random(seed);
+  private final PermissionGenerator permGenerator = new PermissionGenerator(r);
+  private final short replFactor = 2;
+  private final long blockSize = 512;
+  private long fileLen = blockSize * 3;
 
   /**
    * Check the blocks of dst file are cleaned after rename with overwrite
@@ -50,8 +65,6 @@ public class TestAsyncDFSRename {
    */
   @Test(timeout = 60000)
   public void testAsyncRenameWithOverwrite() throws Exception {
-    final short replFactor = 2;
-    final long blockSize = 512;
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
         replFactor).build();
@@ -60,8 +73,6 @@ public class TestAsyncDFSRename {
     AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
 
     try {
-
-      long fileLen = blockSize * 3;
       String src = "/foo/src";
       String dst = "/foo/dst";
       String src2 = "/foo/src2";
@@ -115,8 +126,6 @@ public class TestAsyncDFSRename {
 
   @Test(timeout = 60000)
   public void testCallGetReturnValueMultipleTimes() throws Exception {
-    final short replFactor = 2;
-    final long blockSize = 512;
     final Path renameDir = new Path(
         "/test/testCallGetReturnValueMultipleTimes/");
     final Configuration conf = new HdfsConfiguration();
@@ -127,7 +136,6 @@ public class TestAsyncDFSRename {
     final DistributedFileSystem dfs = cluster.getFileSystem();
     final AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
     final int count = 100;
-    long fileLen = blockSize * 3;
     final Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
 
     assertTrue(dfs.mkdirs(renameDir));
@@ -178,15 +186,15 @@ public class TestAsyncDFSRename {
     }
   }
 
-  @Test(timeout = 120000)
-  public void testAggressiveConcurrentAsyncRenameWithOverwrite()
+  @Test
+  public void testConservativeConcurrentAsyncRenameWithOverwrite()
       throws Exception {
     internalTestConcurrentAsyncRenameWithOverwrite(100,
         "testAggressiveConcurrentAsyncRenameWithOverwrite");
   }
 
   @Test(timeout = 60000)
-  public void testConservativeConcurrentAsyncRenameWithOverwrite()
+  public void testAggressiveConcurrentAsyncRenameWithOverwrite()
       throws Exception {
     internalTestConcurrentAsyncRenameWithOverwrite(10000,
         "testConservativeConcurrentAsyncRenameWithOverwrite");
@@ -194,8 +202,6 @@ public class TestAsyncDFSRename {
 
   private void internalTestConcurrentAsyncRenameWithOverwrite(
       final int asyncCallLimit, final String basePath) throws Exception {
-    final short replFactor = 2;
-    final long blockSize = 512;
     final Path renameDir = new Path(String.format("/test/%s/", basePath));
     Configuration conf = new HdfsConfiguration();
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
@@ -206,7 +212,6 @@ public class TestAsyncDFSRename {
     DistributedFileSystem dfs = cluster.getFileSystem();
     AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
     int count = 1000;
-    long fileLen = blockSize * 3;
     int start = 0, end = 0;
     Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
 
@@ -274,8 +279,206 @@ public class TestAsyncDFSRename {
     }
   }
 
+  @Test
+  public void testConservativeConcurrentAsyncAPI() throws Exception {
+    internalTestConcurrentAsyncAPI(100, "testConservativeConcurrentAsyncAPI");
+  }
+
+  @Test(timeout = 60000)
+  public void testAggressiveConcurrentAsyncAPI() throws Exception {
+    internalTestConcurrentAsyncAPI(10000, "testAggressiveConcurrentAsyncAPI");
+  }
+
+  private void internalTestConcurrentAsyncAPI(final int asyncCallLimit,
+      final String basePath) throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    String group1 = "group1";
+    String group2 = "group2";
+    String user1 = "user1";
+    int count = 500;
+
+    // explicitly turn on permission checking
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+    // set the limit of max async calls
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
+        asyncCallLimit);
+
+    // 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);
+
+    // start mini cluster
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3).build();
+    cluster.waitActive();
+    AsyncDistributedFileSystem adfs = cluster.getFileSystem()
+        .getAsyncDistributedFileSystem();
+
+    // prepare for test
+    FileSystem rootFs = FileSystem.get(conf);
+    final Path parent = new Path(String.format("/test/%s/", basePath));
+    final Path[] srcs = new Path[count];
+    final Path[] dsts = new Path[count];
+    short[] permissions = new short[count];
+    for (int i = 0; i < count; i++) {
+      srcs[i] = new Path(parent, "src" + i);
+      dsts[i] = new Path(parent, "dst" + i);
+      DFSTestUtil.createFile(rootFs, srcs[i], fileLen, replFactor, 1);
+      DFSTestUtil.createFile(rootFs, dsts[i], fileLen, replFactor, 1);
+      assertTrue(rootFs.exists(srcs[i]));
+      assertTrue(rootFs.getFileStatus(srcs[i]).isFile());
+      assertTrue(rootFs.exists(dsts[i]));
+      assertTrue(rootFs.getFileStatus(dsts[i]).isFile());
+      permissions[i] = permGenerator.next();
+    }
+
+    Map<Integer, Future<Void>> renameRetFutures =
+        new HashMap<Integer, Future<Void>>();
+    Map<Integer, Future<Void>> permRetFutures =
+        new HashMap<Integer, Future<Void>>();
+    Map<Integer, Future<Void>> ownerRetFutures =
+        new HashMap<Integer, Future<Void>>();
+    int start = 0, end = 0;
+    // test rename
+    for (int i = 0; i < count; i++) {
+      for (;;) {
+        try {
+          Future<Void> returnFuture = adfs.rename(srcs[i], dsts[i],
+              Rename.OVERWRITE);
+          renameRetFutures.put(i, returnFuture);
+          break;
+        } catch (AsyncCallLimitExceededException e) {
+          start = end;
+          end = i;
+          waitForReturnValues(renameRetFutures, start, end);
+        }
+      }
+    }
+
+    // wait for completing the calls
+    for (int i = start; i < count; i++) {
+      renameRetFutures.get(i).get();
+    }
+
+    // Restart NN and check the rename successfully
+    cluster.restartNameNodes();
+
+    // very the src should not exist, dst should
+    for (int i = 0; i < count; i++) {
+      assertFalse(rootFs.exists(srcs[i]));
+      assertTrue(rootFs.exists(dsts[i]));
+    }
+
+    // test permissions
+    try {
+      for (int i = 0; i < count; i++) {
+        for (;;) {
+          try {
+            Future<Void> retFuture = adfs.setPermission(dsts[i],
+                new FsPermission(permissions[i]));
+            permRetFutures.put(i, retFuture);
+            break;
+          } catch (AsyncCallLimitExceededException e) {
+            start = end;
+            end = i;
+            waitForReturnValues(permRetFutures, start, end);
+          }
+        }
+      }
+      // wait for completing the calls
+      for (int i = start; i < count; i++) {
+        permRetFutures.get(i).get();
+      }
+
+      // Restart NN and check permission then
+      cluster.restartNameNodes();
+
+      // verify the permission
+      for (int i = 0; i < count; i++) {
+        assertTrue(rootFs.exists(dsts[i]));
+        FsPermission fsPerm = new FsPermission(permissions[i]);
+        checkAccessPermissions(rootFs.getFileStatus(dsts[i]),
+            fsPerm.getUserAction());
+      }
+
+      // test setOwner
+      start = 0;
+      end = 0;
+      for (int i = 0; i < count; i++) {
+        for (;;) {
+          try {
+            Future<Void> retFuture = adfs.setOwner(dsts[i], "user1",
+                "group2");
+            ownerRetFutures.put(i, retFuture);
+            break;
+          } catch (AsyncCallLimitExceededException e) {
+            start = end;
+            end = i;
+            waitForReturnValues(ownerRetFutures, start, end);
+          }
+        }
+      }
+      // wait for completing the calls
+      for (int i = start; i < count; i++) {
+        ownerRetFutures.get(i).get();
+      }
+
+      // Restart NN and check owner then
+      cluster.restartNameNodes();
+
+      // verify the owner
+      for (int i = 0; i < count; i++) {
+        assertTrue(rootFs.exists(dsts[i]));
+        assertTrue(
+            "user1".equals(rootFs.getFileStatus(dsts[i]).getOwner()));
+        assertTrue(
+            "group2".equals(rootFs.getFileStatus(dsts[i]).getGroup()));
+      }
+    } catch (AccessControlException ace) {
+      throw ace;
+    } finally {
+      if (rootFs != null) {
+        rootFs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  static void checkAccessPermissions(FileStatus stat, FsAction mode)
+      throws IOException {
+    checkAccessPermissions(UserGroupInformation.getCurrentUser(), stat, mode);
+  }
+
+  static void checkAccessPermissions(final UserGroupInformation ugi,
+      FileStatus stat, FsAction mode) throws IOException {
+    FsPermission perm = stat.getPermission();
+    String user = ugi.getShortUserName();
+    List<String> groups = Arrays.asList(ugi.getGroupNames());
+
+    if (user.equals(stat.getOwner())) {
+      if (perm.getUserAction().implies(mode)) {
+        return;
+      }
+    } else if (groups.contains(stat.getGroup())) {
+      if (perm.getGroupAction().implies(mode)) {
+        return;
+      }
+    } else {
+      if (perm.getOtherAction().implies(mode)) {
+        return;
+      }
+    }
+    throw new AccessControlException(String.format(
+        "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat
+            .getPath(), stat.getOwner(), stat.getGroup(),
+        stat.isDirectory() ? "d" : "-", perm));
+  }
+
   @Test(timeout = 60000)
-  public void testAsyncRenameWithException() throws Exception {
+  public void testAsyncAPIWithException() throws Exception {
     Configuration conf = new HdfsConfiguration();
     String group1 = "group1";
     String group2 = "group2";
@@ -286,9 +489,9 @@ public class TestAsyncDFSRename {
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
 
     // create fake mapping for the groups
-    Map<String, String[]> u2g_map = new HashMap<String, String[]>(1);
-    u2g_map.put(user1, new String[] { group1, group2 });
-    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
+    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[] {
@@ -299,7 +502,7 @@ public class TestAsyncDFSRename {
     cluster.waitActive();
 
     FileSystem rootFs = FileSystem.get(conf);
-    final Path renameDir = new Path("/test/async_rename_exception/");
+    final Path renameDir = new Path("/test/async_api_exception/");
     final Path src = new Path(renameDir, "src");
     final Path dst = new Path(renameDir, "dst");
     rootFs.mkdirs(src);
@@ -312,11 +515,33 @@ public class TestAsyncDFSRename {
           }
         });
 
+    Future<Void> retFuture;
+    try {
+      retFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+      retFuture.get();
+    } catch (ExecutionException e) {
+      checkPermissionDenied(e, src, user1);
+      assertTrue("Permission denied messages must carry the path parent", e
+          .getMessage().contains(src.getParent().toUri().getPath()));
+    }
+
+    FsPermission fsPerm = new FsPermission(permGenerator.next());
+    try {
+      retFuture = adfs.setPermission(src, fsPerm);
+      retFuture.get();
+    } catch (ExecutionException e) {
+      checkPermissionDenied(e, src, user1);
+      assertTrue("Permission denied messages must carry the name of the path",
+          e.getMessage().contains(src.getName()));
+    }
+
     try {
-      Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-      returnFuture.get();
+      retFuture = adfs.setOwner(src, "user1", "group2");
+      retFuture.get();
     } catch (ExecutionException e) {
       checkPermissionDenied(e, src, user1);
+      assertTrue("Permission denied messages must carry the name of the path",
+          e.getMessage().contains(src.getName()));
     } finally {
       if (rootFs != null) {
         rootFs.close();
@@ -334,7 +559,5 @@ public class TestAsyncDFSRename {
         e.getMessage().contains("AccessControlException"));
     assertTrue("Permission denied messages must carry the username", e
         .getMessage().contains(user));
-    assertTrue("Permission denied messages must carry the path parent", e
-        .getMessage().contains(dir.getParent().toUri().getPath()));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cba9a018/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
index aa204cd..66a0380 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
@@ -196,22 +196,35 @@ public class TestDFSPermission {
     return fs.getFileStatus(path).getPermission().toShort();
   }
 
-  /* create a file/directory with the default umask and permission */
   private void create(OpType op, Path name) throws IOException {
-    create(op, name, DEFAULT_UMASK, new FsPermission(DEFAULT_PERMISSION));
+    create(fs, conf, op, name);
+  }
+
+  /* create a file/directory with the default umask and permission */
+  static void create(final FileSystem fs, final Configuration fsConf,
+      OpType op, Path name) throws IOException {
+    create(fs, fsConf, op, name, DEFAULT_UMASK, new FsPermission(
+        DEFAULT_PERMISSION));
+  }
+
+  private void create(OpType op, Path name, short umask,
+      FsPermission permission)
+      throws IOException {
+    create(fs, conf, op, name, umask, permission);
   }
 
   /* create a file/directory with the given umask and permission */
-  private void create(OpType op, Path name, short umask, 
-      FsPermission permission) throws IOException {
+  static void create(final FileSystem fs, final Configuration fsConf,
+      OpType op, Path name, short umask, FsPermission permission)
+      throws IOException {
     // set umask in configuration, converting to padded octal
-    conf.set(FsPermission.UMASK_LABEL, String.format("%1$03o", umask));
+    fsConf.set(FsPermission.UMASK_LABEL, String.format("%1$03o", umask));
 
     // create the file/directory
     switch (op) {
     case CREATE:
       FSDataOutputStream out = fs.create(name, permission, true, 
-          conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+          fsConf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
           fs.getDefaultReplication(name), fs.getDefaultBlockSize(name), null);
       out.close();
       break;
@@ -359,7 +372,7 @@ public class TestDFSPermission {
   final static private String DIR_NAME = "dir";
   final static private String FILE_DIR_NAME = "filedir";
 
-  private enum OpType {CREATE, MKDIRS, OPEN, SET_REPLICATION,
+  enum OpType {CREATE, MKDIRS, OPEN, SET_REPLICATION,
     GET_FILEINFO, IS_DIR, EXISTS, GET_CONTENT_LENGTH, LIST, RENAME, DELETE
   };
 
@@ -615,7 +628,7 @@ public class TestDFSPermission {
   /* A random permission generator that guarantees that each permission
    * value is generated only once.
    */
-  static private class PermissionGenerator {
+  static class PermissionGenerator {
     private final Random r;
     private final short[] permissions = new short[MAX_PERMISSION + 1];
     private int numLeft = MAX_PERMISSION + 1;


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


[24/47] hadoop git commit: Revert "HDFS-10430. Reuse FileSystem#access in TestAsyncDFS. Contributed by Xiaobing Zhou."

Posted by ae...@apache.org.
Revert "HDFS-10430. Reuse FileSystem#access in TestAsyncDFS. Contributed by Xiaobing Zhou."

This reverts commit 21890c4239b6a82fd6aab3454ce3922226efe7b5.


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

Branch: refs/heads/HDFS-1312
Commit: 8cf47d8589badfc07ef4bca3328a420c7c68abbd
Parents: 5360da8
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 18:09:12 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 18:09:12 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    | 36 +++++++++++++++++++-
 1 file changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf47d85/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
index c7615a9..ddcf492 100644
--- 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
@@ -34,6 +34,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -45,16 +46,19 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
 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.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
@@ -441,7 +445,7 @@ public class TestAsyncDFS {
     for (int i = 0; i < NUM_TESTS; i++) {
       assertTrue(fs.exists(dsts[i]));
       FsPermission fsPerm = new FsPermission(permissions[i]);
-      fs.access(dsts[i], fsPerm.getUserAction());
+      checkAccessPermissions(fs.getFileStatus(dsts[i]), fsPerm.getUserAction());
     }
 
     // test setOwner
@@ -470,4 +474,34 @@ public class TestAsyncDFS {
       assertTrue("group2".equals(fs.getFileStatus(dsts[i]).getGroup()));
     }
   }
+
+  static void checkAccessPermissions(FileStatus stat, FsAction mode)
+      throws IOException {
+    checkAccessPermissions(UserGroupInformation.getCurrentUser(), stat, mode);
+  }
+
+  static void checkAccessPermissions(final UserGroupInformation ugi,
+      FileStatus stat, FsAction mode) throws IOException {
+    FsPermission perm = stat.getPermission();
+    String user = ugi.getShortUserName();
+    List<String> groups = Arrays.asList(ugi.getGroupNames());
+
+    if (user.equals(stat.getOwner())) {
+      if (perm.getUserAction().implies(mode)) {
+        return;
+      }
+    } else if (groups.contains(stat.getGroup())) {
+      if (perm.getGroupAction().implies(mode)) {
+        return;
+      }
+    } else {
+      if (perm.getOtherAction().implies(mode)) {
+        return;
+      }
+    }
+    throw new AccessControlException(String.format(
+        "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat
+            .getPath(), stat.getOwner(), stat.getGroup(),
+        stat.isDirectory() ? "d" : "-", perm));
+  }
 }


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


[20/47] hadoop git commit: HDFS-9877. HDFS Namenode UI: Fix browsing directories that need to be encoded (Ravi Prakash via aw)

Posted by ae...@apache.org.
HDFS-9877. HDFS Namenode UI: Fix browsing directories that need to be encoded (Ravi Prakash via aw)


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

Branch: refs/heads/HDFS-1312
Commit: 15f018434c5b715729488fd0b03a11f1bc943470
Parents: 713cb71
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jun 3 17:06:29 2016 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Fri Jun 3 17:06:29 2016 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/15f01843/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 102da9d..adb83a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -33,7 +33,7 @@
   $(window).bind('hashchange', function () {
     $('#alert-panel').hide();
 
-    var dir = window.location.hash.slice(1);
+    var dir = decodeURIComponent(window.location.hash.slice(1));
     if(dir == "") {
       dir = "/";
     }


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


[14/47] hadoop git commit: YARN-5190. Registering/unregistering container metrics in ContainerMonitorImpl and ContainerImpl causing uncaught exception in ContainerMonitorImpl. Contributed by Junping Du

Posted by ae...@apache.org.
YARN-5190. Registering/unregistering container metrics in ContainerMonitorImpl and ContainerImpl causing uncaught exception in ContainerMonitorImpl. Contributed by Junping Du


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

Branch: refs/heads/HDFS-1312
Commit: 99cc439e29794f8e61bebe03b2a7ca4b6743ec92
Parents: 097baaa
Author: Jian He <ji...@apache.org>
Authored: Fri Jun 3 11:10:42 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Fri Jun 3 11:10:42 2016 -0700

----------------------------------------------------------------------
 .../hadoop/metrics2/impl/MetricsSystemImpl.java   |  1 +
 .../hadoop/metrics2/lib/DefaultMetricsSystem.java |  9 +++++++++
 .../monitor/ContainerMetrics.java                 | 18 +++++++++++++-----
 .../monitor/ContainersMonitorImpl.java            | 16 ++++++++++++----
 .../monitor/TestContainerMetrics.java             |  4 +++-
 5 files changed, 38 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99cc439e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
index ef7306b..6986edb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
@@ -255,6 +255,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
     if (namedCallbacks.containsKey(name)) {
       namedCallbacks.remove(name);
     }
+    DefaultMetricsSystem.removeSourceName(name);
   }
 
   synchronized

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99cc439e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java
index c761b58..935f47f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystem.java
@@ -116,6 +116,11 @@ public enum DefaultMetricsSystem {
   }
 
   @InterfaceAudience.Private
+  public static void removeSourceName(String name) {
+    INSTANCE.removeSource(name);
+  }
+
+  @InterfaceAudience.Private
   public static String sourceName(String name, boolean dupOK) {
     return INSTANCE.newSourceName(name, dupOK);
   }
@@ -135,6 +140,10 @@ public enum DefaultMetricsSystem {
     mBeanNames.map.remove(name);
   }
 
+  synchronized void removeSource(String name) {
+    sourceNames.map.remove(name);
+  }
+
   synchronized String newSourceName(String name, boolean dupOK) {
     if (sourceNames.map.containsKey(name)) {
       if (dupOK) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99cc439e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java
index d59abda..31a9aa7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java
@@ -198,6 +198,12 @@ public class ContainerMetrics implements MetricsSource {
         DefaultMetricsSystem.instance(), containerId, flushPeriodMs, delayMs);
   }
 
+  public synchronized static ContainerMetrics getContainerMetrics(
+      ContainerId containerId) {
+    // could be null
+    return usageMetrics.get(containerId);
+  }
+
   synchronized static ContainerMetrics forContainer(
       MetricsSystem ms, ContainerId containerId, long flushPeriodMs,
       long delayMs) {
@@ -237,12 +243,14 @@ public class ContainerMetrics implements MetricsSource {
   }
 
   public synchronized void finished() {
-    this.finished = true;
-    if (timer != null) {
-      timer.cancel();
-      timer = null;
+    if (!finished) {
+      this.finished = true;
+      if (timer != null) {
+        timer.cancel();
+        timer = null;
+      }
+      scheduleTimerTaskForUnregistration();
     }
-    scheduleTimerTaskForUnregistration();
   }
 
   public void recordMemoryUsage(int memoryMBs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99cc439e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index 3fbae10..cfe6f801 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -619,15 +619,16 @@ public class ContainersMonitorImpl extends AbstractService implements
     }
 
     ContainerId containerId = monitoringEvent.getContainerId();
-    ContainerMetrics usageMetrics = ContainerMetrics
-        .forContainer(containerId, containerMetricsPeriodMs,
-        containerMetricsUnregisterDelayMs);
+    ContainerMetrics usageMetrics;
 
     int vmemLimitMBs;
     int pmemLimitMBs;
     int cpuVcores;
     switch (monitoringEvent.getType()) {
     case START_MONITORING_CONTAINER:
+      usageMetrics = ContainerMetrics
+          .forContainer(containerId, containerMetricsPeriodMs,
+          containerMetricsUnregisterDelayMs);
       ContainerStartMonitoringEvent startEvent =
           (ContainerStartMonitoringEvent) monitoringEvent;
       usageMetrics.recordStateChangeDurations(
@@ -640,9 +641,16 @@ public class ContainersMonitorImpl extends AbstractService implements
           vmemLimitMBs, pmemLimitMBs, cpuVcores);
       break;
     case STOP_MONITORING_CONTAINER:
-      usageMetrics.finished();
+      usageMetrics = ContainerMetrics.getContainerMetrics(
+          containerId);
+      if (usageMetrics != null) {
+        usageMetrics.finished();
+      }
       break;
     case CHANGE_MONITORING_CONTAINER_RESOURCE:
+      usageMetrics = ContainerMetrics
+          .forContainer(containerId, containerMetricsPeriodMs,
+          containerMetricsUnregisterDelayMs);
       ChangeMonitoringContainerResourceEvent changeEvent =
           (ChangeMonitoringContainerResourceEvent) monitoringEvent;
       Resource resource = changeEvent.getResource();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99cc439e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.java
index 6966489..1840d62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.java
@@ -142,7 +142,6 @@ public class TestContainerMetrics {
     system.sampleMetrics();
     system.sampleMetrics();
     Thread.sleep(100);
-    system.stop();
     // verify metrics1 is unregistered
     assertTrue(metrics1 != ContainerMetrics.forContainer(
         system, containerId1, 1, 0));
@@ -152,6 +151,9 @@ public class TestContainerMetrics {
     // verify metrics3 is still registered
     assertTrue(metrics3 == ContainerMetrics.forContainer(
         system, containerId3, 1, 0));
+    // YARN-5190: move stop() to the end to verify registering containerId1 and
+    // containerId2 won't get MetricsException thrown.
+    system.stop();
     system.shutdown();
   }
 


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


[08/47] hadoop git commit: HADOOP-13224. Grep job in Single Cluster document fails. (aajisaka)

Posted by ae...@apache.org.
HADOOP-13224. Grep job in Single Cluster document fails. (aajisaka)


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

Branch: refs/heads/HDFS-1312
Commit: ead61c45da3cb70192d3b9c374bdc83087c992df
Parents: dc26601
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jun 3 02:00:13 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jun 3 02:00:13 2016 +0900

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/SingleCluster.md.vm           | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ead61c45/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm b/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
index 72522bc..573ca32 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
@@ -153,7 +153,8 @@ The following instructions are to run a MapReduce job locally. If you want to ex
 
 5.  Copy the input files into the distributed filesystem:
 
-          $ bin/hdfs dfs -put etc/hadoop input
+          $ bin/hdfs dfs -mkdir input
+          $ bin/hdfs dfs -put etc/hadoop/*.xml input
 
 6.  Run some of the examples provided:
 


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


[09/47] hadoop git commit: HDFS-10471. DFSAdmin#SetQuotaCommand's help msg is not correct. Contributed by Yiqun Lin.

Posted by ae...@apache.org.
HDFS-10471. DFSAdmin#SetQuotaCommand's help msg is not correct. Contributed by Yiqun Lin.


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

Branch: refs/heads/HDFS-1312
Commit: 1df6f5735c9d85e644d99d3ebfc4459490657004
Parents: ead61c4
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jun 3 04:10:32 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jun 3 04:10:32 2016 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  | 42 ++++++++++----------
 .../src/test/resources/testHDFSConf.xml         |  2 +-
 2 files changed, 22 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1df6f573/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 08d3da5..45c4952 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -165,15 +165,15 @@ public class DFSAdmin extends FsShell {
     private static final String USAGE =
       "-"+NAME+" <quota> <dirname>...<dirname>";
     private static final String DESCRIPTION = 
-      "-setQuota <quota> <dirname>...<dirname>: " +
-      "Set the quota <quota> for each directory <dirName>.\n" + 
-      "\t\tThe directory quota is a long integer that puts a hard limit\n" +
-      "\t\ton the number of names in the directory tree\n" +
-      "\t\tFor each directory, attempt to set the quota. An error will be reported if\n" +
-      "\t\t1. N is not a positive integer, or\n" +
-      "\t\t2. User is not an administrator, or\n" +
-      "\t\t3. The directory does not exist or is a file.\n" +
-      "\t\tNote: A quota of 1 would force the directory to remain empty.\n";
+        "-setQuota <quota> <dirname>...<dirname>: " +
+        "Set the quota <quota> for each directory <dirName>.\n" +
+        "\t\tThe directory quota is a long integer that puts a hard limit\n" +
+        "\t\ton the number of names in the directory tree\n" +
+        "\t\tFor each directory, attempt to set the quota. An error will be reported if\n" +
+        "\t\t1. quota is not a positive integer, or\n" +
+        "\t\t2. User is not an administrator, or\n" +
+        "\t\t3. The directory does not exist or is a file.\n" +
+        "\t\tNote: A quota of 1 would force the directory to remain empty.\n";
 
     private final long quota; // the quota to be set
 
@@ -263,18 +263,18 @@ public class DFSAdmin extends FsShell {
     private static final String USAGE =
       "-"+NAME+" <quota> [-storageType <storagetype>] <dirname>...<dirname>";
     private static final String DESCRIPTION = USAGE + ": " +
-      "Set the space quota <quota> for each directory <dirName>.\n" +
-      "\t\tThe space quota is a long integer that puts a hard limit\n" +
-      "\t\ton the total size of all the files under the directory tree.\n" +
-      "\t\tThe extra space required for replication is also counted. E.g.\n" +
-      "\t\ta 1GB file with replication of 3 consumes 3GB of the quota.\n\n" +
-      "\t\tQuota can also be specified with a binary prefix for terabytes,\n" +
-      "\t\tpetabytes etc (e.g. 50t is 50TB, 5m is 5MB, 3p is 3PB).\n" + 
-      "\t\tFor each directory, attempt to set the quota. An error will be reported if\n" +
-      "\t\t1. N is not a positive integer, or\n" +
-      "\t\t2. user is not an administrator, or\n" +
-      "\t\t3. the directory does not exist or is a file.\n" +
-      "\t\tThe storage type specific quota is set when -storageType option is specified.\n";
+        "Set the space quota <quota> for each directory <dirName>.\n" +
+        "\t\tThe space quota is a long integer that puts a hard limit\n" +
+        "\t\ton the total size of all the files under the directory tree.\n" +
+        "\t\tThe extra space required for replication is also counted. E.g.\n" +
+        "\t\ta 1GB file with replication of 3 consumes 3GB of the quota.\n\n" +
+        "\t\tQuota can also be specified with a binary prefix for terabytes,\n" +
+        "\t\tpetabytes etc (e.g. 50t is 50TB, 5m is 5MB, 3p is 3PB).\n" +
+        "\t\tFor each directory, attempt to set the quota. An error will be reported if\n" +
+        "\t\t1. quota is not a positive integer or zero, or\n" +
+        "\t\t2. user is not an administrator, or\n" +
+        "\t\t3. the directory does not exist or is a file.\n" +
+        "\t\tThe storage type specific quota is set when -storageType option is specified.\n";
 
     private long quota; // the quota to be set
     private StorageType type;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1df6f573/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index 086bbe9..627f0e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -15868,7 +15868,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*1. N is not a positive integer, or( )*</expected-output>
+          <expected-output>^( |\t)*1. quota is not a positive integer, or( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>


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


[47/47] hadoop git commit: Merge branch 'trunk' into HDFS-1312

Posted by ae...@apache.org.
Merge branch 'trunk' into HDFS-1312


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

Branch: refs/heads/HDFS-1312
Commit: 76a1391d53373aa99251f669aab35d9e54e866f4
Parents: 7c79136 c14c1b2
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Jun 7 10:31:57 2016 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue Jun 7 10:31:57 2016 -0700

----------------------------------------------------------------------
 .../KeyProviderDelegationTokenExtension.java    |  30 +-
 .../crypto/key/kms/KMSClientProvider.java       | 158 +++++-
 .../key/kms/LoadBalancingKMSClientProvider.java |  21 +
 .../hadoop/crypto/key/kms/ValueQueue.java       |   2 +-
 .../org/apache/hadoop/fs/LocalDirAllocator.java | 153 +++---
 .../hadoop/metrics2/impl/MetricsSystemImpl.java |   1 +
 .../metrics2/lib/DefaultMetricsSystem.java      |   9 +
 .../hadoop/security/LdapGroupsMapping.java      |  12 +
 .../hadoop/security/token/DtFileOperations.java |  23 +
 .../hadoop/security/token/DtUtilShell.java      |  47 +-
 .../DelegationTokenAuthenticationHandler.java   |   7 +
 .../web/DelegationTokenAuthenticator.java       |  19 +
 .../java/org/apache/hadoop/util/KMSUtil.java    |  76 +++
 ...rg.apache.hadoop.security.token.TokenRenewer |  14 +
 .../src/main/resources/core-default.xml         |  24 +
 .../hadoop-common/src/site/markdown/Metrics.md  |   1 +
 .../src/site/markdown/SingleCluster.md.vm       |   3 +-
 .../hadoop/fs/contract/ContractTestUtils.java   | 420 +++++++++++++++
 .../org/apache/hadoop/net/ServerSocketUtil.java |  39 ++
 .../hadoop/security/TestLdapGroupsMapping.java  | 140 +++++
 .../hadoop/security/token/TestDtUtilShell.java  |  24 +
 .../delegation/web/TestWebDelegationToken.java  | 114 +++-
 .../key/kms/server/KMSAuthenticationFilter.java |   2 +-
 .../hadoop/crypto/key/kms/server/TestKMS.java   | 127 ++++-
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  38 +-
 .../hadoop/fs/http/server/HttpFSServer.java     | 218 ++++----
 .../server/blockmanagement/BlockManager.java    |   4 +
 .../PendingReconstructionBlocks.java            |  16 +-
 .../server/namenode/EncryptionZoneManager.java  |  35 +-
 .../server/namenode/FSDirEncryptionZoneOp.java  |   2 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  11 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   5 +
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  42 +-
 .../src/main/webapps/hdfs/explorer.html         |  25 +-
 .../src/main/webapps/hdfs/explorer.js           |  53 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |   4 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |  16 +-
 .../TestPendingReconstruction.java              |  20 +-
 .../src/test/resources/testHDFSConf.xml         |   2 +-
 .../hadoop/mapred/LocalContainerLauncher.java   |  28 +
 .../v2/app/job/impl/TaskAttemptImpl.java        |   5 +-
 .../v2/app/launcher/ContainerLauncherEvent.java |  21 +-
 .../v2/app/launcher/ContainerLauncherImpl.java  |  19 +-
 .../v2/app/rm/RMContainerRequestor.java         |   4 +-
 .../v2/app/launcher/TestContainerLauncher.java  |  10 +-
 .../app/launcher/TestContainerLauncherImpl.java |   8 +
 .../hadoop/mapred/ResourceMgrDelegate.java      |   5 +-
 .../hadoop/mapred/TestClientRedirect.java       |   2 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  | 119 +++++
 .../org/apache/hadoop/fs/s3a/Constants.java     |  11 +-
 .../fs/s3a/ProgressableProgressListener.java    |  94 ++++
 .../hadoop/fs/s3a/S3AFastOutputStream.java      |  65 +--
 .../org/apache/hadoop/fs/s3a/S3AFileStatus.java |   7 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 529 ++++++++++++++-----
 .../hadoop/fs/s3a/S3AInstrumentation.java       | 218 +++++---
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |  98 +---
 .../hadoop/fs/s3a/S3AStorageStatistics.java     | 104 ++++
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  48 ++
 .../org/apache/hadoop/fs/s3a/Statistic.java     | 143 +++++
 .../src/site/markdown/tools/hadoop-aws/index.md |  53 +-
 .../hadoop/fs/s3a/AbstractS3ATestBase.java      | 103 ++++
 .../apache/hadoop/fs/s3a/S3ATestConstants.java  |  76 +++
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  | 226 +++++++-
 .../apache/hadoop/fs/s3a/TestS3AEncryption.java | 104 ++++
 .../TestS3AEncryptionAlgorithmPropagation.java  |  82 +++
 .../s3a/TestS3AEncryptionFastOutputStream.java  |  35 ++
 .../hadoop/fs/s3a/TestS3AFileOperationCost.java | 191 +++++++
 .../fs/s3a/TestS3AFileSystemContract.java       |   1 -
 .../hadoop/fs/s3a/scale/S3AScaleTestBase.java   | 130 +----
 .../fs/s3a/scale/TestS3ADeleteManyFiles.java    |  10 +-
 .../s3a/scale/TestS3ADirectoryPerformance.java  | 189 +++++++
 .../scale/TestS3AInputStreamPerformance.java    |   6 +-
 .../src/test/resources/log4j.properties         |   4 +-
 .../yarn/api/ApplicationClientProtocol.java     |   2 +-
 .../yarn/api/ContainerManagementProtocol.java   |   5 +
 .../SignalContainerResponse.java                |   2 +-
 .../yarn/api/records/ExecutionTypeRequest.java  | 124 +++++
 .../yarn/api/records/ResourceRequest.java       |  34 +-
 .../main/proto/applicationclient_protocol.proto |   2 +-
 .../proto/containermanagement_protocol.proto    |   1 +
 .../src/main/proto/yarn_protos.proto            |   7 +-
 .../hadoop/yarn/client/api/YarnClient.java      |   2 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |   4 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   6 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |  17 +-
 .../api/impl/TestDistributedScheduling.java     |   9 +-
 .../yarn/client/api/impl/TestYarnClient.java    |   4 +-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |  70 ++-
 .../yarn/api/ContainerManagementProtocolPB.java |   7 +
 .../ApplicationClientProtocolPBClientImpl.java  |   4 +-
 ...ContainerManagementProtocolPBClientImpl.java |  19 +
 .../ApplicationClientProtocolPBServiceImpl.java |   5 +-
 ...ontainerManagementProtocolPBServiceImpl.java |  20 +
 .../impl/pb/ExecutionTypeRequestPBImpl.java     |  93 ++++
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  15 +
 .../records/impl/pb/ResourceRequestPBImpl.java  |  52 +-
 .../logaggregation/AggregatedLogFormat.java     |  89 +++-
 .../logaggregation/ContainerLogsRequest.java    |  13 +-
 .../yarn/logaggregation/LogCLIHelpers.java      |  25 +-
 .../hadoop/yarn/TestContainerLaunchRPC.java     |  10 +
 .../yarn/TestContainerResourceIncreaseRPC.java  |   8 +
 .../java/org/apache/hadoop/yarn/TestRPC.java    |  10 +
 .../hadoop/yarn/api/TestPBImplRecords.java      |   2 +
 .../webapp/AHSWebServices.java                  |  50 +-
 .../webapp/TestAHSWebServices.java              |  66 +++
 .../containermanager/ContainerManagerImpl.java  |  38 +-
 .../monitor/ContainerMetrics.java               |  18 +-
 .../monitor/ContainersMonitorImpl.java          |  16 +-
 .../nodemanager/scheduler/LocalScheduler.java   |   3 +-
 .../nodemanager/webapp/NMWebServices.java       |  49 +-
 .../yarn/server/nodemanager/TestEventFlow.java  |   3 +
 .../TestNodeStatusUpdaterForLabels.java         |   7 +
 .../amrmproxy/MockResourceManagerFacade.java    |   2 +-
 .../BaseContainerManagerTest.java               |   3 +
 .../monitor/TestContainerMetrics.java           |   4 +-
 .../scheduler/TestLocalScheduler.java           |  11 +-
 .../nodemanager/webapp/TestNMWebServices.java   |  48 +-
 .../server/resourcemanager/ClientRMService.java |   2 +-
 .../reservation/NoOverCommitPolicy.java         |  38 +-
 .../RLESparseResourceAllocation.java            |   6 +-
 .../planning/StageAllocatorGreedyRLE.java       |  16 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |  28 +-
 .../scheduler/AbstractYarnScheduler.java        |   8 -
 .../security/DelegationTokenRenewer.java        |  27 +-
 .../yarn/server/resourcemanager/MockRM.java     |  20 +-
 .../server/resourcemanager/NodeManager.java     |   9 +-
 .../resourcemanager/TestAMAuthorization.java    |   8 +
 .../TestApplicationMasterLauncher.java          |   8 +
 .../TestDistributedSchedulingService.java       |  17 +-
 .../resourcemanager/TestSignalContainer.java    |   2 +-
 .../TestWorkPreservingRMRestart.java            |  92 ++++
 .../TestRLESparseResourceAllocation.java        |  22 +
 .../planning/TestSimpleCapacityReplanner.java   |   2 +-
 .../attempt/TestRMAppAttemptTransitions.java    |   3 +-
 .../security/TestDelegationTokenRenewer.java    |  98 ++++
 135 files changed, 5155 insertions(+), 935 deletions(-)
----------------------------------------------------------------------



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


[29/47] hadoop git commit: Revert "HADOOP-12957. Limit the number of outstanding async calls. Contributed by Xiaobing Zhou"

Posted by ae...@apache.org.
Revert "HADOOP-12957. Limit the number of outstanding async calls.  Contributed by Xiaobing Zhou"

This reverts commit 1b9f18623ab55507bea94888317c7d63d0f4a6f2.


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

Branch: refs/heads/HDFS-1312
Commit: 4d36b221a24e3b626bb91093b0bb0fd377061cae
Parents: f23d5df
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 18:09:18 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 18:09:18 2016 -0700

----------------------------------------------------------------------
 .../hadoop/fs/CommonConfigurationKeys.java      |   3 -
 .../ipc/AsyncCallLimitExceededException.java    |  36 ---
 .../main/java/org/apache/hadoop/ipc/Client.java |  66 +----
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     | 199 ++--------------
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |  12 +-
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 238 ++++++-------------
 6 files changed, 109 insertions(+), 445 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d36b221/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 06614db..86e1b43 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -324,9 +324,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final long HADOOP_SECURITY_UID_NAME_CACHE_TIMEOUT_DEFAULT =
     4*60*60; // 4 hours
   
-  public static final String  IPC_CLIENT_ASYNC_CALLS_MAX_KEY =
-      "ipc.client.async.calls.max";
-  public static final int     IPC_CLIENT_ASYNC_CALLS_MAX_DEFAULT = 100;
   public static final String  IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "ipc.client.fallback-to-simple-auth-allowed";
   public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d36b221/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AsyncCallLimitExceededException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AsyncCallLimitExceededException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AsyncCallLimitExceededException.java
deleted file mode 100644
index db97b6c..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AsyncCallLimitExceededException.java
+++ /dev/null
@@ -1,36 +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.ipc;
-
-import java.io.IOException;
-
-/**
- * Signals that an AsyncCallLimitExceededException has occurred. This class is
- * used to make application code using async RPC aware that limit of max async
- * calls is reached, application code need to retrieve results from response of
- * established async calls to avoid buffer overflow in order for follow-on async
- * calls going correctly.
- */
-public class AsyncCallLimitExceededException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public AsyncCallLimitExceededException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d36b221/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 9be4649..d59aeb89 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -159,9 +159,7 @@ public class Client implements AutoCloseable {
 
   private final boolean fallbackAllowed;
   private final byte[] clientId;
-  private final int maxAsyncCalls;
-  private final AtomicInteger asyncCallCounter = new AtomicInteger(0);
-
+  
   /**
    * Executor on which IPC calls' parameters are sent.
    * Deferring the sending of parameters to a separate
@@ -1290,9 +1288,6 @@ public class Client implements AutoCloseable {
         CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
     this.clientId = ClientId.getClientId();
     this.sendParamsExecutor = clientExcecutorFactory.refAndGetInstance();
-    this.maxAsyncCalls = conf.getInt(
-        CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
-        CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_DEFAULT);
   }
 
   /**
@@ -1359,20 +1354,6 @@ public class Client implements AutoCloseable {
       fallbackToSimpleAuth);
   }
 
-  private void checkAsyncCall() throws IOException {
-    if (isAsynchronousMode()) {
-      if (asyncCallCounter.incrementAndGet() > maxAsyncCalls) {
-        asyncCallCounter.decrementAndGet();
-        String errMsg = String.format(
-            "Exceeded limit of max asynchronous calls: %d, " +
-            "please configure %s to adjust it.",
-            maxAsyncCalls,
-            CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY);
-        throw new AsyncCallLimitExceededException(errMsg);
-      }
-    }
-  }
-
   /**
    * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
    * <code>remoteId</code>, returning the rpc response.
@@ -1393,38 +1374,24 @@ public class Client implements AutoCloseable {
     final Call call = createCall(rpcKind, rpcRequest);
     final Connection connection = getConnection(remoteId, call, serviceClass,
         fallbackToSimpleAuth);
-
     try {
-      checkAsyncCall();
-      try {
-        connection.sendRpcRequest(call);                 // send the rpc request
-      } catch (RejectedExecutionException e) {
-        throw new IOException("connection has been closed", e);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        LOG.warn("interrupted waiting to send rpc request to server", e);
-        throw new IOException(e);
-      }
-    } catch(Exception e) {
-      if (isAsynchronousMode()) {
-        releaseAsyncCall();
-      }
-      throw e;
+      connection.sendRpcRequest(call);                 // send the rpc request
+    } catch (RejectedExecutionException e) {
+      throw new IOException("connection has been closed", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.warn("interrupted waiting to send rpc request to server", e);
+      throw new IOException(e);
     }
 
     if (isAsynchronousMode()) {
       Future<Writable> returnFuture = new AbstractFuture<Writable>() {
-        private final AtomicBoolean callled = new AtomicBoolean(false);
         @Override
         public Writable get() throws InterruptedException, ExecutionException {
-          if (callled.compareAndSet(false, true)) {
-            try {
-              set(getRpcResponse(call, connection));
-            } catch (IOException ie) {
-              setException(ie);
-            } finally {
-              releaseAsyncCall();
-            }
+          try {
+            set(getRpcResponse(call, connection));
+          } catch (IOException ie) {
+            setException(ie);
           }
           return super.get();
         }
@@ -1460,15 +1427,6 @@ public class Client implements AutoCloseable {
     asynchronousMode.set(async);
   }
 
-  private void releaseAsyncCall() {
-    asyncCallCounter.decrementAndGet();
-  }
-
-  @VisibleForTesting
-  int getAsyncCallCount() {
-    return asyncCallCounter.get();
-  }
-
   private Writable getRpcResponse(final Call call, final Connection connection)
       throws IOException {
     synchronized (call) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d36b221/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
index 8ee3a2c..6cf75c7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.ipc;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -35,7 +34,6 @@ 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.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.ipc.RPC.RpcKind;
@@ -56,13 +54,12 @@ public class TestAsyncIPC {
   @Before
   public void setupConf() {
     conf = new Configuration();
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY, 10000);
     Client.setPingInterval(conf, TestIPC.PING_INTERVAL);
     // set asynchronous mode for main thread
     Client.setAsynchronousMode(true);
   }
 
-  static class AsyncCaller extends Thread {
+  protected static class SerialCaller extends Thread {
     private Client client;
     private InetSocketAddress server;
     private int count;
@@ -71,11 +68,11 @@ public class TestAsyncIPC {
         new HashMap<Integer, Future<LongWritable>>();
     Map<Integer, Long> expectedValues = new HashMap<Integer, Long>();
 
-    public AsyncCaller(Client client, InetSocketAddress server, int count) {
+    public SerialCaller(Client client, InetSocketAddress server, int count) {
       this.client = client;
       this.server = server;
       this.count = count;
-      // set asynchronous mode, since AsyncCaller extends Thread
+      // set asynchronous mode, since SerialCaller extends Thread
       Client.setAsynchronousMode(true);
     }
 
@@ -110,111 +107,14 @@ public class TestAsyncIPC {
     }
   }
 
-  static class AsyncLimitlCaller extends Thread {
-    private Client client;
-    private InetSocketAddress server;
-    private int count;
-    private boolean failed;
-    Map<Integer, Future<LongWritable>> returnFutures = new HashMap<Integer, Future<LongWritable>>();
-    Map<Integer, Long> expectedValues = new HashMap<Integer, Long>();
-    int start = 0, end = 0;
-
-    int getStart() {
-      return start;
-    }
-
-    int getEnd() {
-      return end;
-    }
-
-    int getCount() {
-      return count;
-    }
-
-    public AsyncLimitlCaller(Client client, InetSocketAddress server, int count) {
-      this(0, client, server, count);
-    }
-
-    final int callerId;
-
-    public AsyncLimitlCaller(int callerId, Client client, InetSocketAddress server,
-        int count) {
-      this.client = client;
-      this.server = server;
-      this.count = count;
-      // set asynchronous mode, since AsyncLimitlCaller extends Thread
-      Client.setAsynchronousMode(true);
-      this.callerId = callerId;
-    }
-
-    @Override
-    public void run() {
-      // in case Thread#Start is called, which will spawn new thread
-      Client.setAsynchronousMode(true);
-      for (int i = 0; i < count; i++) {
-        try {
-          final long param = TestIPC.RANDOM.nextLong();
-          runCall(i, param);
-        } catch (Exception e) {
-          LOG.fatal(String.format("Caller-%d Call-%d caught: %s", callerId, i,
-              StringUtils.stringifyException(e)));
-          failed = true;
-        }
-      }
-    }
-
-    private void runCall(final int idx, final long param)
-        throws InterruptedException, ExecutionException, IOException {
-      for (;;) {
-        try {
-          doCall(idx, param);
-          return;
-        } catch (AsyncCallLimitExceededException e) {
-          /**
-           * reached limit of async calls, fetch results of finished async calls
-           * to let follow-on calls go
-           */
-          start = end;
-          end = idx;
-          waitForReturnValues(start, end);
-        }
-      }
-    }
-
-    private void doCall(final int idx, final long param) throws IOException {
-      TestIPC.call(client, param, server, conf);
-      Future<LongWritable> returnFuture = Client.getReturnRpcResponse();
-      returnFutures.put(idx, returnFuture);
-      expectedValues.put(idx, param);
-    }
-
-    private void waitForReturnValues(final int start, final int end)
-        throws InterruptedException, ExecutionException {
-      for (int i = start; i < end; i++) {
-        LongWritable value = returnFutures.get(i).get();
-        if (expectedValues.get(i) != value.get()) {
-          LOG.fatal(String.format("Caller-%d Call-%d failed!", callerId, i));
-          failed = true;
-          break;
-        }
-      }
-    }
-  }
-
-  @Test(timeout = 60000)
-  public void testAsyncCall() throws IOException, InterruptedException,
+  @Test
+  public void testSerial() throws IOException, InterruptedException,
       ExecutionException {
-    internalTestAsyncCall(3, false, 2, 5, 100);
-    internalTestAsyncCall(3, true, 2, 5, 10);
+    internalTestSerial(3, false, 2, 5, 100);
+    internalTestSerial(3, true, 2, 5, 10);
   }
 
-  @Test(timeout = 60000)
-  public void testAsyncCallLimit() throws IOException,
-      InterruptedException, ExecutionException {
-    internalTestAsyncCallLimit(100, false, 5, 10, 500);
-  }
-
-  public void internalTestAsyncCall(int handlerCount, boolean handlerSleep,
+  public void internalTestSerial(int handlerCount, boolean handlerSleep,
       int clientCount, int callerCount, int callCount) throws IOException,
       InterruptedException, ExecutionException {
     Server server = new TestIPC.TestServer(handlerCount, handlerSleep, conf);
@@ -226,9 +126,9 @@ public class TestAsyncIPC {
       clients[i] = new Client(LongWritable.class, conf);
     }
 
-    AsyncCaller[] callers = new AsyncCaller[callerCount];
+    SerialCaller[] callers = new SerialCaller[callerCount];
     for (int i = 0; i < callerCount; i++) {
-      callers[i] = new AsyncCaller(clients[i % clientCount], addr, callCount);
+      callers[i] = new SerialCaller(clients[i % clientCount], addr, callCount);
       callers[i].start();
     }
     for (int i = 0; i < callerCount; i++) {
@@ -244,75 +144,6 @@ public class TestAsyncIPC {
     server.stop();
   }
 
-  @Test(timeout = 60000)
-  public void testCallGetReturnRpcResponseMultipleTimes() throws IOException,
-      InterruptedException, ExecutionException {
-    int handlerCount = 10, callCount = 100;
-    Server server = new TestIPC.TestServer(handlerCount, false, conf);
-    InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    server.start();
-    final Client client = new Client(LongWritable.class, conf);
-
-    int asyncCallCount = client.getAsyncCallCount();
-
-    try {
-      AsyncCaller caller = new AsyncCaller(client, addr, callCount);
-      caller.run();
-
-      caller.waitForReturnValues();
-      String msg = String.format(
-          "First time, expected not failed for caller: %s.", caller);
-      assertFalse(msg, caller.failed);
-
-      caller.waitForReturnValues();
-      assertTrue(asyncCallCount == client.getAsyncCallCount());
-      msg = String.format("Second time, expected not failed for caller: %s.",
-          caller);
-      assertFalse(msg, caller.failed);
-
-      assertTrue(asyncCallCount == client.getAsyncCallCount());
-    } finally {
-      client.stop();
-      server.stop();
-    }
-  }
-
-  public void internalTestAsyncCallLimit(int handlerCount, boolean handlerSleep,
-      int clientCount, int callerCount, int callCount) throws IOException,
-      InterruptedException, ExecutionException {
-    Configuration conf = new Configuration();
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY, 100);
-    Client.setPingInterval(conf, TestIPC.PING_INTERVAL);
-
-    Server server = new TestIPC.TestServer(handlerCount, handlerSleep, conf);
-    InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    server.start();
-
-    Client[] clients = new Client[clientCount];
-    for (int i = 0; i < clientCount; i++) {
-      clients[i] = new Client(LongWritable.class, conf);
-    }
-
-    AsyncLimitlCaller[] callers = new AsyncLimitlCaller[callerCount];
-    for (int i = 0; i < callerCount; i++) {
-      callers[i] = new AsyncLimitlCaller(i, clients[i % clientCount], addr,
-          callCount);
-      callers[i].start();
-    }
-    for (int i = 0; i < callerCount; i++) {
-      callers[i].join();
-      callers[i].waitForReturnValues(callers[i].getStart(),
-          callers[i].getCount());
-      String msg = String.format("Expected not failed for caller-%d: %s.", i,
-          callers[i]);
-      assertFalse(msg, callers[i].failed);
-    }
-    for (int i = 0; i < clientCount; i++) {
-      clients[i].stop();
-    }
-    server.stop();
-  }
-
   /**
    * Test if (1) the rpc server uses the call id/retry provided by the rpc
    * client, and (2) the rpc client receives the same call id/retry from the rpc
@@ -365,7 +196,7 @@ public class TestAsyncIPC {
     try {
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
       server.start();
-      final AsyncCaller caller = new AsyncCaller(client, addr, 4);
+      final SerialCaller caller = new SerialCaller(client, addr, 4);
       caller.run();
       caller.waitForReturnValues();
       String msg = String.format("Expected not failed for caller: %s.", caller);
@@ -404,7 +235,7 @@ public class TestAsyncIPC {
     try {
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
       server.start();
-      final AsyncCaller caller = new AsyncCaller(client, addr, 10);
+      final SerialCaller caller = new SerialCaller(client, addr, 10);
       caller.run();
       caller.waitForReturnValues();
       String msg = String.format("Expected not failed for caller: %s.", caller);
@@ -441,7 +272,7 @@ public class TestAsyncIPC {
     try {
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
       server.start();
-      final AsyncCaller caller = new AsyncCaller(client, addr, 10);
+      final SerialCaller caller = new SerialCaller(client, addr, 10);
       caller.run();
       caller.waitForReturnValues();
       String msg = String.format("Expected not failed for caller: %s.", caller);
@@ -482,9 +313,9 @@ public class TestAsyncIPC {
     try {
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
       server.start();
-      AsyncCaller[] callers = new AsyncCaller[callerCount];
+      SerialCaller[] callers = new SerialCaller[callerCount];
       for (int i = 0; i < callerCount; ++i) {
-        callers[i] = new AsyncCaller(client, addr, perCallerCallCount);
+        callers[i] = new SerialCaller(client, addr, perCallerCallCount);
         callers[i].start();
       }
       for (int i = 0; i < callerCount; ++i) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d36b221/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 356ae3f..37899aa 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
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.fs.Options;
@@ -51,14 +50,11 @@ public class AsyncDistributedFileSystem {
     final Callable<T> returnValueCallback = ClientNamenodeProtocolTranslatorPB
         .getReturnValueCallback();
     Future<T> returnFuture = new AbstractFuture<T>() {
-      private final AtomicBoolean called = new AtomicBoolean(false);
       public T get() throws InterruptedException, ExecutionException {
-        if (called.compareAndSet(false, true)) {
-          try {
-            set(returnValueCallback.call());
-          } catch (Exception e) {
-            setException(e);
-          }
+        try {
+          set(returnValueCallback.call());
+        } catch (Exception e) {
+          setException(e);
         }
         return super.get();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d36b221/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 d129299..9322e1a 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.HashMap;
@@ -30,25 +31,80 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestAsyncDFSRename {
+  final Path asyncRenameDir = new Path("/test/async_rename/");
   public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
+  final private static Configuration CONF = new HdfsConfiguration();
+
+  final private static String GROUP1_NAME = "group1";
+  final private static String GROUP2_NAME = "group2";
+  final private static String USER1_NAME = "user1";
+  private static final UserGroupInformation USER1;
+
+  private MiniDFSCluster gCluster;
+
+  static {
+    // explicitly turn on permission checking
+    CONF.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+
+    // create fake mapping for the groups
+    Map<String, String[]> u2g_map = new HashMap<String, String[]>(1);
+    u2g_map.put(USER1_NAME, new String[] { GROUP1_NAME, GROUP2_NAME });
+    DFSTestUtil.updateConfWithFakeGroupMapping(CONF, u2g_map);
+
+    // Initiate all four users
+    USER1 = UserGroupInformation.createUserForTesting(USER1_NAME, new String[] {
+        GROUP1_NAME, GROUP2_NAME });
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    gCluster = new MiniDFSCluster.Builder(CONF).numDataNodes(3).build();
+    gCluster.waitActive();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (gCluster != null) {
+      gCluster.shutdown();
+      gCluster = null;
+    }
+  }
+
+  static int countLease(MiniDFSCluster cluster) {
+    return TestDFSRename.countLease(cluster);
+  }
+
+  void list(DistributedFileSystem dfs, String name) throws IOException {
+    FileSystem.LOG.info("\n\n" + name);
+    for (FileStatus s : dfs.listStatus(asyncRenameDir)) {
+      FileSystem.LOG.info("" + s.getPath());
+    }
+  }
+
+  static void createFile(DistributedFileSystem dfs, Path f) throws IOException {
+    DataOutputStream a_out = dfs.create(f);
+    a_out.writeBytes("something");
+    a_out.close();
+  }
 
   /**
    * Check the blocks of dst file are cleaned after rename with overwrite
    * Restart NN to check the rename successfully
    */
-  @Test(timeout = 60000)
+  @Test
   public void testAsyncRenameWithOverwrite() throws Exception {
     final short replFactor = 2;
     final long blockSize = 512;
@@ -113,134 +169,38 @@ public class TestAsyncDFSRename {
     }
   }
 
-  @Test(timeout = 60000)
-  public void testCallGetReturnValueMultipleTimes() throws Exception {
+  @Test
+  public void testConcurrentAsyncRenameWithOverwrite() throws Exception {
     final short replFactor = 2;
     final long blockSize = 512;
     final Path renameDir = new Path(
-        "/test/testCallGetReturnValueMultipleTimes/");
-    final Configuration conf = new HdfsConfiguration();
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY, 200);
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(2).build();
-    cluster.waitActive();
-    final DistributedFileSystem dfs = cluster.getFileSystem();
-    final AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
-    final int count = 100;
-    long fileLen = blockSize * 3;
-    final Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
-
-    assertTrue(dfs.mkdirs(renameDir));
-
-    try {
-      // concurrently invoking many rename
-      for (int i = 0; i < count; i++) {
-        Path src = new Path(renameDir, "src" + i);
-        Path dst = new Path(renameDir, "dst" + i);
-        DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
-        DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
-        Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-        returnFutures.put(i, returnFuture);
-      }
-
-      for (int i = 0; i < 5; i++) {
-        verifyCallGetReturnValueMultipleTimes(returnFutures, count, cluster,
-            renameDir, dfs);
-      }
-    } finally {
-      if (dfs != null) {
-        dfs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
-  private void verifyCallGetReturnValueMultipleTimes(
-      Map<Integer, Future<Void>> returnFutures, int count,
-      MiniDFSCluster cluster, Path renameDir, DistributedFileSystem dfs)
-      throws InterruptedException, ExecutionException, IOException {
-    // wait for completing the calls
-    for (int i = 0; i < count; i++) {
-      returnFutures.get(i).get();
-    }
-
-    // Restart NN and check the rename successfully
-    cluster.restartNameNodes();
-
-    // very the src dir should not exist, dst should
-    for (int i = 0; i < count; i++) {
-      Path src = new Path(renameDir, "src" + i);
-      Path dst = new Path(renameDir, "dst" + i);
-      assertFalse(dfs.exists(src));
-      assertTrue(dfs.exists(dst));
-    }
-  }
-
-  @Test(timeout = 120000)
-  public void testAggressiveConcurrentAsyncRenameWithOverwrite()
-      throws Exception {
-    internalTestConcurrentAsyncRenameWithOverwrite(100,
-        "testAggressiveConcurrentAsyncRenameWithOverwrite");
-  }
-
-  @Test(timeout = 60000)
-  public void testConservativeConcurrentAsyncRenameWithOverwrite()
-      throws Exception {
-    internalTestConcurrentAsyncRenameWithOverwrite(10000,
-        "testConservativeConcurrentAsyncRenameWithOverwrite");
-  }
-
-  private void internalTestConcurrentAsyncRenameWithOverwrite(
-      final int asyncCallLimit, final String basePath) throws Exception {
-    final short replFactor = 2;
-    final long blockSize = 512;
-    final Path renameDir = new Path(String.format("/test/%s/", basePath));
+        "/test/concurrent_reanme_with_overwrite_dir/");
     Configuration conf = new HdfsConfiguration();
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
-        asyncCallLimit);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
         .build();
     cluster.waitActive();
     DistributedFileSystem dfs = cluster.getFileSystem();
     AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
     int count = 1000;
-    long fileLen = blockSize * 3;
-    int start = 0, end = 0;
-    Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
-
-    assertTrue(dfs.mkdirs(renameDir));
 
     try {
+      long fileLen = blockSize * 3;
+      assertTrue(dfs.mkdirs(renameDir));
+
+      Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
+
       // concurrently invoking many rename
       for (int i = 0; i < count; i++) {
         Path src = new Path(renameDir, "src" + i);
         Path dst = new Path(renameDir, "dst" + i);
         DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
         DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
-        for (;;) {
-          try {
-            LOG.info("rename #" + i);
-            Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-            returnFutures.put(i, returnFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            /**
-             * reached limit of async calls, fetch results of finished async
-             * calls to let follow-on calls go
-             */
-            LOG.error(e);
-            start = end;
-            end = i;
-            LOG.info(String.format("start=%d, end=%d, i=%d", start, end, i));
-            waitForReturnValues(returnFutures, start, end);
-          }
-        }
+        Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+        returnFutures.put(i, returnFuture);
       }
 
       // wait for completing the calls
-      for (int i = start; i < count; i++) {
+      for (int i = 0; i < count; i++) {
         returnFutures.get(i).get();
       }
 
@@ -255,60 +215,26 @@ public class TestAsyncDFSRename {
         assertTrue(dfs.exists(dst));
       }
     } finally {
-      if (dfs != null) {
-        dfs.close();
-      }
+      dfs.delete(renameDir, true);
       if (cluster != null) {
         cluster.shutdown();
       }
     }
   }
 
-  private void waitForReturnValues(
-      final Map<Integer, Future<Void>> returnFutures, final int start,
-      final int end) throws InterruptedException, ExecutionException {
-    LOG.info(String.format("calling waitForReturnValues [%d, %d)", start, end));
-    for (int i = start; i < end; i++) {
-      LOG.info("calling Future#get #" + i);
-      returnFutures.get(i).get();
-    }
-  }
-
-  @Test(timeout = 60000)
+  @Test
   public void testAsyncRenameWithException() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    String group1 = "group1";
-    String group2 = "group2";
-    String user1 = "user1";
-    UserGroupInformation ugi1;
-
-    // explicitly turn on permission checking
-    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-
-    // create fake mapping for the groups
-    Map<String, String[]> u2g_map = new HashMap<String, String[]>(1);
-    u2g_map.put(user1, new String[] { group1, group2 });
-    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
-
-    // Initiate all four users
-    ugi1 = UserGroupInformation.createUserForTesting(user1, new String[] {
-        group1, group2 });
-
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(3).build();
-    cluster.waitActive();
-
-    FileSystem rootFs = FileSystem.get(conf);
+    FileSystem rootFs = FileSystem.get(CONF);
     final Path renameDir = new Path("/test/async_rename_exception/");
     final Path src = new Path(renameDir, "src");
     final Path dst = new Path(renameDir, "dst");
     rootFs.mkdirs(src);
 
-    AsyncDistributedFileSystem adfs = ugi1
+    AsyncDistributedFileSystem adfs = USER1
         .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
           @Override
           public AsyncDistributedFileSystem run() throws Exception {
-            return cluster.getFileSystem().getAsyncDistributedFileSystem();
+            return gCluster.getFileSystem().getAsyncDistributedFileSystem();
           }
         });
 
@@ -316,24 +242,16 @@ public class TestAsyncDFSRename {
       Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
       returnFuture.get();
     } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-    } finally {
-      if (rootFs != null) {
-        rootFs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
+      checkPermissionDenied(e, src);
     }
   }
 
-  private void checkPermissionDenied(final Exception e, final Path dir,
-      final String user) {
+  private void checkPermissionDenied(final Exception e, final Path dir) {
     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));
+        .getMessage().contains(USER1_NAME));
     assertTrue("Permission denied messages must carry the path parent", e
         .getMessage().contains(dir.getParent().toUri().getPath()));
   }


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


[32/47] hadoop git commit: Revert "Revert "HADOOP-12957. Limit the number of outstanding async calls. Contributed by Xiaobing Zhou""

Posted by ae...@apache.org.
Revert "Revert "HADOOP-12957. Limit the number of outstanding async calls.  Contributed by Xiaobing Zhou""

This reverts commit 4d36b221a24e3b626bb91093b0bb0fd377061cae.


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

Branch: refs/heads/HDFS-1312
Commit: aa20fa150d522b9fe469dd99a8e24d7e27d888ea
Parents: eded3d1
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 6 16:28:47 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 6 16:28:47 2016 +0800

----------------------------------------------------------------------
 .../hadoop/fs/CommonConfigurationKeys.java      |   3 +
 .../ipc/AsyncCallLimitExceededException.java    |  36 +++
 .../main/java/org/apache/hadoop/ipc/Client.java |  66 ++++-
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     | 199 ++++++++++++++--
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |  12 +-
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 238 +++++++++++++------
 6 files changed, 445 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa20fa15/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 86e1b43..06614db 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -324,6 +324,9 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final long HADOOP_SECURITY_UID_NAME_CACHE_TIMEOUT_DEFAULT =
     4*60*60; // 4 hours
   
+  public static final String  IPC_CLIENT_ASYNC_CALLS_MAX_KEY =
+      "ipc.client.async.calls.max";
+  public static final int     IPC_CLIENT_ASYNC_CALLS_MAX_DEFAULT = 100;
   public static final String  IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "ipc.client.fallback-to-simple-auth-allowed";
   public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa20fa15/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AsyncCallLimitExceededException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AsyncCallLimitExceededException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AsyncCallLimitExceededException.java
new file mode 100644
index 0000000..db97b6c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AsyncCallLimitExceededException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+
+/**
+ * Signals that an AsyncCallLimitExceededException has occurred. This class is
+ * used to make application code using async RPC aware that limit of max async
+ * calls is reached, application code need to retrieve results from response of
+ * established async calls to avoid buffer overflow in order for follow-on async
+ * calls going correctly.
+ */
+public class AsyncCallLimitExceededException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public AsyncCallLimitExceededException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa20fa15/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index d59aeb89..9be4649 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -159,7 +159,9 @@ public class Client implements AutoCloseable {
 
   private final boolean fallbackAllowed;
   private final byte[] clientId;
-  
+  private final int maxAsyncCalls;
+  private final AtomicInteger asyncCallCounter = new AtomicInteger(0);
+
   /**
    * Executor on which IPC calls' parameters are sent.
    * Deferring the sending of parameters to a separate
@@ -1288,6 +1290,9 @@ public class Client implements AutoCloseable {
         CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
     this.clientId = ClientId.getClientId();
     this.sendParamsExecutor = clientExcecutorFactory.refAndGetInstance();
+    this.maxAsyncCalls = conf.getInt(
+        CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
+        CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_DEFAULT);
   }
 
   /**
@@ -1354,6 +1359,20 @@ public class Client implements AutoCloseable {
       fallbackToSimpleAuth);
   }
 
+  private void checkAsyncCall() throws IOException {
+    if (isAsynchronousMode()) {
+      if (asyncCallCounter.incrementAndGet() > maxAsyncCalls) {
+        asyncCallCounter.decrementAndGet();
+        String errMsg = String.format(
+            "Exceeded limit of max asynchronous calls: %d, " +
+            "please configure %s to adjust it.",
+            maxAsyncCalls,
+            CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY);
+        throw new AsyncCallLimitExceededException(errMsg);
+      }
+    }
+  }
+
   /**
    * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
    * <code>remoteId</code>, returning the rpc response.
@@ -1374,24 +1393,38 @@ public class Client implements AutoCloseable {
     final Call call = createCall(rpcKind, rpcRequest);
     final Connection connection = getConnection(remoteId, call, serviceClass,
         fallbackToSimpleAuth);
+
     try {
-      connection.sendRpcRequest(call);                 // send the rpc request
-    } catch (RejectedExecutionException e) {
-      throw new IOException("connection has been closed", e);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      LOG.warn("interrupted waiting to send rpc request to server", e);
-      throw new IOException(e);
+      checkAsyncCall();
+      try {
+        connection.sendRpcRequest(call);                 // send the rpc request
+      } catch (RejectedExecutionException e) {
+        throw new IOException("connection has been closed", e);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        LOG.warn("interrupted waiting to send rpc request to server", e);
+        throw new IOException(e);
+      }
+    } catch(Exception e) {
+      if (isAsynchronousMode()) {
+        releaseAsyncCall();
+      }
+      throw e;
     }
 
     if (isAsynchronousMode()) {
       Future<Writable> returnFuture = new AbstractFuture<Writable>() {
+        private final AtomicBoolean callled = new AtomicBoolean(false);
         @Override
         public Writable get() throws InterruptedException, ExecutionException {
-          try {
-            set(getRpcResponse(call, connection));
-          } catch (IOException ie) {
-            setException(ie);
+          if (callled.compareAndSet(false, true)) {
+            try {
+              set(getRpcResponse(call, connection));
+            } catch (IOException ie) {
+              setException(ie);
+            } finally {
+              releaseAsyncCall();
+            }
           }
           return super.get();
         }
@@ -1427,6 +1460,15 @@ public class Client implements AutoCloseable {
     asynchronousMode.set(async);
   }
 
+  private void releaseAsyncCall() {
+    asyncCallCounter.decrementAndGet();
+  }
+
+  @VisibleForTesting
+  int getAsyncCallCount() {
+    return asyncCallCounter.get();
+  }
+
   private Writable getRpcResponse(final Call call, final Connection connection)
       throws IOException {
     synchronized (call) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa20fa15/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
index 6cf75c7..8ee3a2c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ipc;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -34,6 +35,7 @@ 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.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.ipc.RPC.RpcKind;
@@ -54,12 +56,13 @@ public class TestAsyncIPC {
   @Before
   public void setupConf() {
     conf = new Configuration();
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY, 10000);
     Client.setPingInterval(conf, TestIPC.PING_INTERVAL);
     // set asynchronous mode for main thread
     Client.setAsynchronousMode(true);
   }
 
-  protected static class SerialCaller extends Thread {
+  static class AsyncCaller extends Thread {
     private Client client;
     private InetSocketAddress server;
     private int count;
@@ -68,11 +71,11 @@ public class TestAsyncIPC {
         new HashMap<Integer, Future<LongWritable>>();
     Map<Integer, Long> expectedValues = new HashMap<Integer, Long>();
 
-    public SerialCaller(Client client, InetSocketAddress server, int count) {
+    public AsyncCaller(Client client, InetSocketAddress server, int count) {
       this.client = client;
       this.server = server;
       this.count = count;
-      // set asynchronous mode, since SerialCaller extends Thread
+      // set asynchronous mode, since AsyncCaller extends Thread
       Client.setAsynchronousMode(true);
     }
 
@@ -107,14 +110,111 @@ public class TestAsyncIPC {
     }
   }
 
-  @Test
-  public void testSerial() throws IOException, InterruptedException,
+  static class AsyncLimitlCaller extends Thread {
+    private Client client;
+    private InetSocketAddress server;
+    private int count;
+    private boolean failed;
+    Map<Integer, Future<LongWritable>> returnFutures = new HashMap<Integer, Future<LongWritable>>();
+    Map<Integer, Long> expectedValues = new HashMap<Integer, Long>();
+    int start = 0, end = 0;
+
+    int getStart() {
+      return start;
+    }
+
+    int getEnd() {
+      return end;
+    }
+
+    int getCount() {
+      return count;
+    }
+
+    public AsyncLimitlCaller(Client client, InetSocketAddress server, int count) {
+      this(0, client, server, count);
+    }
+
+    final int callerId;
+
+    public AsyncLimitlCaller(int callerId, Client client, InetSocketAddress server,
+        int count) {
+      this.client = client;
+      this.server = server;
+      this.count = count;
+      // set asynchronous mode, since AsyncLimitlCaller extends Thread
+      Client.setAsynchronousMode(true);
+      this.callerId = callerId;
+    }
+
+    @Override
+    public void run() {
+      // in case Thread#Start is called, which will spawn new thread
+      Client.setAsynchronousMode(true);
+      for (int i = 0; i < count; i++) {
+        try {
+          final long param = TestIPC.RANDOM.nextLong();
+          runCall(i, param);
+        } catch (Exception e) {
+          LOG.fatal(String.format("Caller-%d Call-%d caught: %s", callerId, i,
+              StringUtils.stringifyException(e)));
+          failed = true;
+        }
+      }
+    }
+
+    private void runCall(final int idx, final long param)
+        throws InterruptedException, ExecutionException, IOException {
+      for (;;) {
+        try {
+          doCall(idx, param);
+          return;
+        } catch (AsyncCallLimitExceededException e) {
+          /**
+           * reached limit of async calls, fetch results of finished async calls
+           * to let follow-on calls go
+           */
+          start = end;
+          end = idx;
+          waitForReturnValues(start, end);
+        }
+      }
+    }
+
+    private void doCall(final int idx, final long param) throws IOException {
+      TestIPC.call(client, param, server, conf);
+      Future<LongWritable> returnFuture = Client.getReturnRpcResponse();
+      returnFutures.put(idx, returnFuture);
+      expectedValues.put(idx, param);
+    }
+
+    private void waitForReturnValues(final int start, final int end)
+        throws InterruptedException, ExecutionException {
+      for (int i = start; i < end; i++) {
+        LongWritable value = returnFutures.get(i).get();
+        if (expectedValues.get(i) != value.get()) {
+          LOG.fatal(String.format("Caller-%d Call-%d failed!", callerId, i));
+          failed = true;
+          break;
+        }
+      }
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testAsyncCall() throws IOException, InterruptedException,
       ExecutionException {
-    internalTestSerial(3, false, 2, 5, 100);
-    internalTestSerial(3, true, 2, 5, 10);
+    internalTestAsyncCall(3, false, 2, 5, 100);
+    internalTestAsyncCall(3, true, 2, 5, 10);
   }
 
-  public void internalTestSerial(int handlerCount, boolean handlerSleep,
+  @Test(timeout = 60000)
+  public void testAsyncCallLimit() throws IOException,
+      InterruptedException, ExecutionException {
+    internalTestAsyncCallLimit(100, false, 5, 10, 500);
+  }
+
+  public void internalTestAsyncCall(int handlerCount, boolean handlerSleep,
       int clientCount, int callerCount, int callCount) throws IOException,
       InterruptedException, ExecutionException {
     Server server = new TestIPC.TestServer(handlerCount, handlerSleep, conf);
@@ -126,9 +226,9 @@ public class TestAsyncIPC {
       clients[i] = new Client(LongWritable.class, conf);
     }
 
-    SerialCaller[] callers = new SerialCaller[callerCount];
+    AsyncCaller[] callers = new AsyncCaller[callerCount];
     for (int i = 0; i < callerCount; i++) {
-      callers[i] = new SerialCaller(clients[i % clientCount], addr, callCount);
+      callers[i] = new AsyncCaller(clients[i % clientCount], addr, callCount);
       callers[i].start();
     }
     for (int i = 0; i < callerCount; i++) {
@@ -144,6 +244,75 @@ public class TestAsyncIPC {
     server.stop();
   }
 
+  @Test(timeout = 60000)
+  public void testCallGetReturnRpcResponseMultipleTimes() throws IOException,
+      InterruptedException, ExecutionException {
+    int handlerCount = 10, callCount = 100;
+    Server server = new TestIPC.TestServer(handlerCount, false, conf);
+    InetSocketAddress addr = NetUtils.getConnectAddress(server);
+    server.start();
+    final Client client = new Client(LongWritable.class, conf);
+
+    int asyncCallCount = client.getAsyncCallCount();
+
+    try {
+      AsyncCaller caller = new AsyncCaller(client, addr, callCount);
+      caller.run();
+
+      caller.waitForReturnValues();
+      String msg = String.format(
+          "First time, expected not failed for caller: %s.", caller);
+      assertFalse(msg, caller.failed);
+
+      caller.waitForReturnValues();
+      assertTrue(asyncCallCount == client.getAsyncCallCount());
+      msg = String.format("Second time, expected not failed for caller: %s.",
+          caller);
+      assertFalse(msg, caller.failed);
+
+      assertTrue(asyncCallCount == client.getAsyncCallCount());
+    } finally {
+      client.stop();
+      server.stop();
+    }
+  }
+
+  public void internalTestAsyncCallLimit(int handlerCount, boolean handlerSleep,
+      int clientCount, int callerCount, int callCount) throws IOException,
+      InterruptedException, ExecutionException {
+    Configuration conf = new Configuration();
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY, 100);
+    Client.setPingInterval(conf, TestIPC.PING_INTERVAL);
+
+    Server server = new TestIPC.TestServer(handlerCount, handlerSleep, conf);
+    InetSocketAddress addr = NetUtils.getConnectAddress(server);
+    server.start();
+
+    Client[] clients = new Client[clientCount];
+    for (int i = 0; i < clientCount; i++) {
+      clients[i] = new Client(LongWritable.class, conf);
+    }
+
+    AsyncLimitlCaller[] callers = new AsyncLimitlCaller[callerCount];
+    for (int i = 0; i < callerCount; i++) {
+      callers[i] = new AsyncLimitlCaller(i, clients[i % clientCount], addr,
+          callCount);
+      callers[i].start();
+    }
+    for (int i = 0; i < callerCount; i++) {
+      callers[i].join();
+      callers[i].waitForReturnValues(callers[i].getStart(),
+          callers[i].getCount());
+      String msg = String.format("Expected not failed for caller-%d: %s.", i,
+          callers[i]);
+      assertFalse(msg, callers[i].failed);
+    }
+    for (int i = 0; i < clientCount; i++) {
+      clients[i].stop();
+    }
+    server.stop();
+  }
+
   /**
    * Test if (1) the rpc server uses the call id/retry provided by the rpc
    * client, and (2) the rpc client receives the same call id/retry from the rpc
@@ -196,7 +365,7 @@ public class TestAsyncIPC {
     try {
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
       server.start();
-      final SerialCaller caller = new SerialCaller(client, addr, 4);
+      final AsyncCaller caller = new AsyncCaller(client, addr, 4);
       caller.run();
       caller.waitForReturnValues();
       String msg = String.format("Expected not failed for caller: %s.", caller);
@@ -235,7 +404,7 @@ public class TestAsyncIPC {
     try {
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
       server.start();
-      final SerialCaller caller = new SerialCaller(client, addr, 10);
+      final AsyncCaller caller = new AsyncCaller(client, addr, 10);
       caller.run();
       caller.waitForReturnValues();
       String msg = String.format("Expected not failed for caller: %s.", caller);
@@ -272,7 +441,7 @@ public class TestAsyncIPC {
     try {
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
       server.start();
-      final SerialCaller caller = new SerialCaller(client, addr, 10);
+      final AsyncCaller caller = new AsyncCaller(client, addr, 10);
       caller.run();
       caller.waitForReturnValues();
       String msg = String.format("Expected not failed for caller: %s.", caller);
@@ -313,9 +482,9 @@ public class TestAsyncIPC {
     try {
       InetSocketAddress addr = NetUtils.getConnectAddress(server);
       server.start();
-      SerialCaller[] callers = new SerialCaller[callerCount];
+      AsyncCaller[] callers = new AsyncCaller[callerCount];
       for (int i = 0; i < callerCount; ++i) {
-        callers[i] = new SerialCaller(client, addr, perCallerCallCount);
+        callers[i] = new AsyncCaller(client, addr, perCallerCallCount);
         callers[i].start();
       }
       for (int i = 0; i < callerCount; ++i) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa20fa15/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 37899aa..356ae3f 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
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.fs.Options;
@@ -50,11 +51,14 @@ public class AsyncDistributedFileSystem {
     final Callable<T> returnValueCallback = ClientNamenodeProtocolTranslatorPB
         .getReturnValueCallback();
     Future<T> returnFuture = new AbstractFuture<T>() {
+      private final AtomicBoolean called = new AtomicBoolean(false);
       public T get() throws InterruptedException, ExecutionException {
-        try {
-          set(returnValueCallback.call());
-        } catch (Exception e) {
-          setException(e);
+        if (called.compareAndSet(false, true)) {
+          try {
+            set(returnValueCallback.call());
+          } catch (Exception e) {
+            setException(e);
+          }
         }
         return super.get();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa20fa15/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 9322e1a..d129299 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
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.HashMap;
@@ -31,80 +30,25 @@ 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.FileStatus;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 
 public class TestAsyncDFSRename {
-  final Path asyncRenameDir = new Path("/test/async_rename/");
   public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
-  final private static Configuration CONF = new HdfsConfiguration();
-
-  final private static String GROUP1_NAME = "group1";
-  final private static String GROUP2_NAME = "group2";
-  final private static String USER1_NAME = "user1";
-  private static final UserGroupInformation USER1;
-
-  private MiniDFSCluster gCluster;
-
-  static {
-    // explicitly turn on permission checking
-    CONF.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-
-    // create fake mapping for the groups
-    Map<String, String[]> u2g_map = new HashMap<String, String[]>(1);
-    u2g_map.put(USER1_NAME, new String[] { GROUP1_NAME, GROUP2_NAME });
-    DFSTestUtil.updateConfWithFakeGroupMapping(CONF, u2g_map);
-
-    // Initiate all four users
-    USER1 = UserGroupInformation.createUserForTesting(USER1_NAME, new String[] {
-        GROUP1_NAME, GROUP2_NAME });
-  }
-
-  @Before
-  public void setUp() throws IOException {
-    gCluster = new MiniDFSCluster.Builder(CONF).numDataNodes(3).build();
-    gCluster.waitActive();
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    if (gCluster != null) {
-      gCluster.shutdown();
-      gCluster = null;
-    }
-  }
-
-  static int countLease(MiniDFSCluster cluster) {
-    return TestDFSRename.countLease(cluster);
-  }
-
-  void list(DistributedFileSystem dfs, String name) throws IOException {
-    FileSystem.LOG.info("\n\n" + name);
-    for (FileStatus s : dfs.listStatus(asyncRenameDir)) {
-      FileSystem.LOG.info("" + s.getPath());
-    }
-  }
-
-  static void createFile(DistributedFileSystem dfs, Path f) throws IOException {
-    DataOutputStream a_out = dfs.create(f);
-    a_out.writeBytes("something");
-    a_out.close();
-  }
 
   /**
    * Check the blocks of dst file are cleaned after rename with overwrite
    * Restart NN to check the rename successfully
    */
-  @Test
+  @Test(timeout = 60000)
   public void testAsyncRenameWithOverwrite() throws Exception {
     final short replFactor = 2;
     final long blockSize = 512;
@@ -169,38 +113,134 @@ public class TestAsyncDFSRename {
     }
   }
 
-  @Test
-  public void testConcurrentAsyncRenameWithOverwrite() throws Exception {
+  @Test(timeout = 60000)
+  public void testCallGetReturnValueMultipleTimes() throws Exception {
     final short replFactor = 2;
     final long blockSize = 512;
     final Path renameDir = new Path(
-        "/test/concurrent_reanme_with_overwrite_dir/");
+        "/test/testCallGetReturnValueMultipleTimes/");
+    final Configuration conf = new HdfsConfiguration();
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY, 200);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(2).build();
+    cluster.waitActive();
+    final DistributedFileSystem dfs = cluster.getFileSystem();
+    final AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
+    final int count = 100;
+    long fileLen = blockSize * 3;
+    final Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
+
+    assertTrue(dfs.mkdirs(renameDir));
+
+    try {
+      // concurrently invoking many rename
+      for (int i = 0; i < count; i++) {
+        Path src = new Path(renameDir, "src" + i);
+        Path dst = new Path(renameDir, "dst" + i);
+        DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
+        DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
+        Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+        returnFutures.put(i, returnFuture);
+      }
+
+      for (int i = 0; i < 5; i++) {
+        verifyCallGetReturnValueMultipleTimes(returnFutures, count, cluster,
+            renameDir, dfs);
+      }
+    } finally {
+      if (dfs != null) {
+        dfs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private void verifyCallGetReturnValueMultipleTimes(
+      Map<Integer, Future<Void>> returnFutures, int count,
+      MiniDFSCluster cluster, Path renameDir, DistributedFileSystem dfs)
+      throws InterruptedException, ExecutionException, IOException {
+    // wait for completing the calls
+    for (int i = 0; i < count; i++) {
+      returnFutures.get(i).get();
+    }
+
+    // Restart NN and check the rename successfully
+    cluster.restartNameNodes();
+
+    // very the src dir should not exist, dst should
+    for (int i = 0; i < count; i++) {
+      Path src = new Path(renameDir, "src" + i);
+      Path dst = new Path(renameDir, "dst" + i);
+      assertFalse(dfs.exists(src));
+      assertTrue(dfs.exists(dst));
+    }
+  }
+
+  @Test(timeout = 120000)
+  public void testAggressiveConcurrentAsyncRenameWithOverwrite()
+      throws Exception {
+    internalTestConcurrentAsyncRenameWithOverwrite(100,
+        "testAggressiveConcurrentAsyncRenameWithOverwrite");
+  }
+
+  @Test(timeout = 60000)
+  public void testConservativeConcurrentAsyncRenameWithOverwrite()
+      throws Exception {
+    internalTestConcurrentAsyncRenameWithOverwrite(10000,
+        "testConservativeConcurrentAsyncRenameWithOverwrite");
+  }
+
+  private void internalTestConcurrentAsyncRenameWithOverwrite(
+      final int asyncCallLimit, final String basePath) throws Exception {
+    final short replFactor = 2;
+    final long blockSize = 512;
+    final Path renameDir = new Path(String.format("/test/%s/", basePath));
     Configuration conf = new HdfsConfiguration();
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
+        asyncCallLimit);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
         .build();
     cluster.waitActive();
     DistributedFileSystem dfs = cluster.getFileSystem();
     AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
     int count = 1000;
+    long fileLen = blockSize * 3;
+    int start = 0, end = 0;
+    Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
 
-    try {
-      long fileLen = blockSize * 3;
-      assertTrue(dfs.mkdirs(renameDir));
-
-      Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
+    assertTrue(dfs.mkdirs(renameDir));
 
+    try {
       // concurrently invoking many rename
       for (int i = 0; i < count; i++) {
         Path src = new Path(renameDir, "src" + i);
         Path dst = new Path(renameDir, "dst" + i);
         DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
         DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
-        Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-        returnFutures.put(i, returnFuture);
+        for (;;) {
+          try {
+            LOG.info("rename #" + i);
+            Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+            returnFutures.put(i, returnFuture);
+            break;
+          } catch (AsyncCallLimitExceededException e) {
+            /**
+             * reached limit of async calls, fetch results of finished async
+             * calls to let follow-on calls go
+             */
+            LOG.error(e);
+            start = end;
+            end = i;
+            LOG.info(String.format("start=%d, end=%d, i=%d", start, end, i));
+            waitForReturnValues(returnFutures, start, end);
+          }
+        }
       }
 
       // wait for completing the calls
-      for (int i = 0; i < count; i++) {
+      for (int i = start; i < count; i++) {
         returnFutures.get(i).get();
       }
 
@@ -215,26 +255,60 @@ public class TestAsyncDFSRename {
         assertTrue(dfs.exists(dst));
       }
     } finally {
-      dfs.delete(renameDir, true);
+      if (dfs != null) {
+        dfs.close();
+      }
       if (cluster != null) {
         cluster.shutdown();
       }
     }
   }
 
-  @Test
+  private void waitForReturnValues(
+      final Map<Integer, Future<Void>> returnFutures, final int start,
+      final int end) throws InterruptedException, ExecutionException {
+    LOG.info(String.format("calling waitForReturnValues [%d, %d)", start, end));
+    for (int i = start; i < end; i++) {
+      LOG.info("calling Future#get #" + i);
+      returnFutures.get(i).get();
+    }
+  }
+
+  @Test(timeout = 60000)
   public void testAsyncRenameWithException() throws Exception {
-    FileSystem rootFs = FileSystem.get(CONF);
+    Configuration conf = new HdfsConfiguration();
+    String group1 = "group1";
+    String group2 = "group2";
+    String user1 = "user1";
+    UserGroupInformation ugi1;
+
+    // explicitly turn on permission checking
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+
+    // create fake mapping for the groups
+    Map<String, String[]> u2g_map = new HashMap<String, String[]>(1);
+    u2g_map.put(user1, new String[] { group1, group2 });
+    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
+
+    // Initiate all four users
+    ugi1 = UserGroupInformation.createUserForTesting(user1, new String[] {
+        group1, group2 });
+
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3).build();
+    cluster.waitActive();
+
+    FileSystem rootFs = FileSystem.get(conf);
     final Path renameDir = new Path("/test/async_rename_exception/");
     final Path src = new Path(renameDir, "src");
     final Path dst = new Path(renameDir, "dst");
     rootFs.mkdirs(src);
 
-    AsyncDistributedFileSystem adfs = USER1
+    AsyncDistributedFileSystem adfs = ugi1
         .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
           @Override
           public AsyncDistributedFileSystem run() throws Exception {
-            return gCluster.getFileSystem().getAsyncDistributedFileSystem();
+            return cluster.getFileSystem().getAsyncDistributedFileSystem();
           }
         });
 
@@ -242,16 +316,24 @@ public class TestAsyncDFSRename {
       Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
       returnFuture.get();
     } catch (ExecutionException e) {
-      checkPermissionDenied(e, src);
+      checkPermissionDenied(e, src, user1);
+    } finally {
+      if (rootFs != null) {
+        rootFs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 
-  private void checkPermissionDenied(final Exception e, final Path dir) {
+  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(USER1_NAME));
+        .getMessage().contains(user));
     assertTrue("Permission denied messages must carry the path parent", e
         .getMessage().contains(dir.getParent().toUri().getPath()));
   }


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


[15/47] hadoop git commit: YARN-5098. Fixed ResourceManager's DelegationTokenRenewer to replace expiring system-tokens if RM stops and only restarts after a long time. Contributed by Jian He.

Posted by ae...@apache.org.
YARN-5098. Fixed ResourceManager's DelegationTokenRenewer to replace expiring system-tokens if RM stops and only restarts after a long time. Contributed by Jian He.


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

Branch: refs/heads/HDFS-1312
Commit: f10ebc67f57a4a2e3cc916c41154ab9b6a4635c9
Parents: 99cc439
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Fri Jun 3 13:00:07 2016 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Fri Jun 3 13:00:07 2016 -0700

----------------------------------------------------------------------
 .../security/DelegationTokenRenewer.java        | 27 ++++--
 .../security/TestDelegationTokenRenewer.java    | 98 ++++++++++++++++++++
 2 files changed, 118 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f10ebc67/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index fd12f11..4177ee2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.service.AbstractService;
@@ -459,6 +460,18 @@ public class DelegationTokenRenewer extends AbstractService {
           try {
             renewToken(dttr);
           } catch (IOException ioe) {
+            if (ioe instanceof SecretManager.InvalidToken
+                && dttr.maxDate < Time.now()
+                && evt instanceof DelegationTokenRenewerAppRecoverEvent
+                && token.getKind().equals(HDFS_DELEGATION_KIND)) {
+              LOG.info("Failed to renew hdfs token " + dttr
+                  + " on recovery as it expired, requesting new hdfs token for "
+                  + applicationId + ", user=" + evt.getUser(), ioe);
+              requestNewHdfsDelegationTokenAsProxyUser(
+                  Arrays.asList(applicationId), evt.getUser(),
+                  evt.shouldCancelAtEnd());
+              continue;
+            }
             throw new IOException("Failed to renew token: " + dttr.token, ioe);
           }
         }
@@ -485,7 +498,8 @@ public class DelegationTokenRenewer extends AbstractService {
     }
 
     if (!hasHdfsToken) {
-      requestNewHdfsDelegationToken(Arrays.asList(applicationId), evt.getUser(),
+      requestNewHdfsDelegationTokenAsProxyUser(Arrays.asList(applicationId),
+          evt.getUser(),
         shouldCancelAtEnd);
     }
   }
@@ -586,8 +600,7 @@ public class DelegationTokenRenewer extends AbstractService {
     } catch (InterruptedException e) {
       throw new IOException(e);
     }
-    LOG.info("Renewed delegation-token= [" + dttr + "], for "
-        + dttr.referringAppIds);
+    LOG.info("Renewed delegation-token= [" + dttr + "]");
   }
 
   // Request new hdfs token if the token is about to expire, and remove the old
@@ -625,12 +638,12 @@ public class DelegationTokenRenewer extends AbstractService {
         }
       }
       LOG.info("Token= (" + dttr + ") is expiring, request new token.");
-      requestNewHdfsDelegationToken(applicationIds, dttr.user,
+      requestNewHdfsDelegationTokenAsProxyUser(applicationIds, dttr.user,
           dttr.shouldCancelAtEnd);
     }
   }
 
-  private void requestNewHdfsDelegationToken(
+  private void requestNewHdfsDelegationTokenAsProxyUser(
       Collection<ApplicationId> referringAppIds,
       String user, boolean shouldCancelAtEnd) throws IOException,
       InterruptedException {
@@ -912,8 +925,8 @@ public class DelegationTokenRenewer extends AbstractService {
       // Setup tokens for renewal during recovery
       DelegationTokenRenewer.this.handleAppSubmitEvent(event);
     } catch (Throwable t) {
-      LOG.warn(
-          "Unable to add the application to the delegation token renewer.", t);
+      LOG.warn("Unable to add the application to the delegation token"
+          + " renewer on recovery.", t);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f10ebc67/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
index 1bfac8d..74fe534 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
@@ -43,6 +43,7 @@ import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
@@ -84,6 +85,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -968,6 +970,101 @@ public class TestDelegationTokenRenewer {
     Assert.assertTrue(appCredentials.getAllTokens().contains(expectedToken));
   }
 
+
+  // 1. token is expired before app completes.
+  // 2. RM shutdown.
+  // 3. When RM recovers the app, token renewal will fail as token expired.
+  //    RM should request a new token and sent it to NM for log-aggregation.
+  @Test
+  public void testRMRestartWithExpiredToken() throws Exception {
+    Configuration yarnConf = new YarnConfiguration();
+    yarnConf
+        .setBoolean(YarnConfiguration.RM_PROXY_USER_PRIVILEGES_ENABLED, true);
+    yarnConf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        "kerberos");
+    yarnConf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    yarnConf
+        .set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    UserGroupInformation.setConfiguration(yarnConf);
+
+    // create Token1:
+    Text userText1 = new Text("user1");
+    DelegationTokenIdentifier dtId1 = new DelegationTokenIdentifier(userText1,
+        new Text("renewer1"), userText1);
+    final Token<DelegationTokenIdentifier> originalToken =
+        new Token<>(dtId1.getBytes(), "password1".getBytes(), dtId1.getKind(),
+            new Text("service1"));
+    Credentials credentials = new Credentials();
+    credentials.addToken(userText1, originalToken);
+
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(yarnConf);
+    MockRM rm1 = new TestSecurityMockRM(yarnConf, memStore);
+    rm1.start();
+    RMApp app = rm1.submitApp(200, "name", "user",
+        new HashMap<ApplicationAccessType, String>(), false, "default", 1,
+        credentials);
+
+    // create token2
+    Text userText2 = new Text("user1");
+    DelegationTokenIdentifier dtId2 =
+        new DelegationTokenIdentifier(userText1, new Text("renewer2"),
+            userText2);
+    final Token<DelegationTokenIdentifier> updatedToken =
+        new Token<DelegationTokenIdentifier>(dtId2.getBytes(),
+            "password2".getBytes(), dtId2.getKind(), new Text("service2"));
+    AtomicBoolean firstRenewInvoked = new AtomicBoolean(false);
+    AtomicBoolean secondRenewInvoked = new AtomicBoolean(false);
+    MockRM rm2 = new TestSecurityMockRM(yarnConf, memStore) {
+      @Override
+      protected DelegationTokenRenewer createDelegationTokenRenewer() {
+        return new DelegationTokenRenewer() {
+
+          @Override
+          protected void renewToken(final DelegationTokenToRenew dttr)
+              throws IOException {
+
+            if (dttr.token.equals(updatedToken)) {
+              secondRenewInvoked.set(true);
+              super.renewToken(dttr);
+            } else if (dttr.token.equals(originalToken)){
+              firstRenewInvoked.set(true);
+              throw new InvalidToken("Failed to renew");
+            } else {
+              throw new IOException("Unexpected");
+            }
+          }
+
+          @Override
+          protected Token<?>[] obtainSystemTokensForUser(String user,
+              final Credentials credentials) throws IOException {
+            credentials.addToken(updatedToken.getService(), updatedToken);
+            return new Token<?>[] { updatedToken };
+          }
+        };
+      }
+    };
+
+    // simulating restart the rm
+    rm2.start();
+
+    // check nm can retrieve the token
+    final MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 15120, rm2.getResourceTrackerService());
+    nm1.registerNode();
+    NodeHeartbeatResponse response = nm1.nodeHeartbeat(true);
+    ByteBuffer tokenBuffer =
+        response.getSystemCredentialsForApps().get(app.getApplicationId());
+    Assert.assertNotNull(tokenBuffer);
+    Credentials appCredentials = new Credentials();
+    DataInputByteBuffer buf = new DataInputByteBuffer();
+    tokenBuffer.rewind();
+    buf.reset(tokenBuffer);
+    appCredentials.readTokenStorageStream(buf);
+    Assert.assertTrue(firstRenewInvoked.get() && secondRenewInvoked.get());
+    Assert.assertTrue(appCredentials.getAllTokens().contains(updatedToken));
+  }
+
   // YARN will get the token for the app submitted without the delegation token.
   @Test
   public void testAppSubmissionWithoutDelegationToken() throws Exception {
@@ -1158,4 +1255,5 @@ public class TestDelegationTokenRenewer {
     Assert.assertTrue(dttr.isTimerCancelled());
     Assert.assertTrue(Renewer.cancelled);
   }
+
 }


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


[10/47] hadoop git commit: HDFS-10341. Add a metric to expose the timeout number of pending replication blocks. (Contributed by Akira Ajisaka)

Posted by ae...@apache.org.
HDFS-10341. Add a metric to expose the timeout number of pending replication blocks. (Contributed by Akira Ajisaka)


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

Branch: refs/heads/HDFS-1312
Commit: 97e244947719d483c3f80521a00fec8e13dcb637
Parents: 1df6f57
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jun 2 13:14:45 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 2 13:14:45 2016 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md  |  1 +
 .../server/blockmanagement/BlockManager.java    |  4 ++++
 .../PendingReconstructionBlocks.java            | 16 +++++++++++++++-
 .../hdfs/server/namenode/FSNamesystem.java      |  5 +++++
 .../TestPendingReconstruction.java              | 20 ++++++++++++++------
 5 files changed, 39 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/97e24494/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index 699316f..e4e2443 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -218,6 +218,7 @@ Each metrics record contains tags such as HAState and Hostname as additional inf
 | `TotalSyncCount` | Total number of sync operations performed by edit log |
 | `TotalSyncTimes` | Total number of milliseconds spent by various edit logs in sync operation|
 | `NameDirSize` | NameNode name directories size in bytes |
+| `NumTimedOutPendingReconstructions` | The number of timed out reconstructions. Not the number of unique blocks that timed out. |
 
 JournalNode
 -----------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97e24494/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index ed57a86..1a76e09 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -198,6 +198,10 @@ public class BlockManager implements BlockStatsMXBean {
   public int getPendingDataNodeMessageCount() {
     return pendingDNMessages.count();
   }
+  /** Used by metrics. */
+  public long getNumTimedOutPendingReconstructions() {
+    return pendingReconstruction.getNumTimedOuts();
+  }
 
   /**replicationRecheckInterval is how often namenode checks for new replication work*/
   private final long replicationRecheckInterval;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97e24494/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java
index 528199c..956e94f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReconstructionBlocks.java
@@ -50,6 +50,7 @@ class PendingReconstructionBlocks {
   private final ArrayList<BlockInfo> timedOutItems;
   Daemon timerThread = null;
   private volatile boolean fsRunning = true;
+  private long timedOutCount = 0L;
 
   //
   // It might take anywhere between 5 to 10 minutes before
@@ -125,6 +126,7 @@ class PendingReconstructionBlocks {
     synchronized (pendingReconstructions) {
       pendingReconstructions.clear();
       timedOutItems.clear();
+      timedOutCount = 0L;
     }
   }
 
@@ -149,6 +151,16 @@ class PendingReconstructionBlocks {
   }
 
   /**
+   * Used for metrics.
+   * @return The number of timeouts
+   */
+  long getNumTimedOuts() {
+    synchronized (timedOutItems) {
+      return timedOutCount + timedOutItems.size();
+    }
+  }
+
+  /**
    * Returns a list of blocks that have timed out their
    * reconstruction requests. Returns null if no blocks have
    * timed out.
@@ -158,9 +170,11 @@ class PendingReconstructionBlocks {
       if (timedOutItems.size() <= 0) {
         return null;
       }
+      int size = timedOutItems.size();
       BlockInfo[] blockList = timedOutItems.toArray(
-          new BlockInfo[timedOutItems.size()]);
+          new BlockInfo[size]);
       timedOutItems.clear();
+      timedOutCount += size;
       return blockList;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97e24494/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 a2df6d2..c9f2487 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
@@ -4507,6 +4507,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   public long getExcessBlocks() {
     return blockManager.getExcessBlocksCount();
   }
+
+  @Metric
+  public long getNumTimedOutPendingReconstructions() {
+    return blockManager.getNumTimedOutPendingReconstructions();
+  }
   
   // HA-only metric
   @Metric

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97e24494/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReconstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReconstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReconstruction.java
index d07c657..c30f630 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReconstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReconstruction.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
@@ -117,14 +119,15 @@ public class TestPendingReconstruction {
     //
     // verify that nothing has timed out so far
     //
-    assertTrue(pendingReconstructions.getTimedOutBlocks() == null);
+    assertNull(pendingReconstructions.getTimedOutBlocks());
+    assertEquals(0L, pendingReconstructions.getNumTimedOuts());
 
     //
     // Wait for one second and then insert some more items.
     //
     try {
       Thread.sleep(1000);
-    } catch (Exception e) {
+    } catch (Exception ignored) {
     }
 
     for (int i = 10; i < 15; i++) {
@@ -133,7 +136,8 @@ public class TestPendingReconstruction {
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(i)));
     }
-    assertTrue(pendingReconstructions.size() == 15);
+    assertEquals(15, pendingReconstructions.size());
+    assertEquals(0L, pendingReconstructions.getNumTimedOuts());
 
     //
     // Wait for everything to timeout.
@@ -153,10 +157,14 @@ public class TestPendingReconstruction {
     // Verify that everything has timed out.
     //
     assertEquals("Size of pendingReconstructions ", 0, pendingReconstructions.size());
+    assertEquals(15L, pendingReconstructions.getNumTimedOuts());
     Block[] timedOut = pendingReconstructions.getTimedOutBlocks();
-    assertTrue(timedOut != null && timedOut.length == 15);
-    for (int i = 0; i < timedOut.length; i++) {
-      assertTrue(timedOut[i].getBlockId() < 15);
+    assertNotNull(timedOut);
+    assertEquals(15, timedOut.length);
+    // Verify the number is not reset
+    assertEquals(15L, pendingReconstructions.getNumTimedOuts());
+    for (Block block : timedOut) {
+      assertTrue(block.getBlockId() < 15);
     }
     pendingReconstructions.stop();
   }


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


[22/47] hadoop git commit: HDFS-10481. HTTPFS server should correctly impersonate as end user to open file. Contributed by Xiao Chen.

Posted by ae...@apache.org.
HDFS-10481. HTTPFS server should correctly impersonate as end user to open file. Contributed by Xiao Chen.


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

Branch: refs/heads/HDFS-1312
Commit: 47e0321ee91149331e6ae72e7caa41d1de078b6c
Parents: 99a771c
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 17:21:17 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 17:21:17 2016 -0700

----------------------------------------------------------------------
 .../hadoop/fs/http/server/HttpFSServer.java     | 218 ++++++++++---------
 1 file changed, 114 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47e0321e/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
index b7b63fa..db4692a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
@@ -79,6 +79,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.security.AccessControlException;
+import java.security.PrivilegedExceptionAction;
 import java.text.MessageFormat;
 import java.util.EnumSet;
 import java.util.List;
@@ -94,6 +95,7 @@ import java.util.Map;
 @InterfaceAudience.Private
 public class HttpFSServer {
   private static Logger AUDIT_LOG = LoggerFactory.getLogger("httpfsaudit");
+  private static final Logger LOG = LoggerFactory.getLogger(HttpFSServer.class);
 
   /**
    * Executes a {@link FileSystemAccess.FileSystemExecutor} using a filesystem for the effective
@@ -205,115 +207,123 @@ public class HttpFSServer {
     MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
     MDC.put("hostname", request.getRemoteAddr());
     switch (op.value()) {
-      case OPEN: {
-        //Invoking the command directly using an unmanaged FileSystem that is
-        // released by the FileSystemReleaseFilter
-        FSOperations.FSOpen command = new FSOperations.FSOpen(path);
-        FileSystem fs = createFileSystem(user);
-        InputStream is = command.execute(fs);
-        Long offset = params.get(OffsetParam.NAME, OffsetParam.class);
-        Long len = params.get(LenParam.NAME, LenParam.class);
-        AUDIT_LOG.info("[{}] offset [{}] len [{}]",
-                       new Object[]{path, offset, len});
-        InputStreamEntity entity = new InputStreamEntity(is, offset, len);
-        response =
+    case OPEN: {
+      //Invoking the command directly using an unmanaged FileSystem that is
+      // released by the FileSystemReleaseFilter
+      final FSOperations.FSOpen command = new FSOperations.FSOpen(path);
+      final FileSystem fs = createFileSystem(user);
+      InputStream is = null;
+      UserGroupInformation ugi = UserGroupInformation
+          .createProxyUser(user.getShortUserName(),
+              UserGroupInformation.getLoginUser());
+      try {
+        is = ugi.doAs(new PrivilegedExceptionAction<InputStream>() {
+          @Override
+          public InputStream run() throws Exception {
+            return command.execute(fs);
+          }
+        });
+      } catch (InterruptedException ie) {
+        LOG.info("Open interrupted.", ie);
+        Thread.currentThread().interrupt();
+      }
+      Long offset = params.get(OffsetParam.NAME, OffsetParam.class);
+      Long len = params.get(LenParam.NAME, LenParam.class);
+      AUDIT_LOG.info("[{}] offset [{}] len [{}]",
+          new Object[] { path, offset, len });
+      InputStreamEntity entity = new InputStreamEntity(is, offset, len);
+      response =
           Response.ok(entity).type(MediaType.APPLICATION_OCTET_STREAM).build();
-        break;
-      }
-      case GETFILESTATUS: {
-        FSOperations.FSFileStatus command =
-          new FSOperations.FSFileStatus(path);
-        Map json = fsExecute(user, command);
-        AUDIT_LOG.info("[{}]", path);
-        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
-        break;
-      }
-      case LISTSTATUS: {
-        String filter = params.get(FilterParam.NAME, FilterParam.class);
-        FSOperations.FSListStatus command = new FSOperations.FSListStatus(
-          path, filter);
-        Map json = fsExecute(user, command);
-        AUDIT_LOG.info("[{}] filter [{}]", path,
-                       (filter != null) ? filter : "-");
-        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
-        break;
-      }
-      case GETHOMEDIRECTORY: {
-        enforceRootPath(op.value(), path);
-        FSOperations.FSHomeDir command = new FSOperations.FSHomeDir();
-        JSONObject json = fsExecute(user, command);
-        AUDIT_LOG.info("");
-        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
-        break;
-      }
-      case INSTRUMENTATION: {
-        enforceRootPath(op.value(), path);
-        Groups groups = HttpFSServerWebApp.get().get(Groups.class);
-        List<String> userGroups = groups.getGroups(user.getShortUserName());
-        if (!userGroups.contains(HttpFSServerWebApp.get().getAdminGroup())) {
-          throw new AccessControlException(
+      break;
+    }
+    case GETFILESTATUS: {
+      FSOperations.FSFileStatus command = new FSOperations.FSFileStatus(path);
+      Map json = fsExecute(user, command);
+      AUDIT_LOG.info("[{}]", path);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    case LISTSTATUS: {
+      String filter = params.get(FilterParam.NAME, FilterParam.class);
+      FSOperations.FSListStatus command =
+          new FSOperations.FSListStatus(path, filter);
+      Map json = fsExecute(user, command);
+      AUDIT_LOG.info("[{}] filter [{}]", path, (filter != null) ? filter : "-");
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    case GETHOMEDIRECTORY: {
+      enforceRootPath(op.value(), path);
+      FSOperations.FSHomeDir command = new FSOperations.FSHomeDir();
+      JSONObject json = fsExecute(user, command);
+      AUDIT_LOG.info("");
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    case INSTRUMENTATION: {
+      enforceRootPath(op.value(), path);
+      Groups groups = HttpFSServerWebApp.get().get(Groups.class);
+      List<String> userGroups = groups.getGroups(user.getShortUserName());
+      if (!userGroups.contains(HttpFSServerWebApp.get().getAdminGroup())) {
+        throw new AccessControlException(
             "User not in HttpFSServer admin group");
-        }
-        Instrumentation instrumentation =
-          HttpFSServerWebApp.get().get(Instrumentation.class);
-        Map snapshot = instrumentation.getSnapshot();
-        response = Response.ok(snapshot).build();
-        break;
       }
-      case GETCONTENTSUMMARY: {
-        FSOperations.FSContentSummary command =
+      Instrumentation instrumentation =
+          HttpFSServerWebApp.get().get(Instrumentation.class);
+      Map snapshot = instrumentation.getSnapshot();
+      response = Response.ok(snapshot).build();
+      break;
+    }
+    case GETCONTENTSUMMARY: {
+      FSOperations.FSContentSummary command =
           new FSOperations.FSContentSummary(path);
-        Map json = fsExecute(user, command);
-        AUDIT_LOG.info("[{}]", path);
-        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
-        break;
-      }
-      case GETFILECHECKSUM: {
-        FSOperations.FSFileChecksum command =
+      Map json = fsExecute(user, command);
+      AUDIT_LOG.info("[{}]", path);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    case GETFILECHECKSUM: {
+      FSOperations.FSFileChecksum command =
           new FSOperations.FSFileChecksum(path);
-        Map json = fsExecute(user, command);
-        AUDIT_LOG.info("[{}]", path);
-        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
-        break;
-      }
-      case GETFILEBLOCKLOCATIONS: {
-        response = Response.status(Response.Status.BAD_REQUEST).build();
-        break;
-      }
-      case GETACLSTATUS: {
-        FSOperations.FSAclStatus command =
-                new FSOperations.FSAclStatus(path);
-        Map json = fsExecute(user, command);
-        AUDIT_LOG.info("ACL status for [{}]", path);
-        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
-        break;
-      }
-      case GETXATTRS: {
-        List<String> xattrNames = params.getValues(XAttrNameParam.NAME, 
-            XAttrNameParam.class);
-        XAttrCodec encoding = params.get(XAttrEncodingParam.NAME, 
-            XAttrEncodingParam.class);
-        FSOperations.FSGetXAttrs command = new FSOperations.FSGetXAttrs(path, 
-            xattrNames, encoding);
-        @SuppressWarnings("rawtypes")
-        Map json = fsExecute(user, command);
-        AUDIT_LOG.info("XAttrs for [{}]", path);
-        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
-        break;
-      }
-      case LISTXATTRS: {
-        FSOperations.FSListXAttrs command = new FSOperations.FSListXAttrs(path);
-        @SuppressWarnings("rawtypes")
-        Map json = fsExecute(user, command);
-        AUDIT_LOG.info("XAttr names for [{}]", path);
-        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
-        break;
-      }
-      default: {
-        throw new IOException(
-          MessageFormat.format("Invalid HTTP GET operation [{0}]",
-                               op.value()));
-      }
+      Map json = fsExecute(user, command);
+      AUDIT_LOG.info("[{}]", path);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    case GETFILEBLOCKLOCATIONS: {
+      response = Response.status(Response.Status.BAD_REQUEST).build();
+      break;
+    }
+    case GETACLSTATUS: {
+      FSOperations.FSAclStatus command = new FSOperations.FSAclStatus(path);
+      Map json = fsExecute(user, command);
+      AUDIT_LOG.info("ACL status for [{}]", path);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    case GETXATTRS: {
+      List<String> xattrNames =
+          params.getValues(XAttrNameParam.NAME, XAttrNameParam.class);
+      XAttrCodec encoding =
+          params.get(XAttrEncodingParam.NAME, XAttrEncodingParam.class);
+      FSOperations.FSGetXAttrs command =
+          new FSOperations.FSGetXAttrs(path, xattrNames, encoding);
+      @SuppressWarnings("rawtypes") Map json = fsExecute(user, command);
+      AUDIT_LOG.info("XAttrs for [{}]", path);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    case LISTXATTRS: {
+      FSOperations.FSListXAttrs command = new FSOperations.FSListXAttrs(path);
+      @SuppressWarnings("rawtypes") Map json = fsExecute(user, command);
+      AUDIT_LOG.info("XAttr names for [{}]", path);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
+    default: {
+      throw new IOException(
+          MessageFormat.format("Invalid HTTP GET operation [{0}]", op.value()));
+    }
     }
     return response;
   }


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


[18/47] hadoop git commit: HADOOP-13105. Support timeouts in LDAP queries in LdapGroupsMapping. Contributed by Mingliang Liu.

Posted by ae...@apache.org.
HADOOP-13105. Support timeouts in LDAP queries in LdapGroupsMapping. Contributed by Mingliang Liu.


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

Branch: refs/heads/HDFS-1312
Commit: d82bc8501869be78780fc09752dbf7af918c14af
Parents: 78b3a03
Author: Chris Nauroth <cn...@apache.org>
Authored: Fri Jun 3 16:38:30 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Fri Jun 3 16:38:30 2016 -0700

----------------------------------------------------------------------
 .../hadoop/security/LdapGroupsMapping.java      |  12 ++
 .../src/main/resources/core-default.xml         |  24 ++++
 .../hadoop/security/TestLdapGroupsMapping.java  | 140 +++++++++++++++++++
 3 files changed, 176 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d82bc850/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
index 498b92e..da87369 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
@@ -179,6 +179,13 @@ public class LdapGroupsMapping
     LDAP_CONFIG_PREFIX + ".directory.search.timeout";
   public static final int DIRECTORY_SEARCH_TIMEOUT_DEFAULT = 10000; // 10s
 
+  public static final String CONNECTION_TIMEOUT =
+      LDAP_CONFIG_PREFIX + ".connection.timeout.ms";
+  public static final int CONNECTION_TIMEOUT_DEFAULT = 60 * 1000; // 60 seconds
+  public static final String READ_TIMEOUT =
+      LDAP_CONFIG_PREFIX + ".read.timeout.ms";
+  public static final int READ_TIMEOUT_DEFAULT = 60 * 1000; // 60 seconds
+
   private static final Log LOG = LogFactory.getLog(LdapGroupsMapping.class);
 
   private static final SearchControls SEARCH_CONTROLS = new SearchControls();
@@ -432,6 +439,11 @@ public class LdapGroupsMapping
       env.put(Context.SECURITY_PRINCIPAL, bindUser);
       env.put(Context.SECURITY_CREDENTIALS, bindPassword);
 
+      env.put("com.sun.jndi.ldap.connect.timeout", conf.get(CONNECTION_TIMEOUT,
+          String.valueOf(CONNECTION_TIMEOUT_DEFAULT)));
+      env.put("com.sun.jndi.ldap.read.timeout", conf.get(READ_TIMEOUT,
+          String.valueOf(READ_TIMEOUT_DEFAULT)));
+
       ctx = new InitialDirContext(env);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d82bc850/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index b3f8cd5..a65246b 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -166,6 +166,30 @@
 </property>
 
 <property>
+  <name>hadoop.security.group.mapping.ldap.connection.timeout.ms</name>
+  <value>60000</value>
+  <description>
+    This property is the connection timeout (in milliseconds) for LDAP
+    operations. If the LDAP provider doesn't establish a connection within the
+    specified period, it will abort the connect attempt. Non-positive value
+    means no LDAP connection timeout is specified in which case it waits for the
+    connection to establish until the underlying network times out.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.read.timeout.ms</name>
+  <value>60000</value>
+  <description>
+    This property is the read timeout (in milliseconds) for LDAP
+    operations. If the LDAP provider doesn't get a LDAP response within the
+    specified period, it will abort the read attempt. Non-positive value
+    means no read timeout is specified in which case it waits for the response
+    infinitely.
+  </description>
+</property>
+
+<property>
   <name>hadoop.security.group.mapping.ldap.url</name>
   <value></value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d82bc850/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
index 9319016..9f9f994 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
@@ -17,8 +17,13 @@
  */
 package org.apache.hadoop.security;
 
+import static org.apache.hadoop.security.LdapGroupsMapping.CONNECTION_TIMEOUT;
+import static org.apache.hadoop.security.LdapGroupsMapping.READ_TIMEOUT;
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.times;
@@ -29,8 +34,11 @@ import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.Writer;
+import java.net.ServerSocket;
+import java.net.Socket;
 import java.util.Arrays;
 import java.util.List;
+import java.util.concurrent.CountDownLatch;
 
 import javax.naming.CommunicationException;
 import javax.naming.NamingException;
@@ -38,16 +46,38 @@ import javax.naming.directory.SearchControls;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
 import org.apache.hadoop.test.GenericTestUtils;
+
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 @SuppressWarnings("unchecked")
 public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      TestLdapGroupsMapping.class);
+
+  /**
+   * To construct a LDAP InitialDirContext object, it will firstly initiate a
+   * protocol session to server for authentication. After a session is
+   * established, a method of authentication is negotiated between the server
+   * and the client. When the client is authenticated, the LDAP server will send
+   * a bind response, whose message contents are bytes as the
+   * {@link #AUTHENTICATE_SUCCESS_MSG}. After receiving this bind response
+   * message, the LDAP context is considered connected to the server and thus
+   * can issue query requests for determining group membership.
+   */
+  private static final byte[] AUTHENTICATE_SUCCESS_MSG =
+      {48, 12, 2, 1, 1, 97, 7, 10, 1, 0, 4, 0, 4, 0};
+
   @Before
   public void setupMocks() throws NamingException {
     when(getUserSearchResult().getNameInNamespace()).
@@ -176,4 +206,114 @@ public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
     // extract password
     Assert.assertEquals("", mapping.getPassword(conf,"invalid-alias", ""));
   }
+
+  /**
+   * Test that if the {@link LdapGroupsMapping#CONNECTION_TIMEOUT} is set in the
+   * configuration, the LdapGroupsMapping connection will timeout by this value
+   * if it does not get a LDAP response from the server.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test (timeout = 30000)
+  public void testLdapConnectionTimeout()
+      throws IOException, InterruptedException {
+    final int connectionTimeoutMs = 3 * 1000; // 3s
+    try (ServerSocket serverSock = new ServerSocket(0)) {
+      final CountDownLatch finLatch = new CountDownLatch(1);
+
+      // Below we create a LDAP server which will accept a client request;
+      // but it will never reply to the bind (connect) request.
+      // Client of this LDAP server is expected to get a connection timeout.
+      final Thread ldapServer = new Thread(new Runnable() {
+        @Override
+        public void run() {
+          try {
+            try (Socket ignored = serverSock.accept()) {
+              finLatch.await();
+            }
+          } catch (Exception e) {
+            e.printStackTrace();
+          }
+        }
+      });
+      ldapServer.start();
+
+      final LdapGroupsMapping mapping = new LdapGroupsMapping();
+      final Configuration conf = new Configuration();
+      conf.set(LdapGroupsMapping.LDAP_URL_KEY,
+          "ldap://localhost:" + serverSock.getLocalPort());
+      conf.setInt(CONNECTION_TIMEOUT, connectionTimeoutMs);
+      mapping.setConf(conf);
+
+      try {
+        mapping.doGetGroups("hadoop");
+        fail("The LDAP query should have timed out!");
+      } catch (NamingException ne) {
+        LOG.debug("Got the exception while LDAP querying: ", ne);
+        assertExceptionContains("LDAP response read timed out, timeout used:" +
+            connectionTimeoutMs + "ms", ne);
+        assertFalse(ne.getMessage().contains("remaining name"));
+      } finally {
+        finLatch.countDown();
+      }
+      ldapServer.join();
+    }
+  }
+
+  /**
+   * Test that if the {@link LdapGroupsMapping#READ_TIMEOUT} is set in the
+   * configuration, the LdapGroupsMapping query will timeout by this value if
+   * it does not get a LDAP response from the server.
+   *
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test(timeout = 30000)
+  public void testLdapReadTimeout() throws IOException, InterruptedException {
+    final int readTimeoutMs = 4 * 1000; // 4s
+    try (ServerSocket serverSock = new ServerSocket(0)) {
+      final CountDownLatch finLatch = new CountDownLatch(1);
+
+      // Below we create a LDAP server which will accept a client request,
+      // authenticate it successfully; but it will never reply to the following
+      // query request.
+      // Client of this LDAP server is expected to get a read timeout.
+      final Thread ldapServer = new Thread(new Runnable() {
+        @Override
+        public void run() {
+          try {
+            try (Socket clientSock = serverSock.accept()) {
+              IOUtils.skipFully(clientSock.getInputStream(), 1);
+              clientSock.getOutputStream().write(AUTHENTICATE_SUCCESS_MSG);
+              finLatch.await();
+            }
+          } catch (Exception e) {
+            e.printStackTrace();
+          }
+        }
+      });
+      ldapServer.start();
+
+      final LdapGroupsMapping mapping = new LdapGroupsMapping();
+      final Configuration conf = new Configuration();
+      conf.set(LdapGroupsMapping.LDAP_URL_KEY,
+          "ldap://localhost:" + serverSock.getLocalPort());
+      conf.setInt(READ_TIMEOUT, readTimeoutMs);
+      mapping.setConf(conf);
+
+      try {
+        mapping.doGetGroups("hadoop");
+        fail("The LDAP query should have timed out!");
+      } catch (NamingException ne) {
+        LOG.debug("Got the exception while LDAP querying: ", ne);
+        assertExceptionContains("LDAP response read timed out, timeout used:" +
+            readTimeoutMs + "ms", ne);
+        assertExceptionContains("remaining name", ne);
+      } finally {
+        finLatch.countDown();
+      }
+      ldapServer.join();
+    }
+  }
+
 }


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


[11/47] hadoop git commit: HADOOP-13171. Add StorageStatistics to S3A; instrument some more operations. Contributed by Steve Loughran.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
new file mode 100644
index 0000000..d29cb2f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -0,0 +1,143 @@
+/*
+ * 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.fs.s3a;
+
+/**
+ * Statistic which are collected in S3A.
+ * These statistics are available at a low level in {@link S3AStorageStatistics}
+ * and as metrics in {@link S3AInstrumentation}
+ */
+public enum Statistic {
+
+  DIRECTORIES_CREATED("directories_created",
+      "Total number of directories created through the object store."),
+  DIRECTORIES_DELETED("directories_deleted",
+      "Total number of directories deleted through the object store."),
+  FILES_COPIED("files_copied",
+      "Total number of files copied within the object store."),
+  FILES_COPIED_BYTES("files_copied_bytes",
+      "Total number of bytes copied within the object store."),
+  FILES_CREATED("files_created",
+      "Total number of files created through the object store."),
+  FILES_DELETED("files_deleted",
+      "Total number of files deleted from the object store."),
+  IGNORED_ERRORS("ignored_errors", "Errors caught and ignored"),
+  INVOCATION_COPY_FROM_LOCAL_FILE("invocations_copyfromlocalfile",
+      "Calls of copyFromLocalFile()"),
+  INVOCATION_EXISTS("invocations_exists",
+      "Calls of exists()"),
+  INVOCATION_GET_FILE_STATUS("invocations_getfilestatus",
+      "Calls of getFileStatus()"),
+  INVOCATION_GLOB_STATUS("invocations_globstatus",
+      "Calls of globStatus()"),
+  INVOCATION_IS_DIRECTORY("invocations_is_directory",
+      "Calls of isDirectory()"),
+  INVOCATION_IS_FILE("invocations_is_file",
+      "Calls of isFile()"),
+  INVOCATION_LIST_FILES("invocations_listfiles",
+      "Calls of listFiles()"),
+  INVOCATION_LIST_LOCATED_STATUS("invocations_listlocatedstatus",
+      "Calls of listLocatedStatus()"),
+  INVOCATION_LIST_STATUS("invocations_liststatus",
+      "Calls of listStatus()"),
+  INVOCATION_MKDIRS("invocations_mdkirs",
+      "Calls of mkdirs()"),
+  INVOCATION_RENAME("invocations_rename",
+      "Calls of rename()"),
+  OBJECT_COPY_REQUESTS("object_copy_requests", "Object copy requests"),
+  OBJECT_DELETE_REQUESTS("object_delete_requests", "Object delete requests"),
+  OBJECT_LIST_REQUESTS("object_list_requests",
+      "Number of object listings made"),
+  OBJECT_METADATA_REQUESTS("object_metadata_requests",
+      "Number of requests for object metadata"),
+  OBJECT_MULTIPART_UPLOAD_ABORTED("object_multipart_aborted",
+      "Object multipart upload aborted"),
+  OBJECT_PUT_REQUESTS("object_put_requests",
+      "Object put/multipart upload count"),
+  OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"),
+  STREAM_ABORTED("streamAborted",
+      "Count of times the TCP stream was aborted"),
+  STREAM_BACKWARD_SEEK_OPERATIONS("streamBackwardSeekOperations",
+      "Number of executed seek operations which went backwards in a stream"),
+  STREAM_CLOSED("streamClosed", "Count of times the TCP stream was closed"),
+  STREAM_CLOSE_OPERATIONS("streamCloseOperations",
+      "Total count of times an attempt to close a data stream was made"),
+  STREAM_FORWARD_SEEK_OPERATIONS("streamForwardSeekOperations",
+      "Number of executed seek operations which went forward in a stream"),
+  STREAM_OPENED("streamOpened",
+      "Total count of times an input stream to object store was opened"),
+  STREAM_READ_EXCEPTIONS("streamReadExceptions",
+      "Number of seek operations invoked on input streams"),
+  STREAM_READ_FULLY_OPERATIONS("streamReadFullyOperations",
+      "count of readFully() operations in streams"),
+  STREAM_READ_OPERATIONS("streamReadOperations",
+      "Count of read() operations in streams"),
+  STREAM_READ_OPERATIONS_INCOMPLETE("streamReadOperationsIncomplete",
+      "Count of incomplete read() operations in streams"),
+  STREAM_SEEK_BYTES_BACKWARDS("streamBytesBackwardsOnSeek",
+      "Count of bytes moved backwards during seek operations"),
+  STREAM_SEEK_BYTES_READ("streamBytesRead",
+      "Count of bytes read during seek() in stream operations"),
+  STREAM_SEEK_BYTES_SKIPPED("streamBytesSkippedOnSeek",
+      "Count of bytes skipped during forward seek operation"),
+  STREAM_SEEK_OPERATIONS("streamSeekOperations",
+      "Number of read exceptions caught and attempted to recovered from");
+
+  Statistic(String symbol, String description) {
+    this.symbol = symbol;
+    this.description = description;
+  }
+
+  private final String symbol;
+  private final String description;
+
+  public String getSymbol() {
+    return symbol;
+  }
+
+  /**
+   * Get a statistic from a symbol.
+   * @param symbol statistic to look up
+   * @return the value or null.
+   */
+  public static Statistic fromSymbol(String symbol) {
+    if (symbol != null) {
+      for (Statistic opType : values()) {
+        if (opType.getSymbol().equals(symbol)) {
+          return opType;
+        }
+      }
+    }
+    return null;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  /**
+   * The string value is simply the symbol.
+   * This makes this operation very low cost.
+   * @return the symbol of this statistic.
+   */
+  @Override
+  public String toString() {
+    return symbol;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 4f5a077..7a5e455 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -735,9 +735,19 @@ The exact number of operations to perform is configurable in the option
 Larger values generate more load, and are recommended when testing locally,
 or in batch runs.
 
-Smaller values should result in faster test runs, especially when the object
+Smaller values results in faster test runs, especially when the object
 store is a long way away.
 
+Operations which work on directories have a separate option: this controls
+the width and depth of tests creating recursive directories. Larger
+values create exponentially more directories, with consequent performance
+impact.
+
+      <property>
+        <name>scale.test.directory.count</name>
+        <value>2</value>
+      </property>
+
 DistCp tests targeting S3A support a configurable file size.  The default is
 10 MB, but the configuration value is expressed in KB so that it can be tuned
 smaller to achieve faster test runs.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index a4f9b99..04010d6 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -21,7 +21,9 @@ package org.apache.hadoop.fs.s3a;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
+import org.junit.Assert;
 import org.junit.internal.AssumptionViolatedException;
+import org.slf4j.Logger;
 
 import java.io.IOException;
 import java.net.URI;
@@ -190,4 +192,155 @@ public class S3ATestUtils {
     }
   }
 
+  /**
+   * Reset all metrics in a list.
+   * @param metrics metrics to reset
+   */
+  public static void reset(S3ATestUtils.MetricDiff... metrics) {
+    for (S3ATestUtils.MetricDiff metric : metrics) {
+      metric.reset();
+    }
+  }
+
+  /**
+   * Print all metrics in a list.
+   * @param log log to print the metrics to.
+   * @param metrics metrics to process
+   */
+  public static void print(Logger log, S3ATestUtils.MetricDiff... metrics) {
+    for (S3ATestUtils.MetricDiff metric : metrics) {
+      log.info(metric.toString());
+    }
+  }
+
+  /**
+   * Print all metrics in a list, then reset them.
+   * @param log log to print the metrics to.
+   * @param metrics metrics to process
+   */
+  public static void printThenReset(Logger log,
+      S3ATestUtils.MetricDiff... metrics) {
+    print(log, metrics);
+    reset(metrics);
+  }
+
+  /**
+   * Helper class to do diffs of metrics.
+   */
+  public static final class MetricDiff {
+    private final S3AFileSystem fs;
+    private final Statistic statistic;
+    private long startingValue;
+
+    /**
+     * Constructor.
+     * Invokes {@link #reset()} so it is immediately capable of measuring the
+     * difference in metric values.
+     *
+     * @param fs the filesystem to monitor
+     * @param statistic the statistic to monitor.
+     */
+    public MetricDiff(S3AFileSystem fs, Statistic statistic) {
+      this.fs = fs;
+      this.statistic = statistic;
+      reset();
+    }
+
+    /**
+     * Reset the starting value to the current value.
+     * Diffs will be against this new value.
+     */
+    public void reset() {
+      startingValue = currentValue();
+    }
+
+    /**
+     * Get the current value of the metric.
+     * @return the latest value.
+     */
+    public long currentValue() {
+      return fs.getInstrumentation().getCounterValue(statistic);
+    }
+
+    /**
+     * Get the difference between the the current value and
+     * {@link #startingValue}.
+     * @return the difference.
+     */
+    public long diff() {
+      return currentValue() - startingValue;
+    }
+
+    @Override
+    public String toString() {
+      long c = currentValue();
+      final StringBuilder sb = new StringBuilder(statistic.getSymbol());
+      sb.append(" starting=").append(startingValue);
+      sb.append(" current=").append(c);
+      sb.append(" diff=").append(c - startingValue);
+      return sb.toString();
+    }
+
+    /**
+     * Assert that the value of {@link #diff()} matches that expected.
+     * @param expected expected value.
+     */
+    public void assertDiffEquals(long expected) {
+      Assert.assertEquals("Count of " + this,
+          expected, diff());
+    }
+
+    /**
+     * Assert that the value of {@link #diff()} matches that of another
+     * instance.
+     * @param that the other metric diff instance.
+     */
+    public void assertDiffEquals(MetricDiff that) {
+      Assert.assertEquals(this.toString() + " != " + that,
+          this.diff(), that.diff());
+    }
+
+    /**
+     * Comparator for assertions.
+     * @param that other metric diff
+     * @return true if the value is {@code ==} the other's
+     */
+    public boolean diffEquals(MetricDiff that) {
+      return this.currentValue() == that.currentValue();
+    }
+
+    /**
+     * Comparator for assertions.
+     * @param that other metric diff
+     * @return true if the value is {@code <} the other's
+     */
+    public boolean diffLessThan(MetricDiff that) {
+      return this.currentValue() < that.currentValue();
+    }
+
+    /**
+     * Comparator for assertions.
+     * @param that other metric diff
+     * @return true if the value is {@code <=} the other's
+     */
+    public boolean diffLessThanOrEquals(MetricDiff that) {
+      return this.currentValue() <= that.currentValue();
+    }
+
+    /**
+     * Get the statistic
+     * @return the statistic
+     */
+    public Statistic getStatistic() {
+      return statistic;
+    }
+
+    /**
+     * Get the starting value; that set in the last {@link #reset()}.
+     * @return the starting value for diffs.
+     */
+    public long getStartingValue() {
+      return startingValue;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileOperationCost.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileOperationCost.java
new file mode 100644
index 0000000..0a8dd2d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileOperationCost.java
@@ -0,0 +1,191 @@
+/*
+ * 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.fs.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.net.URI;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.Statistic.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
+import static org.apache.hadoop.test.GenericTestUtils.getTestDir;
+
+/**
+ * Use metrics to assert about the cost of file status queries.
+ * {@link S3AFileSystem#getFileStatus(Path)}.
+ */
+public class TestS3AFileOperationCost extends AbstractFSContractTestBase {
+
+  private MetricDiff metadataRequests;
+  private MetricDiff listRequests;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestS3AFileOperationCost.class);
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new S3AContract(conf);
+  }
+
+  @Override
+  public S3AFileSystem getFileSystem() {
+    return (S3AFileSystem) super.getFileSystem();
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    S3AFileSystem fs = getFileSystem();
+    metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
+    listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS);
+  }
+
+  @Test
+  public void testCostOfGetFileStatusOnFile() throws Throwable {
+    describe("performing getFileStatus on a file");
+    Path simpleFile = path("simple.txt");
+    S3AFileSystem fs = getFileSystem();
+    touch(fs, simpleFile);
+    resetMetricDiffs();
+    S3AFileStatus status = fs.getFileStatus(simpleFile);
+    assertTrue("not a file: " + status, status.isFile());
+    metadataRequests.assertDiffEquals(1);
+    listRequests.assertDiffEquals(0);
+  }
+
+  private void resetMetricDiffs() {
+    reset(metadataRequests, listRequests);
+  }
+
+  @Test
+  public void testCostOfGetFileStatusOnEmptyDir() throws Throwable {
+    describe("performing getFileStatus on an empty directory");
+    S3AFileSystem fs = getFileSystem();
+    Path dir = path("empty");
+    fs.mkdirs(dir);
+    resetMetricDiffs();
+    S3AFileStatus status = fs.getFileStatus(dir);
+    assertTrue("not empty: " + status, status.isEmptyDirectory());
+    metadataRequests.assertDiffEquals(2);
+    listRequests.assertDiffEquals(0);
+  }
+
+  @Test
+  public void testCostOfGetFileStatusOnMissingFile() throws Throwable {
+    describe("performing getFileStatus on a missing file");
+    S3AFileSystem fs = getFileSystem();
+    Path path = path("missing");
+    resetMetricDiffs();
+    try {
+      S3AFileStatus status = fs.getFileStatus(path);
+      fail("Got a status back from a missing file path " + status);
+    } catch (FileNotFoundException expected) {
+      // expected
+    }
+    metadataRequests.assertDiffEquals(2);
+    listRequests.assertDiffEquals(1);
+  }
+
+  @Test
+  public void testCostOfGetFileStatusOnMissingSubPath() throws Throwable {
+    describe("performing getFileStatus on a missing file");
+    S3AFileSystem fs = getFileSystem();
+    Path path = path("missingdir/missingpath");
+    resetMetricDiffs();
+    try {
+      S3AFileStatus status = fs.getFileStatus(path);
+      fail("Got a status back from a missing file path " + status);
+    } catch (FileNotFoundException expected) {
+      // expected
+    }
+    metadataRequests.assertDiffEquals(2);
+    listRequests.assertDiffEquals(1);
+  }
+
+  @Test
+  public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable {
+    describe("performing getFileStatus on a non-empty directory");
+    S3AFileSystem fs = getFileSystem();
+    Path dir = path("empty");
+    fs.mkdirs(dir);
+    Path simpleFile = new Path(dir, "simple.txt");
+    touch(fs, simpleFile);
+    resetMetricDiffs();
+    S3AFileStatus status = fs.getFileStatus(dir);
+    if (status.isEmptyDirectory()) {
+      // erroneous state
+      String fsState = fs.toString();
+      fail("FileStatus says directory isempty: " + status
+          + "\n" + ContractTestUtils.ls(fs, dir)
+          + "\n" + fsState);
+    }
+    metadataRequests.assertDiffEquals(2);
+    listRequests.assertDiffEquals(1);
+  }
+
+  @Test
+  public void testCostOfCopyFromLocalFile() throws Throwable {
+    describe("testCostOfCopyFromLocalFile");
+    File localTestDir = getTestDir("tmp");
+    localTestDir.mkdirs();
+    File tmpFile = File.createTempFile("tests3acost", ".txt",
+        localTestDir);
+    tmpFile.delete();
+    try {
+      URI localFileURI = tmpFile.toURI();
+      FileSystem localFS = FileSystem.get(localFileURI,
+          getFileSystem().getConf());
+      Path localPath = new Path(localFileURI);
+      int len = 10 * 1024;
+      byte[] data = dataset(len, 'A', 'Z');
+      writeDataset(localFS, localPath, data, len, 1024, true);
+      S3AFileSystem s3a = getFileSystem();
+      MetricDiff copyLocalOps = new MetricDiff(s3a,
+          INVOCATION_COPY_FROM_LOCAL_FILE);
+      MetricDiff putRequests = new MetricDiff(s3a,
+          OBJECT_PUT_REQUESTS);
+      MetricDiff putBytes = new MetricDiff(s3a,
+          OBJECT_PUT_BYTES);
+
+      Path remotePath = path("copied");
+      s3a.copyFromLocalFile(false, true, localPath, remotePath);
+      verifyFileContents(s3a, remotePath, data);
+      copyLocalOps.assertDiffEquals(1);
+      putRequests.assertDiffEquals(1);
+      putBytes.assertDiffEquals(len);
+      // print final stats
+      LOG.info("Filesystem {}", s3a);
+    } finally {
+      tmpFile.delete();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
index d65f693..21639b1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
@@ -34,13 +34,11 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.InputStream;
-import java.util.Locale;
-
-import static org.junit.Assume.assumeTrue;
 
 /**
  * Base class for scale tests; here is where the common scale configuration
@@ -51,11 +49,57 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
   @Rule
   public TestName methodName = new TestName();
 
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
   @BeforeClass
   public static void nameThread() {
     Thread.currentThread().setName("JUnit");
   }
 
+  /**
+   * The number of operations to perform: {@value}.
+   */
+  public static final String KEY_OPERATION_COUNT =
+      SCALE_TEST + "operation.count";
+
+  /**
+   * The number of directory operations to perform: {@value}.
+   */
+  public static final String KEY_DIRECTORY_COUNT =
+      SCALE_TEST + "directory.count";
+
+  /**
+   * The readahead buffer: {@value}.
+   */
+  public static final String KEY_READ_BUFFER_SIZE =
+      S3A_SCALE_TEST + "read.buffer.size";
+
+  public static final int DEFAULT_READ_BUFFER_SIZE = 16384;
+
+  /**
+   * Key for a multi MB test file: {@value}.
+   */
+  public static final String KEY_CSVTEST_FILE =
+      S3A_SCALE_TEST + "csvfile";
+
+  /**
+   * Default path for the multi MB test file: {@value}.
+   */
+  public static final String DEFAULT_CSVTEST_FILE
+      = "s3a://landsat-pds/scene_list.gz";
+
+  /**
+   * The default number of operations to perform: {@value}.
+   */
+  public static final long DEFAULT_OPERATION_COUNT = 2005;
+
+  /**
+   * Default number of directories to create when performing
+   * directory performance/scale tests.
+   */
+  public static final int DEFAULT_DIRECTORY_COUNT = 2;
+
   protected S3AFileSystem fs;
 
   protected static final Logger LOG =
@@ -132,108 +176,4 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants {
     }
   }
 
-  /**
-   * Make times more readable, by adding a "," every three digits.
-   * @param nanos nanos or other large number
-   * @return a string for logging
-   */
-  protected static String toHuman(long nanos) {
-    return String.format(Locale.ENGLISH, "%,d", nanos);
-  }
-
-  /**
-   * Log the bandwidth of a timer as inferred from the number of
-   * bytes processed.
-   * @param timer timer
-   * @param bytes bytes processed in the time period
-   */
-  protected void bandwidth(NanoTimer timer, long bytes) {
-    LOG.info("Bandwidth = {}  MB/S",
-        timer.bandwidthDescription(bytes));
-  }
-
-  /**
-   * Work out the bandwidth in MB/s
-   * @param bytes bytes
-   * @param durationNS duration in nanos
-   * @return the number of megabytes/second of the recorded operation
-   */
-  public static double bandwidthMBs(long bytes, long durationNS) {
-    return (bytes * 1000.0 ) / durationNS;
-  }
-
-  /**
-   * A simple class for timing operations in nanoseconds, and for
-   * printing some useful results in the process.
-   */
-  protected static class NanoTimer {
-    final long startTime;
-    long endTime;
-
-    public NanoTimer() {
-      startTime = now();
-    }
-
-    /**
-     * End the operation
-     * @return the duration of the operation
-     */
-    public long end() {
-      endTime = now();
-      return duration();
-    }
-
-    /**
-     * End the operation; log the duration
-     * @param format message
-     * @param args any arguments
-     * @return the duration of the operation
-     */
-    public long end(String format, Object... args) {
-      long d = end();
-      LOG.info("Duration of {}: {} nS",
-          String.format(format, args), toHuman(d));
-      return d;
-    }
-
-    long now() {
-      return System.nanoTime();
-    }
-
-    long duration() {
-      return endTime - startTime;
-    }
-
-    double bandwidth(long bytes) {
-      return S3AScaleTestBase.bandwidthMBs(bytes, duration());
-    }
-
-    /**
-     * Bandwidth as bytes per second
-     * @param bytes bytes in
-     * @return the number of bytes per second this operation timed.
-     */
-    double bandwidthBytes(long bytes) {
-      return (bytes * 1.0 ) / duration();
-    }
-
-    /**
-     * How many nanoseconds per byte
-     * @param bytes bytes processed in this time period
-     * @return the nanoseconds it took each byte to be processed
-     */
-    long nanosPerByte(long bytes) {
-      return duration() / bytes;
-    }
-
-    /**
-     * Get a description of the bandwidth, even down to fractions of
-     * a MB
-     * @param bytes bytes processed
-     * @return bandwidth
-     */
-    String bandwidthDescription(long bytes) {
-      return String.format("%,.6f", bandwidth(bytes));
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java
index af1883e..5e07dcb 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADeleteManyFiles.java
@@ -20,9 +20,7 @@ package org.apache.hadoop.fs.s3a.scale;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,15 +32,13 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import static org.junit.Assert.assertEquals;
-
+/**
+ * Test some scalable operations related to file renaming and deletion.
+ */
 public class TestS3ADeleteManyFiles extends S3AScaleTestBase {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestS3ADeleteManyFiles.class);
 
-  @Rule
-  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
-
   /**
    * CAUTION: If this test starts failing, please make sure that the
    * {@link org.apache.hadoop.fs.s3a.Constants#MAX_THREADS} configuration is not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
new file mode 100644
index 0000000..7ece394
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3ADirectoryPerformance.java
@@ -0,0 +1,189 @@
+/*
+ * 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.fs.s3a.scale;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.s3a.Statistic.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+
+/**
+ * Test the performance of listing files/directories.
+ */
+public class TestS3ADirectoryPerformance extends S3AScaleTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      TestS3ADirectoryPerformance.class);
+
+  @Test
+  public void testListOperations() throws Throwable {
+    describe("Test recursive list operations");
+    final Path scaleTestDir = getTestPath();
+    final Path listDir = new Path(scaleTestDir, "lists");
+
+    // scale factor.
+    int scale = getConf().getInt(KEY_DIRECTORY_COUNT, DEFAULT_DIRECTORY_COUNT);
+    int width = scale;
+    int depth = scale;
+    int files = scale;
+    MetricDiff metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
+    MetricDiff listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS);
+    MetricDiff listStatusCalls = new MetricDiff(fs, INVOCATION_LIST_FILES);
+    MetricDiff getFileStatusCalls =
+        new MetricDiff(fs, INVOCATION_GET_FILE_STATUS);
+    NanoTimer createTimer = new NanoTimer();
+    TreeScanResults created =
+        createSubdirs(fs, listDir, depth, width, files, 0);
+    // add some empty directories
+    int emptyDepth = 1 * scale;
+    int emptyWidth = 3 * scale;
+
+    created.add(createSubdirs(fs, listDir, emptyDepth, emptyWidth, 0,
+        0, "empty", "f-", ""));
+    createTimer.end("Time to create %s", created);
+    LOG.info("Time per operation: {}",
+        toHuman(createTimer.nanosPerOperation(created.totalCount())));
+    printThenReset(LOG,
+        metadataRequests,
+        listRequests,
+        listStatusCalls,
+        getFileStatusCalls);
+
+    try {
+      // Scan the directory via an explicit tree walk.
+      // This is the baseline for any listing speedups.
+      MetricDiff treewalkMetadataRequests =
+          new MetricDiff(fs, OBJECT_METADATA_REQUESTS);
+      MetricDiff treewalkListRequests = new MetricDiff(fs,
+          OBJECT_LIST_REQUESTS);
+      MetricDiff treewalkListStatusCalls = new MetricDiff(fs,
+          INVOCATION_LIST_FILES);
+      MetricDiff treewalkGetFileStatusCalls =
+          new MetricDiff(fs, INVOCATION_GET_FILE_STATUS);
+      NanoTimer treeWalkTimer = new NanoTimer();
+      TreeScanResults treewalkResults = treeWalk(fs, listDir);
+      treeWalkTimer.end("List status via treewalk");
+
+      print(LOG,
+          treewalkMetadataRequests,
+          treewalkListRequests,
+          treewalkListStatusCalls,
+          treewalkGetFileStatusCalls);
+      assertEquals("Files found in listFiles(recursive=true) " +
+              " created=" + created + " listed=" + treewalkResults,
+          created.getFileCount(), treewalkResults.getFileCount());
+
+
+      // listFiles() does the recursion internally
+      NanoTimer listFilesRecursiveTimer = new NanoTimer();
+
+      TreeScanResults listFilesResults = new TreeScanResults(
+          fs.listFiles(listDir, true));
+
+      listFilesRecursiveTimer.end("listFiles(recursive=true) of %s", created);
+      assertEquals("Files found in listFiles(recursive=true) " +
+          " created=" + created  + " listed=" + listFilesResults,
+          created.getFileCount(), listFilesResults.getFileCount());
+
+      treewalkListRequests.assertDiffEquals(listRequests);
+      printThenReset(LOG,
+          metadataRequests, listRequests,
+          listStatusCalls, getFileStatusCalls);
+
+      NanoTimer globStatusTimer = new NanoTimer();
+      FileStatus[] globStatusFiles = fs.globStatus(listDir);
+      globStatusTimer.end("Time to globStatus() %s", globStatusTimer);
+      LOG.info("Time for glob status {} entries: {}",
+          globStatusFiles.length,
+          toHuman(createTimer.duration()));
+      printThenReset(LOG,
+          metadataRequests,
+          listRequests,
+          listStatusCalls,
+          getFileStatusCalls);
+
+    } finally {
+      // deletion at the end of the run
+      NanoTimer deleteTimer = new NanoTimer();
+      fs.delete(listDir, true);
+      deleteTimer.end("Deleting directory tree");
+      printThenReset(LOG,
+          metadataRequests, listRequests,
+          listStatusCalls, getFileStatusCalls);
+    }
+  }
+
+  @Test
+  public void testTimeToStatEmptyDirectory() throws Throwable {
+    describe("Time to stat an empty directory");
+    Path path = new Path(getTestPath(), "empty");
+    fs.mkdirs(path);
+    timeToStatPath(path);
+  }
+
+  @Test
+  public void testTimeToStatNonEmptyDirectory() throws Throwable {
+    describe("Time to stat a non-empty directory");
+    Path path = new Path(getTestPath(), "dir");
+    fs.mkdirs(path);
+    touch(fs, new Path(path, "file"));
+    timeToStatPath(path);
+  }
+
+  @Test
+  public void testTimeToStatFile() throws Throwable {
+    describe("Time to stat a simple file");
+    Path path = new Path(getTestPath(), "file");
+    touch(fs, path);
+    timeToStatPath(path);
+  }
+
+  @Test
+  public void testTimeToStatRoot() throws Throwable {
+    describe("Time to stat the root path");
+    timeToStatPath(new Path("/"));
+  }
+
+  private void timeToStatPath(Path path) throws IOException {
+    describe("Timing getFileStatus(\"%s\")", path);
+    MetricDiff metadataRequests =
+        new MetricDiff(fs, Statistic.OBJECT_METADATA_REQUESTS);
+    MetricDiff listRequests =
+        new MetricDiff(fs, Statistic.OBJECT_LIST_REQUESTS);
+    long attempts = getOperationCount();
+    NanoTimer timer = new NanoTimer();
+    for (long l = 0; l < attempts; l++) {
+      fs.getFileStatus(path);
+    }
+    timer.end("Time to execute %d getFileStatusCalls", attempts);
+    LOG.info("Time per call: {}", toHuman(timer.nanosPerOperation(attempts)));
+    LOG.info("metadata: {}", metadataRequests);
+    LOG.info("metadata per operation {}", metadataRequests.diff() / attempts);
+    LOG.info("listObjects: {}", listRequests);
+    LOG.info("listObjects: per operation {}", listRequests.diff() / attempts);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java
index 0c8b273..5222a4e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java
@@ -36,8 +36,10 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+
 /**
- * Look at the performance of S3a operations
+ * Look at the performance of S3a operations.
  */
 public class TestS3AInputStreamPerformance extends S3AScaleTestBase {
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -151,7 +153,7 @@ public class TestS3AInputStreamPerformance extends S3AScaleTestBase {
     readTimer.end("Time to read %d bytes", len);
     bandwidth(readTimer, count);
     assertEquals("Not enough bytes were read)", len, count);
-    long nanosPerByte = readTimer.nanosPerByte(count);
+    long nanosPerByte = readTimer.nanosPerOperation(count);
     LOG.info("An open() call has the equivalent duration of reading {} bytes",
         toHuman( timeOpen.duration() / nanosPerByte));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c58a59f7/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
index bc85425..1330ed1 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
@@ -15,7 +15,9 @@ log4j.rootLogger=info,stdout
 log4j.threshold=ALL
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
 
 # for debugging low level S3a operations, uncomment this line
 # log4j.logger.org.apache.hadoop.fs.s3a=DEBUG


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


[45/47] hadoop git commit: HDFS-10485. Fix findbugs warning in FSEditLog.java. (aajisaka)

Posted by ae...@apache.org.
HDFS-10485. Fix findbugs warning in FSEditLog.java. (aajisaka)


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

Branch: refs/heads/HDFS-1312
Commit: e620530301fd3e62537d4b7bc3d8ed296bda1ffc
Parents: bddea5f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Jun 7 17:52:03 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jun 7 17:52:55 2016 +0900

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/namenode/FSEditLog.java    | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6205303/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 809d9e6..57229da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -27,6 +27,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -174,7 +175,7 @@ public class FSEditLog implements LogsPurgeable {
   
   // these are statistics counters.
   private long numTransactions;        // number of transactions
-  private long numTransactionsBatchedInSync;
+  private final AtomicLong numTransactionsBatchedInSync = new AtomicLong();
   private long totalTimeTransactions;  // total time for all transactions
   private NameNodeMetrics metrics;
 
@@ -672,7 +673,7 @@ public class FSEditLog implements LogsPurgeable {
       if (metrics != null) { // Metrics non-null only when used inside name node
         metrics.addSync(elapsed);
         metrics.incrTransactionsBatchedInSync(editsBatchedInSync);
-        numTransactionsBatchedInSync += editsBatchedInSync;
+        numTransactionsBatchedInSync.addAndGet(editsBatchedInSync);
       }
       
     } finally {
@@ -712,7 +713,7 @@ public class FSEditLog implements LogsPurgeable {
     buf.append(" Total time for transactions(ms): ");
     buf.append(totalTimeTransactions);
     buf.append(" Number of transactions batched in Syncs: ");
-    buf.append(numTransactionsBatchedInSync);
+    buf.append(numTransactionsBatchedInSync.get());
     buf.append(" Number of syncs: ");
     buf.append(editLogStream.getNumSync());
     buf.append(" SyncTimes(ms): ");
@@ -1281,7 +1282,9 @@ public class FSEditLog implements LogsPurgeable {
         "Cannot start log segment at txid %s when next expected " +
         "txid is %s", segmentTxId, txid + 1);
     
-    numTransactions = totalTimeTransactions = numTransactionsBatchedInSync = 0;
+    numTransactions = 0;
+    totalTimeTransactions = 0;
+    numTransactionsBatchedInSync.set(0L);
 
     // TODO no need to link this back to storage anymore!
     // See HDFS-2174.


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


[41/47] hadoop git commit: HDFS-10458. getFileEncryptionInfo should return quickly for non-encrypted cluster.

Posted by ae...@apache.org.
HDFS-10458. getFileEncryptionInfo should return quickly for non-encrypted cluster.


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

Branch: refs/heads/HDFS-1312
Commit: 6de9213df111a9a4ed875db995d67af72d08a798
Parents: a3f78d8
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Jun 6 15:52:39 2016 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jun 6 15:52:39 2016 -0700

----------------------------------------------------------------------
 .../server/namenode/EncryptionZoneManager.java  | 35 +++++++++++++++++---
 .../server/namenode/FSDirEncryptionZoneOp.java  |  2 +-
 2 files changed, 31 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6de9213d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
index 8454c04..41dbb59 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
@@ -95,7 +95,7 @@ public class EncryptionZoneManager {
     }
   }
 
-  private final TreeMap<Long, EncryptionZoneInt> encryptionZones;
+  private TreeMap<Long, EncryptionZoneInt> encryptionZones = null;
   private final FSDirectory dir;
   private final int maxListEncryptionZonesResponses;
 
@@ -106,7 +106,6 @@ public class EncryptionZoneManager {
    */
   public EncryptionZoneManager(FSDirectory dir, Configuration conf) {
     this.dir = dir;
-    encryptionZones = new TreeMap<Long, EncryptionZoneInt>();
     maxListEncryptionZonesResponses = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
         DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES_DEFAULT
@@ -143,6 +142,9 @@ public class EncryptionZoneManager {
       CipherSuite suite, CryptoProtocolVersion version, String keyName) {
     final EncryptionZoneInt ez = new EncryptionZoneInt(
         inodeId, suite, version, keyName);
+    if (encryptionZones == null) {
+      encryptionZones = new TreeMap<>();
+    }
     encryptionZones.put(inodeId, ez);
   }
 
@@ -153,7 +155,9 @@ public class EncryptionZoneManager {
    */
   void removeEncryptionZone(Long inodeId) {
     assert dir.hasWriteLock();
-    encryptionZones.remove(inodeId);
+    if (hasCreatedEncryptionZone()) {
+      encryptionZones.remove(inodeId);
+    }
   }
 
   /**
@@ -201,6 +205,9 @@ public class EncryptionZoneManager {
   private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip) {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
+    if (!hasCreatedEncryptionZone()) {
+      return null;
+    }
     List<INode> inodes = iip.getReadOnlyINodes();
     for (int i = inodes.size() - 1; i >= 0; i--) {
       final INode inode = inodes.get(i);
@@ -313,7 +320,8 @@ public class EncryptionZoneManager {
       throw new IOException("Attempt to create an encryption zone for a file.");
     }
 
-    if (encryptionZones.get(srcINode.getId()) != null) {
+    if (hasCreatedEncryptionZone() && encryptionZones.
+        get(srcINode.getId()) != null) {
       throw new IOException("Directory " + src + " is already an encryption " +
           "zone.");
     }
@@ -340,6 +348,9 @@ public class EncryptionZoneManager {
   BatchedListEntries<EncryptionZone> listEncryptionZones(long prevId)
       throws IOException {
     assert dir.hasReadLock();
+    if (!hasCreatedEncryptionZone()) {
+      return new BatchedListEntries<EncryptionZone>(Lists.newArrayList(), false);
+    }
     NavigableMap<Long, EncryptionZoneInt> tailMap = encryptionZones.tailMap
         (prevId, false);
     final int numResponses = Math.min(maxListEncryptionZonesResponses,
@@ -379,7 +390,18 @@ public class EncryptionZoneManager {
    * @return number of encryption zones.
    */
   public int getNumEncryptionZones() {
-    return encryptionZones.size();
+    return hasCreatedEncryptionZone() ?
+        encryptionZones.size() : 0;
+  }
+
+  /**
+   * @return Whether there has been any attempt to create an encryption zone in
+   * the cluster at all. If not, it is safe to quickly return null when
+   * checking the encryption information of any file or directory in the
+   * cluster.
+   */
+  public boolean hasCreatedEncryptionZone() {
+    return encryptionZones != null;
   }
 
   /**
@@ -387,6 +409,9 @@ public class EncryptionZoneManager {
    */
   String[] getKeyNames() {
     assert dir.hasReadLock();
+    if (!hasCreatedEncryptionZone()) {
+      return new String[0];
+    }
     String[] ret = new String[encryptionZones.size()];
     int index = 0;
     for (Map.Entry<Long, EncryptionZoneInt> entry : encryptionZones

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6de9213d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
index bd25419..2997179 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
@@ -254,7 +254,7 @@ final class FSDirEncryptionZoneOp {
   static FileEncryptionInfo getFileEncryptionInfo(final FSDirectory fsd,
       final INode inode, final int snapshotId, final INodesInPath iip)
       throws IOException {
-    if (!inode.isFile()) {
+    if (!inode.isFile() || !fsd.ezManager.hasCreatedEncryptionZone()) {
       return null;
     }
     fsd.readLock();


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


[17/47] hadoop git commit: HADOOP-13109. Add ability to edit existing token file via dtutil -alias flag (Matthew Paduano via aw)

Posted by ae...@apache.org.
HADOOP-13109. Add ability to edit existing token file via dtutil -alias flag (Matthew Paduano via aw)


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

Branch: refs/heads/HDFS-1312
Commit: 78b3a038319cb351632250279f171b756c7f24b0
Parents: db54670
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jun 3 15:34:24 2016 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Fri Jun 3 15:34:39 2016 -0700

----------------------------------------------------------------------
 .../hadoop/security/token/DtFileOperations.java | 23 ++++++++++
 .../hadoop/security/token/DtUtilShell.java      | 47 +++++++++++++++++---
 .../hadoop/security/token/TestDtUtilShell.java  | 24 ++++++++++
 3 files changed, 89 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78b3a038/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
index bdda7c9..1396054 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
@@ -199,6 +199,29 @@ public final class DtFileOperations {
     doFormattedWrite(tokenFile, fileFormat, creds, conf);
   }
 
+  /** Alias a token from a file and save back to file in the local filesystem.
+   *  @param tokenFile a local File object to hold the input and output.
+   *  @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output
+   *  @param alias overwrite service field of fetched token with this text.
+   *  @param service only apply alias to tokens matching this service text.
+   *  @param conf Configuration object passed along.
+   *  @throws IOException
+   */
+  public static void aliasTokenFile(File tokenFile, String fileFormat,
+      Text alias, Text service, Configuration conf) throws Exception {
+    Credentials newCreds = new Credentials();
+    Credentials creds = Credentials.readTokenStorageFile(tokenFile, conf);
+    for (Token<?> token : creds.getAllTokens()) {
+      newCreds.addToken(token.getService(), token);
+      if (token.getService().equals(service)) {
+        Token<?> aliasedToken = token.copyToken();
+        aliasedToken.setService(alias);
+        newCreds.addToken(alias, aliasedToken);
+      }
+    }
+    doFormattedWrite(tokenFile, fileFormat, newCreds, conf);
+  }
+
   /** Append tokens from list of files in local filesystem, saving to last file.
    *  @param tokenFiles list of local File objects.  Last file holds the output.
    *  @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78b3a038/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java
index fb74c31..f00e6fd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java
@@ -41,7 +41,7 @@ public class DtUtilShell extends CommandShell {
       DtFileOperations.FORMAT_PB + ")]";
   public static final String DT_USAGE = "hadoop dtutil " +
       "[-keytab <keytab_file> -principal <principal_name>] " +
-      "subcommand (help|print|get|append|cancel|remove|renew) " +
+      "subcommand (help|print|get|edit|append|cancel|remove|renew) " +
          FORMAT_SUBSTRING + " [-alias <alias>] filename...";
 
   // command line options
@@ -50,6 +50,7 @@ public class DtUtilShell extends CommandShell {
   private static final String PRINCIPAL = "-principal";
   private static final String PRINT = "print";
   private static final String GET = "get";
+  private static final String EDIT = "edit";
   private static final String APPEND = "append";
   private static final String CANCEL = "cancel";
   private static final String REMOVE = "remove";
@@ -127,6 +128,8 @@ public class DtUtilShell extends CommandShell {
           setSubCommand(new Print());
         } else if (command.equals(GET)) {
           setSubCommand(new Get(args[++i]));
+        } else if (command.equals(EDIT)) {
+          setSubCommand(new Edit());
         } else if (command.equals(APPEND)) {
           setSubCommand(new Append());
         } else if (command.equals(CANCEL)) {
@@ -172,10 +175,12 @@ public class DtUtilShell extends CommandShell {
 
   @Override
   public String getCommandUsage() {
-    return String.format("%n%s%n   %s%n   %s%n   %s%n   %s%n   %s%n   %s%n%n",
-                  DT_USAGE, (new Print()).getUsage(), (new Get()).getUsage(),
-                  (new Append()).getUsage(), (new Remove(true)).getUsage(),
-                  (new Remove(false)).getUsage(), (new Renew()).getUsage());
+    return String.format(
+        "%n%s%n   %s%n   %s%n   %s%n   %s%n   %s%n   %s%n   %s%n%n",
+        DT_USAGE, (new Print()).getUsage(), (new Get()).getUsage(),
+        (new Edit()).getUsage(), (new Append()).getUsage(),
+        (new Remove(true)).getUsage(), (new Remove(false)).getUsage(),
+        (new Renew()).getUsage());
   }
 
   private class Print extends SubCommand {
@@ -242,6 +247,38 @@ public class DtUtilShell extends CommandShell {
     }
   }
 
+  private class Edit extends SubCommand {
+    public static final String EDIT_USAGE =
+        "dtutil edit -service <service> -alias <alias> " +
+        FORMAT_SUBSTRING + "filename...";
+
+    @Override
+    public boolean validate() {
+      if (service == null) {
+        LOG.error("must pass -service field with dtutil edit command");
+        return false;
+      }
+      if (alias == null) {
+        LOG.error("must pass -alias field with dtutil edit command");
+        return false;
+      }
+      return true;
+    }
+
+    @Override
+    public void execute() throws Exception {
+      for (File tokenFile : tokenFiles) {
+        DtFileOperations.aliasTokenFile(
+            tokenFile, format, alias, service, getConf());
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return EDIT_USAGE;
+    }
+  }
+
   private class Append extends SubCommand {
     public static final String APPEND_USAGE =
         "dtutil append " + FORMAT_SUBSTRING + "filename...";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78b3a038/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
index 8efa589..5f8f830 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
@@ -158,6 +158,30 @@ public class TestDtUtilShell {
   }
 
   @Test
+  public void testEdit() throws Exception {
+    String oldService = SERVICE2.toString();
+    String newAlias = "newName:12345";
+    args = new String[] {"edit",
+        "-service", oldService, "-alias", newAlias, tokenFilename2};
+    rc = dt.run(args);
+    assertEquals("test simple edit exit code", 0, rc);
+    args = new String[] {"print", "-alias", oldService, tokenFilename2};
+    rc = dt.run(args);
+    assertEquals("test simple edit print old exit code", 0, rc);
+    assertTrue("test simple edit output kind old:\n" + outContent.toString(),
+               outContent.toString().contains(KIND.toString()));
+    assertTrue("test simple edit output service old:\n" + outContent.toString(),
+               outContent.toString().contains(oldService));
+    args = new String[] {"print", "-alias", newAlias, tokenFilename2};
+    rc = dt.run(args);
+    assertEquals("test simple edit print new exit code", 0, rc);
+    assertTrue("test simple edit output kind new:\n" + outContent.toString(),
+               outContent.toString().contains(KIND.toString()));
+    assertTrue("test simple edit output service new:\n" + outContent.toString(),
+               outContent.toString().contains(newAlias));
+  }
+
+  @Test
   public void testAppend() throws Exception {
     args = new String[] {"append", tokenFilename, tokenFilename2};
     rc = dt.run(args);


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


[30/47] hadoop git commit: Revert "HDFS-10224. Implement asynchronous rename for DistributedFileSystem. Contributed by Xiaobing Zhou"

Posted by ae...@apache.org.
Revert "HDFS-10224. Implement asynchronous rename for DistributedFileSystem.  Contributed by Xiaobing Zhou"

This reverts commit fc94810d3f537e51e826fc21ade7867892b9d8dc.


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

Branch: refs/heads/HDFS-1312
Commit: 106234d873c60fa52cd0d812fb1cdc0c6b998a6d
Parents: 4d36b22
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 18:09:55 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 18:09:55 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileSystem.java   |   1 +
 .../main/java/org/apache/hadoop/ipc/Client.java |  11 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |  34 +--
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     |   2 +-
 .../hadoop/hdfs/AsyncDistributedFileSystem.java | 110 --------
 .../hadoop/hdfs/DistributedFileSystem.java      |  22 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  45 +---
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 258 -------------------
 8 files changed, 20 insertions(+), 463 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/106234d8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 9e13a7a..0ecd8b7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -1252,6 +1252,7 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Renames Path src to Path dst
    * <ul>
+   * <li
    * <li>Fails if src is a file and dst is a directory.
    * <li>Fails if src is a directory and dst is a file.
    * <li>Fails if the parent of dst does not exist or is a file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/106234d8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index d59aeb89..f206861 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -119,8 +119,7 @@ public class Client implements AutoCloseable {
 
   private static final ThreadLocal<Integer> callId = new ThreadLocal<Integer>();
   private static final ThreadLocal<Integer> retryCount = new ThreadLocal<Integer>();
-  private static final ThreadLocal<Future<?>>
-      RETURN_RPC_RESPONSE = new ThreadLocal<>();
+  private static final ThreadLocal<Future<?>> returnValue = new ThreadLocal<>();
   private static final ThreadLocal<Boolean> asynchronousMode =
       new ThreadLocal<Boolean>() {
         @Override
@@ -131,8 +130,8 @@ public class Client implements AutoCloseable {
 
   @SuppressWarnings("unchecked")
   @Unstable
-  public static <T> Future<T> getReturnRpcResponse() {
-    return (Future<T>) RETURN_RPC_RESPONSE.get();
+  public static <T> Future<T> getReturnValue() {
+    return (Future<T>) returnValue.get();
   }
 
   /** Set call id and retry count for the next call. */
@@ -1397,7 +1396,7 @@ public class Client implements AutoCloseable {
         }
       };
 
-      RETURN_RPC_RESPONSE.set(returnFuture);
+      returnValue.set(returnFuture);
       return null;
     } else {
       return getRpcResponse(call, connection);
@@ -1411,7 +1410,7 @@ public class Client implements AutoCloseable {
    *          synchronous mode.
    */
   @Unstable
-  public static boolean isAsynchronousMode() {
+  static boolean isAsynchronousMode() {
     return asynchronousMode.get();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/106234d8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 8fcdb78..071e2e8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -26,9 +26,7 @@ import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.net.InetSocketAddress;
 import java.util.Map;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.net.SocketFactory;
@@ -37,7 +35,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataOutputOutputStream;
 import org.apache.hadoop.io.Writable;
@@ -70,9 +67,7 @@ import com.google.protobuf.TextFormat;
 @InterfaceStability.Evolving
 public class ProtobufRpcEngine implements RpcEngine {
   public static final Log LOG = LogFactory.getLog(ProtobufRpcEngine.class);
-  private static final ThreadLocal<Callable<?>>
-      RETURN_MESSAGE_CALLBACK = new ThreadLocal<>();
-
+  
   static { // Register the rpcRequest deserializer for WritableRpcEngine 
     org.apache.hadoop.ipc.Server.registerProtocolEngine(
         RPC.RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWrapper.class,
@@ -81,12 +76,6 @@ public class ProtobufRpcEngine implements RpcEngine {
 
   private static final ClientCache CLIENTS = new ClientCache();
 
-  @SuppressWarnings("unchecked")
-  @Unstable
-  public static <T> Callable<T> getReturnMessageCallback() {
-    return (Callable<T>) RETURN_MESSAGE_CALLBACK.get();
-  }
-
   public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
       InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
       SocketFactory factory, int rpcTimeout) throws IOException {
@@ -200,7 +189,7 @@ public class ProtobufRpcEngine implements RpcEngine {
      * the server.
      */
     @Override
-    public Object invoke(Object proxy, final Method method, Object[] args)
+    public Object invoke(Object proxy, Method method, Object[] args)
         throws ServiceException {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
@@ -262,23 +251,6 @@ public class ProtobufRpcEngine implements RpcEngine {
         LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
       }
       
-      if (Client.isAsynchronousMode()) {
-        final Future<RpcResponseWrapper> frrw = Client.getReturnRpcResponse();
-        Callable<Message> callback = new Callable<Message>() {
-          @Override
-          public Message call() throws Exception {
-            return getReturnMessage(method, frrw.get());
-          }
-        };
-        RETURN_MESSAGE_CALLBACK.set(callback);
-        return null;
-      } else {
-        return getReturnMessage(method, val);
-      }
-    }
-
-    private Message getReturnMessage(final Method method,
-        final RpcResponseWrapper rrw) throws ServiceException {
       Message prototype = null;
       try {
         prototype = getReturnProtoType(method);
@@ -288,7 +260,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       Message returnMessage;
       try {
         returnMessage = prototype.newBuilderForType()
-            .mergeFrom(rrw.theResponseRead).build();
+            .mergeFrom(val.theResponseRead).build();
 
         if (LOG.isTraceEnabled()) {
           LOG.trace(Thread.currentThread().getId() + ": Response <- " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/106234d8/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
index 6cf75c7..de4395e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
@@ -84,7 +84,7 @@ public class TestAsyncIPC {
         try {
           final long param = TestIPC.RANDOM.nextLong();
           TestIPC.call(client, param, server, conf);
-          Future<LongWritable> returnFuture = Client.getReturnRpcResponse();
+          Future<LongWritable> returnFuture = Client.getReturnValue();
           returnFutures.put(i, returnFuture);
           expectedValues.put(i, param);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/106234d8/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
deleted file mode 100644
index 37899aa..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AsyncDistributedFileSystem.java
+++ /dev/null
@@ -1,110 +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 java.io.IOException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-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.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
-import org.apache.hadoop.ipc.Client;
-
-import com.google.common.util.concurrent.AbstractFuture;
-
-/****************************************************************
- * Implementation of the asynchronous distributed file system.
- * This instance of this class is the way end-user code interacts
- * with a Hadoop DistributedFileSystem in an asynchronous manner.
- *
- *****************************************************************/
-@Unstable
-public class AsyncDistributedFileSystem {
-
-  private final DistributedFileSystem dfs;
-
-  AsyncDistributedFileSystem(final DistributedFileSystem dfs) {
-    this.dfs = dfs;
-  }
-
-  static <T> Future<T> getReturnValue() {
-    final Callable<T> returnValueCallback = ClientNamenodeProtocolTranslatorPB
-        .getReturnValueCallback();
-    Future<T> returnFuture = new AbstractFuture<T>() {
-      public T get() throws InterruptedException, ExecutionException {
-        try {
-          set(returnValueCallback.call());
-        } catch (Exception e) {
-          setException(e);
-        }
-        return super.get();
-      }
-    };
-    return returnFuture;
-  }
-
-  /**
-   * Renames Path src to Path dst
-   * <ul>
-   * <li>Fails if src is a file and dst is a directory.
-   * <li>Fails if src is a directory and dst is a file.
-   * <li>Fails if the parent of dst does not exist or is a file.
-   * </ul>
-   * <p>
-   * If OVERWRITE option is not passed as an argument, rename fails if the dst
-   * already exists.
-   * <p>
-   * If OVERWRITE option is passed as an argument, rename overwrites the dst if
-   * it is a file or an empty directory. Rename fails if dst is a non-empty
-   * directory.
-   * <p>
-   * Note that atomicity of rename is dependent on the file system
-   * implementation. Please refer to the file system documentation for details.
-   * This default implementation is non atomic.
-   *
-   * @param src
-   *          path to be renamed
-   * @param dst
-   *          new path after rename
-   * @throws IOException
-   *           on failure
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<Void> rename(Path src, Path dst,
-      final Options.Rename... options) throws IOException {
-    dfs.getFsStatistics().incrementWriteOps(1);
-
-    final Path absSrc = dfs.fixRelativePart(src);
-    final Path absDst = dfs.fixRelativePart(dst);
-
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().rename(dfs.getPathName(absSrc), dfs.getPathName(absDst),
-          options);
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/106234d8/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 0ae4d70..5e54edd 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
@@ -31,7 +31,6 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
@@ -205,7 +204,7 @@ public class DistributedFileSystem extends FileSystem {
    * @return path component of {file}
    * @throws IllegalArgumentException if URI does not belong to this DFS
    */
-  String getPathName(Path file) {
+  private String getPathName(Path file) {
     checkPath(file);
     String result = file.toUri().getPath();
     if (!DFSUtilClient.isValidName(result)) {
@@ -2510,23 +2509,4 @@ public class DistributedFileSystem extends FileSystem {
     }
     return ret;
   }
-
-  private final AsyncDistributedFileSystem adfs =
-      new AsyncDistributedFileSystem(this);
-
-  /** @return an {@link AsyncDistributedFileSystem} object. */
-  @Unstable
-  public AsyncDistributedFileSystem getAsyncDistributedFileSystem() {
-    return adfs;
-  }
-
-  @Override
-  protected Path fixRelativePart(Path p) {
-    return super.fixRelativePart(p);
-  }
-
-  Statistics getFsStatistics() {
-    return statistics;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/106234d8/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 f4074b6..513a5e3 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
@@ -24,14 +24,11 @@ import java.util.EnumSet;
 import java.util.List;
 
 import com.google.common.collect.Lists;
-import java.util.concurrent.Callable;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
-import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -58,7 +55,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
@@ -139,6 +135,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Recove
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
@@ -156,15 +153,13 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPer
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
-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.EncryptionZonesProtos;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.*;
 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;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
@@ -182,9 +177,8 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.RPC;
@@ -196,9 +190,12 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenReque
 import org.apache.hadoop.security.token.Token;
 
 import com.google.protobuf.ByteString;
-import com.google.protobuf.Message;
 import com.google.protobuf.ServiceException;
 
+import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
+    .EncryptionZoneProto;
+
 /**
  * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
  * while translating from the parameter types used in ClientProtocol to the
@@ -209,8 +206,6 @@ import com.google.protobuf.ServiceException;
 public class ClientNamenodeProtocolTranslatorPB implements
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
-  private static final ThreadLocal<Callable<?>>
-      RETURN_VALUE_CALLBACK = new ThreadLocal<>();
 
   static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
       GetServerDefaultsRequestProto.newBuilder().build();
@@ -244,12 +239,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
     rpcProxy = proxy;
   }
 
-  @SuppressWarnings("unchecked")
-  @Unstable
-  public static <T> Callable<T> getReturnValueCallback() {
-    return (Callable<T>) RETURN_VALUE_CALLBACK.get();
-  }
-
   @Override
   public void close() {
     RPC.stopProxy(rpcProxy);
@@ -486,7 +475,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
     RenameRequestProto req = RenameRequestProto.newBuilder()
         .setSrc(src)
         .setDst(dst).build();
-
     try {
       return rpcProxy.rename(null, req).getResult();
     } catch (ServiceException e) {
@@ -511,22 +499,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         setDst(dst).setOverwriteDest(overwrite).
         build();
     try {
-      if (Client.isAsynchronousMode()) {
-        rpcProxy.rename2(null, req);
-
-        final Callable<Message> returnMessageCallback = ProtobufRpcEngine
-            .getReturnMessageCallback();
-        Callable<Void> callBack = new Callable<Void>() {
-          @Override
-          public Void call() throws Exception {
-            returnMessageCallback.call();
-            return null;
-          }
-        };
-        RETURN_VALUE_CALLBACK.set(callBack);
-      } else {
-        rpcProxy.rename2(null, req);
-      }
+      rpcProxy.rename2(null, req);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/106234d8/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
deleted file mode 100644
index 9322e1a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncDFSRename.java
+++ /dev/null
@@ -1,258 +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.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.HashMap;
-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.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestAsyncDFSRename {
-  final Path asyncRenameDir = new Path("/test/async_rename/");
-  public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
-  final private static Configuration CONF = new HdfsConfiguration();
-
-  final private static String GROUP1_NAME = "group1";
-  final private static String GROUP2_NAME = "group2";
-  final private static String USER1_NAME = "user1";
-  private static final UserGroupInformation USER1;
-
-  private MiniDFSCluster gCluster;
-
-  static {
-    // explicitly turn on permission checking
-    CONF.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-
-    // create fake mapping for the groups
-    Map<String, String[]> u2g_map = new HashMap<String, String[]>(1);
-    u2g_map.put(USER1_NAME, new String[] { GROUP1_NAME, GROUP2_NAME });
-    DFSTestUtil.updateConfWithFakeGroupMapping(CONF, u2g_map);
-
-    // Initiate all four users
-    USER1 = UserGroupInformation.createUserForTesting(USER1_NAME, new String[] {
-        GROUP1_NAME, GROUP2_NAME });
-  }
-
-  @Before
-  public void setUp() throws IOException {
-    gCluster = new MiniDFSCluster.Builder(CONF).numDataNodes(3).build();
-    gCluster.waitActive();
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    if (gCluster != null) {
-      gCluster.shutdown();
-      gCluster = null;
-    }
-  }
-
-  static int countLease(MiniDFSCluster cluster) {
-    return TestDFSRename.countLease(cluster);
-  }
-
-  void list(DistributedFileSystem dfs, String name) throws IOException {
-    FileSystem.LOG.info("\n\n" + name);
-    for (FileStatus s : dfs.listStatus(asyncRenameDir)) {
-      FileSystem.LOG.info("" + s.getPath());
-    }
-  }
-
-  static void createFile(DistributedFileSystem dfs, Path f) throws IOException {
-    DataOutputStream a_out = dfs.create(f);
-    a_out.writeBytes("something");
-    a_out.close();
-  }
-
-  /**
-   * Check the blocks of dst file are cleaned after rename with overwrite
-   * Restart NN to check the rename successfully
-   */
-  @Test
-  public void testAsyncRenameWithOverwrite() throws Exception {
-    final short replFactor = 2;
-    final long blockSize = 512;
-    Configuration conf = new Configuration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
-        replFactor).build();
-    cluster.waitActive();
-    DistributedFileSystem dfs = cluster.getFileSystem();
-    AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
-
-    try {
-
-      long fileLen = blockSize * 3;
-      String src = "/foo/src";
-      String dst = "/foo/dst";
-      String src2 = "/foo/src2";
-      String dst2 = "/foo/dst2";
-      Path srcPath = new Path(src);
-      Path dstPath = new Path(dst);
-      Path srcPath2 = new Path(src2);
-      Path dstPath2 = new Path(dst2);
-
-      DFSTestUtil.createFile(dfs, srcPath, fileLen, replFactor, 1);
-      DFSTestUtil.createFile(dfs, dstPath, fileLen, replFactor, 1);
-      DFSTestUtil.createFile(dfs, srcPath2, fileLen, replFactor, 1);
-      DFSTestUtil.createFile(dfs, dstPath2, fileLen, replFactor, 1);
-
-      LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(
-          cluster.getNameNode(), dst, 0, fileLen);
-      LocatedBlocks lbs2 = NameNodeAdapter.getBlockLocations(
-          cluster.getNameNode(), dst2, 0, fileLen);
-      BlockManager bm = NameNodeAdapter.getNamesystem(cluster.getNameNode())
-          .getBlockManager();
-      assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock()
-          .getLocalBlock()) != null);
-      assertTrue(bm.getStoredBlock(lbs2.getLocatedBlocks().get(0).getBlock()
-          .getLocalBlock()) != null);
-
-      Future<Void> retVal1 = adfs.rename(srcPath, dstPath, Rename.OVERWRITE);
-      Future<Void> retVal2 = adfs.rename(srcPath2, dstPath2, Rename.OVERWRITE);
-      retVal1.get();
-      retVal2.get();
-
-      assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock()
-          .getLocalBlock()) == null);
-      assertTrue(bm.getStoredBlock(lbs2.getLocatedBlocks().get(0).getBlock()
-          .getLocalBlock()) == null);
-
-      // Restart NN and check the rename successfully
-      cluster.restartNameNodes();
-      assertFalse(dfs.exists(srcPath));
-      assertTrue(dfs.exists(dstPath));
-      assertFalse(dfs.exists(srcPath2));
-      assertTrue(dfs.exists(dstPath2));
-    } finally {
-      if (dfs != null) {
-        dfs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
-  @Test
-  public void testConcurrentAsyncRenameWithOverwrite() throws Exception {
-    final short replFactor = 2;
-    final long blockSize = 512;
-    final Path renameDir = new Path(
-        "/test/concurrent_reanme_with_overwrite_dir/");
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
-        .build();
-    cluster.waitActive();
-    DistributedFileSystem dfs = cluster.getFileSystem();
-    AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
-    int count = 1000;
-
-    try {
-      long fileLen = blockSize * 3;
-      assertTrue(dfs.mkdirs(renameDir));
-
-      Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
-
-      // concurrently invoking many rename
-      for (int i = 0; i < count; i++) {
-        Path src = new Path(renameDir, "src" + i);
-        Path dst = new Path(renameDir, "dst" + i);
-        DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
-        DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
-        Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-        returnFutures.put(i, returnFuture);
-      }
-
-      // wait for completing the calls
-      for (int i = 0; i < count; i++) {
-        returnFutures.get(i).get();
-      }
-
-      // Restart NN and check the rename successfully
-      cluster.restartNameNodes();
-
-      // very the src dir should not exist, dst should
-      for (int i = 0; i < count; i++) {
-        Path src = new Path(renameDir, "src" + i);
-        Path dst = new Path(renameDir, "dst" + i);
-        assertFalse(dfs.exists(src));
-        assertTrue(dfs.exists(dst));
-      }
-    } finally {
-      dfs.delete(renameDir, true);
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
-  @Test
-  public void testAsyncRenameWithException() throws Exception {
-    FileSystem rootFs = FileSystem.get(CONF);
-    final Path renameDir = new Path("/test/async_rename_exception/");
-    final Path src = new Path(renameDir, "src");
-    final Path dst = new Path(renameDir, "dst");
-    rootFs.mkdirs(src);
-
-    AsyncDistributedFileSystem adfs = USER1
-        .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
-          @Override
-          public AsyncDistributedFileSystem run() throws Exception {
-            return gCluster.getFileSystem().getAsyncDistributedFileSystem();
-          }
-        });
-
-    try {
-      Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-      returnFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src);
-    }
-  }
-
-  private void checkPermissionDenied(final Exception e, final Path dir) {
-    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(USER1_NAME));
-    assertTrue("Permission denied messages must carry the path parent", e
-        .getMessage().contains(dir.getParent().toUri().getPath()));
-  }
-}
\ No newline at end of file


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


[43/47] hadoop git commit: YARN-4525. Fix bug in RLESparseResourceAllocation.getRangeOverlapping(). (Ishai Menache and Carlo Curino via asuresh)

Posted by ae...@apache.org.
YARN-4525. Fix bug in RLESparseResourceAllocation.getRangeOverlapping(). (Ishai Menache and Carlo Curino via asuresh)


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

Branch: refs/heads/HDFS-1312
Commit: 3a154f75ed85d864b3ffd35818992418f2b6aa59
Parents: 7a9b737
Author: Arun Suresh <as...@apache.org>
Authored: Mon Jun 6 21:18:32 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Jun 6 21:18:32 2016 -0700

----------------------------------------------------------------------
 .../RLESparseResourceAllocation.java            |  6 +++++-
 .../TestRLESparseResourceAllocation.java        | 22 ++++++++++++++++++++
 2 files changed, 27 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a154f75/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
index 63defb5..c18a93e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/RLESparseResourceAllocation.java
@@ -510,7 +510,11 @@ public class RLESparseResourceAllocation {
           long previous = a.floorKey(start);
           a = a.tailMap(previous, true);
         }
-        a = a.headMap(end, true);
+
+        if (end < a.lastKey()) {
+          a = a.headMap(end, true);
+        }
+
       }
       RLESparseResourceAllocation ret =
           new RLESparseResourceAllocation(a, resourceCalculator);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a154f75/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
index b526484..f8d2a4a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestRLESparseResourceAllocation.java
@@ -283,6 +283,28 @@ public class TestRLESparseResourceAllocation {
   }
 
   @Test
+  public void testRangeOverlapping() {
+    ResourceCalculator resCalc = new DefaultResourceCalculator();
+
+    RLESparseResourceAllocation r =
+        new RLESparseResourceAllocation(resCalc);
+    int[] alloc = {10, 10, 10, 10, 10, 10};
+    int start = 100;
+    Set<Entry<ReservationInterval, Resource>> inputs =
+        generateAllocation(start, alloc, false).entrySet();
+    for (Entry<ReservationInterval, Resource> ip : inputs) {
+      r.addInterval(ip.getKey(), ip.getValue());
+    }
+    long s = r.getEarliestStartTime();
+    long d = r.getLatestNonNullTime();
+
+    // tries to trigger "out-of-range" bug
+    r =  r.getRangeOverlapping(s, d);
+    r = r.getRangeOverlapping(s-1, d-1);
+    r = r.getRangeOverlapping(s+1, d+1);
+  }
+
+  @Test
   public void testBlocks() {
     ResourceCalculator resCalc = new DefaultResourceCalculator();
 


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


[16/47] hadoop git commit: YARN-5165. Fix NoOvercommitPolicy to take advantage of RLE representation of plan. (Carlo Curino via asuresh)

Posted by ae...@apache.org.
YARN-5165. Fix NoOvercommitPolicy to take advantage of RLE representation of plan. (Carlo Curino via asuresh)


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

Branch: refs/heads/HDFS-1312
Commit: db54670e83a84c1d7deff2c225725687cf9e5f14
Parents: f10ebc6
Author: Arun Suresh <as...@apache.org>
Authored: Fri Jun 3 14:49:32 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Fri Jun 3 14:49:32 2016 -0700

----------------------------------------------------------------------
 .../reservation/NoOverCommitPolicy.java         | 38 ++++++++------------
 .../planning/TestSimpleCapacityReplanner.java   |  2 +-
 2 files changed, 15 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/db54670e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
index 119520b..814d4b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/NoOverCommitPolicy.java
@@ -21,11 +21,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.MismatchedUserException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ResourceOverCommitException;
-import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * This policy enforce a simple physical cluster capacity constraints, by
@@ -52,29 +50,21 @@ public class NoOverCommitPolicy implements SharingPolicy {
               + oldReservation.getUser() + " != " + reservation.getUser());
     }
 
-    long startTime = reservation.getStartTime();
-    long endTime = reservation.getEndTime();
-    long step = plan.getStep();
+    RLESparseResourceAllocation available = plan.getAvailableResourceOverTime(
+        reservation.getUser(), reservation.getReservationId(),
+        reservation.getStartTime(), reservation.getEndTime());
 
-    // for every instant in time, check we are respecting cluster capacity
-    for (long t = startTime; t < endTime; t += step) {
-      Resource currExistingAllocTot = plan.getTotalCommittedResources(t);
-      Resource currNewAlloc = reservation.getResourcesAtTime(t);
-      Resource currOldAlloc = Resource.newInstance(0, 0);
-      if (oldReservation != null) {
-        oldReservation.getResourcesAtTime(t);
-      }
-      // check the cluster is never over committed
-      // currExistingAllocTot + currNewAlloc - currOldAlloc >
-      // capPlan.getTotalCapacity()
-      if (Resources.greaterThan(plan.getResourceCalculator(), plan
-          .getTotalCapacity(), Resources.subtract(
-          Resources.add(currExistingAllocTot, currNewAlloc), currOldAlloc),
-          plan.getTotalCapacity())) {
-        throw new ResourceOverCommitException("Resources at time " + t
-            + " would be overcommitted by " + "accepting reservation: "
-            + reservation.getReservationId());
-      }
+    // test the reservation does not exceed what is available
+    try {
+      RLESparseResourceAllocation
+          .merge(plan.getResourceCalculator(), plan.getTotalCapacity(),
+              available, reservation.getResourcesOverTime(),
+              RLESparseResourceAllocation.RLEOperator.subtractTestNonNegative,
+              reservation.getStartTime(), reservation.getEndTime());
+    } catch (PlanningException p) {
+      throw new ResourceOverCommitException(
+          "Resources at time " + " would be overcommitted by "
+              + "accepting reservation: " + reservation.getReservationId());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db54670e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
index d762b36..e01608c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/TestSimpleCapacityReplanner.java
@@ -53,7 +53,7 @@ public class TestSimpleCapacityReplanner {
   @Test
   public void testReplanningPlanCapacityLoss() throws PlanningException {
 
-    Resource clusterCapacity = Resource.newInstance(100 * 1024, 10);
+    Resource clusterCapacity = Resource.newInstance(100 * 1024, 100);
     Resource minAlloc = Resource.newInstance(1024, 1);
     Resource maxAlloc = Resource.newInstance(1024 * 8, 8);
 


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


[36/47] hadoop git commit: Revert "Revert "HDFS-10431 Refactor and speedup TestAsyncDFSRename. Contributed by Xiaobing Zhou""

Posted by ae...@apache.org.
Revert "Revert "HDFS-10431 Refactor and speedup TestAsyncDFSRename.  Contributed by Xiaobing Zhou""

This reverts commit 5ee5912ebd541d5b4c33ecd46dfdebe1e23b56c3.


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

Branch: refs/heads/HDFS-1312
Commit: db41e6d285a3b425ffd7c11c7baa8253c7929439
Parents: b3d81f3
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 6 16:31:34 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 6 16:31:34 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    | 233 +++++++-
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 563 ++++---------------
 2 files changed, 313 insertions(+), 483 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/db41e6d2/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
index 67262dd..ddcf492 100644
--- 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
@@ -29,13 +29,16 @@ 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.assertFalse;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
@@ -43,15 +46,21 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
 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.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -63,21 +72,28 @@ import com.google.common.collect.Lists;
  * */
 public class TestAsyncDFS {
   public static final Log LOG = LogFactory.getLog(TestAsyncDFS.class);
-  private static final int NUM_TESTS = 1000;
+  private final short replFactor = 1;
+  private final long blockSize = 512;
+  private long fileLen = blockSize * 3;
+  private final long seed = Time.now();
+  private final Random r = new Random(seed);
+  private final PermissionGenerator permGenerator = new PermissionGenerator(r);
+  private static final int NUM_TESTS = 50;
   private static final int NUM_NN_HANDLER = 10;
-  private static final int ASYNC_CALL_LIMIT = 100;
+  private static final int ASYNC_CALL_LIMIT = 1000;
 
   private Configuration conf;
   private MiniDFSCluster cluster;
   private FileSystem fs;
+  private AsyncDistributedFileSystem adfs;
 
   @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);
+    // explicitly turn on permission checking
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     // set the limit of max async calls
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
         ASYNC_CALL_LIMIT);
@@ -86,6 +102,7 @@ public class TestAsyncDFS {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
     fs = FileSystem.get(conf);
+    adfs = cluster.getFileSystem().getAsyncDistributedFileSystem();
   }
 
   @After
@@ -130,13 +147,9 @@ public class TestAsyncDFS {
     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++) {
+    final Path[] paths = new Path[NUM_TESTS];
+    for (int i = 0; i < NUM_TESTS; i++) {
       paths[i] = new Path(parent, "acl" + i);
       FileSystem.mkdirs(fs, paths[i],
           FsPermission.createImmutable((short) 0750));
@@ -153,7 +166,7 @@ public class TestAsyncDFS {
     int start = 0, end = 0;
     try {
       // test setAcl
-      for (int i = 0; i < count; i++) {
+      for (int i = 0; i < NUM_TESTS; i++) {
         for (;;) {
           try {
             Future<Void> retFuture = adfs.setAcl(paths[i], aclSpec);
@@ -166,12 +179,12 @@ public class TestAsyncDFS {
           }
         }
       }
-      waitForAclReturnValues(setAclRetFutures, end, count);
+      waitForAclReturnValues(setAclRetFutures, end, NUM_TESTS);
 
       // test getAclStatus
       start = 0;
       end = 0;
-      for (int i = 0; i < count; i++) {
+      for (int i = 0; i < NUM_TESTS; i++) {
         for (;;) {
           try {
             Future<AclStatus> retFuture = adfs.getAclStatus(paths[i]);
@@ -185,13 +198,23 @@ public class TestAsyncDFS {
           }
         }
       }
-      waitForAclReturnValues(getAclRetFutures, end, count, paths,
+      waitForAclReturnValues(getAclRetFutures, end, NUM_TESTS, paths,
           expectedAclSpec);
     } catch (Exception e) {
       throw e;
     }
   }
 
+  static void waitForReturnValues(final Map<Integer, Future<Void>> retFutures,
+      final int start, final int end)
+      throws InterruptedException, ExecutionException {
+    LOG.info(String.format("calling waitForReturnValues [%d, %d)", start, end));
+    for (int i = start; i < end; i++) {
+      LOG.info("calling Future#get #" + i);
+      retFutures.get(i).get();
+    }
+  }
+
   private void waitForAclReturnValues(
       final Map<Integer, Future<Void>> aclRetFutures, final int start,
       final int end) throws InterruptedException, ExecutionException {
@@ -266,9 +289,12 @@ public class TestAsyncDFS {
 
     final Path parent = new Path("/test/async_api_exception/");
     final Path aclDir = new Path(parent, "aclDir");
-    fs.mkdirs(aclDir, FsPermission.createImmutable((short) 0770));
+    final Path src = new Path(parent, "src");
+    final Path dst = new Path(parent, "dst");
+    fs.mkdirs(aclDir, FsPermission.createImmutable((short) 0700));
+    fs.mkdirs(src);
 
-    AsyncDistributedFileSystem adfs = ugi1
+    AsyncDistributedFileSystem adfs1 = ugi1
         .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
           @Override
           public AsyncDistributedFileSystem run() throws Exception {
@@ -277,9 +303,36 @@ public class TestAsyncDFS {
         });
 
     Future<Void> retFuture;
+    // test rename
+    try {
+      retFuture = adfs1.rename(src, dst, Rename.OVERWRITE);
+      retFuture.get();
+    } catch (ExecutionException e) {
+      checkPermissionDenied(e, src, user1);
+      assertTrue("Permission denied messages must carry the path parent", e
+          .getMessage().contains(src.getParent().toUri().getPath()));
+    }
+
+    // test setPermission
+    FsPermission fsPerm = new FsPermission(permGenerator.next());
+    try {
+      retFuture = adfs1.setPermission(src, fsPerm);
+      retFuture.get();
+    } catch (ExecutionException e) {
+      checkPermissionDenied(e, src, user1);
+    }
+
+    // test setOwner
+    try {
+      retFuture = adfs1.setOwner(src, "user1", "group2");
+      retFuture.get();
+    } catch (ExecutionException e) {
+      checkPermissionDenied(e, src, user1);
+    }
+
     // test setAcl
     try {
-      retFuture = adfs.setAcl(aclDir,
+      retFuture = adfs1.setAcl(aclDir,
           Lists.newArrayList(aclEntry(ACCESS, USER, ALL)));
       retFuture.get();
       fail("setAcl should fail with permission denied");
@@ -289,7 +342,7 @@ public class TestAsyncDFS {
 
     // test getAclStatus
     try {
-      Future<AclStatus> aclRetFuture = adfs.getAclStatus(aclDir);
+      Future<AclStatus> aclRetFuture = adfs1.getAclStatus(aclDir);
       aclRetFuture.get();
       fail("getAclStatus should fail with permission denied");
     } catch (ExecutionException e) {
@@ -307,4 +360,148 @@ public class TestAsyncDFS {
     assertTrue("Permission denied messages must carry the name of the path",
         e.getMessage().contains(dir.getName()));
   }
+
+
+  @Test(timeout = 120000)
+  public void testConcurrentAsyncAPI() throws Exception {
+    String group1 = "group1";
+    String group2 = "group2";
+    String user1 = "user1";
+
+    // 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);
+
+    // prepare for test
+    final Path parent = new Path(
+        String.format("/test/%s/", "testConcurrentAsyncAPI"));
+    final Path[] srcs = new Path[NUM_TESTS];
+    final Path[] dsts = new Path[NUM_TESTS];
+    short[] permissions = new short[NUM_TESTS];
+    for (int i = 0; i < NUM_TESTS; i++) {
+      srcs[i] = new Path(parent, "src" + i);
+      dsts[i] = new Path(parent, "dst" + i);
+      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
+      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
+      assertTrue(fs.exists(srcs[i]));
+      assertTrue(fs.getFileStatus(srcs[i]).isFile());
+      assertTrue(fs.exists(dsts[i]));
+      assertTrue(fs.getFileStatus(dsts[i]).isFile());
+      permissions[i] = permGenerator.next();
+    }
+
+    Map<Integer, Future<Void>> renameRetFutures =
+        new HashMap<Integer, Future<Void>>();
+    Map<Integer, Future<Void>> permRetFutures =
+        new HashMap<Integer, Future<Void>>();
+    Map<Integer, Future<Void>> ownerRetFutures =
+        new HashMap<Integer, Future<Void>>();
+    int start = 0, end = 0;
+    // test rename
+    for (int i = 0; i < NUM_TESTS; i++) {
+      for (;;) {
+        try {
+          Future<Void> returnFuture = adfs.rename(srcs[i], dsts[i],
+              Rename.OVERWRITE);
+          renameRetFutures.put(i, returnFuture);
+          break;
+        } catch (AsyncCallLimitExceededException e) {
+          start = end;
+          end = i;
+          waitForReturnValues(renameRetFutures, start, end);
+        }
+      }
+    }
+
+    // wait for completing the calls
+    waitForAclReturnValues(renameRetFutures, end, NUM_TESTS);
+
+    // verify the src should not exist, dst should
+    for (int i = 0; i < NUM_TESTS; i++) {
+      assertFalse(fs.exists(srcs[i]));
+      assertTrue(fs.exists(dsts[i]));
+    }
+
+    // test permissions
+    for (int i = 0; i < NUM_TESTS; i++) {
+      for (;;) {
+        try {
+          Future<Void> retFuture = adfs.setPermission(dsts[i],
+              new FsPermission(permissions[i]));
+          permRetFutures.put(i, retFuture);
+          break;
+        } catch (AsyncCallLimitExceededException e) {
+          start = end;
+          end = i;
+          waitForReturnValues(permRetFutures, start, end);
+        }
+      }
+    }
+    // wait for completing the calls
+    waitForAclReturnValues(permRetFutures, end, NUM_TESTS);
+
+    // verify the permission
+    for (int i = 0; i < NUM_TESTS; i++) {
+      assertTrue(fs.exists(dsts[i]));
+      FsPermission fsPerm = new FsPermission(permissions[i]);
+      checkAccessPermissions(fs.getFileStatus(dsts[i]), fsPerm.getUserAction());
+    }
+
+    // test setOwner
+    start = 0;
+    end = 0;
+    for (int i = 0; i < NUM_TESTS; i++) {
+      for (;;) {
+        try {
+          Future<Void> retFuture = adfs.setOwner(dsts[i], "user1", "group2");
+          ownerRetFutures.put(i, retFuture);
+          break;
+        } catch (AsyncCallLimitExceededException e) {
+          start = end;
+          end = i;
+          waitForReturnValues(ownerRetFutures, start, end);
+        }
+      }
+    }
+    // wait for completing the calls
+    waitForAclReturnValues(ownerRetFutures, end, NUM_TESTS);
+
+    // verify the owner
+    for (int i = 0; i < NUM_TESTS; i++) {
+      assertTrue(fs.exists(dsts[i]));
+      assertTrue("user1".equals(fs.getFileStatus(dsts[i]).getOwner()));
+      assertTrue("group2".equals(fs.getFileStatus(dsts[i]).getGroup()));
+    }
+  }
+
+  static void checkAccessPermissions(FileStatus stat, FsAction mode)
+      throws IOException {
+    checkAccessPermissions(UserGroupInformation.getCurrentUser(), stat, mode);
+  }
+
+  static void checkAccessPermissions(final UserGroupInformation ugi,
+      FileStatus stat, FsAction mode) throws IOException {
+    FsPermission perm = stat.getPermission();
+    String user = ugi.getShortUserName();
+    List<String> groups = Arrays.asList(ugi.getGroupNames());
+
+    if (user.equals(stat.getOwner())) {
+      if (perm.getUserAction().implies(mode)) {
+        return;
+      }
+    } else if (groups.contains(stat.getGroup())) {
+      if (perm.getGroupAction().implies(mode)) {
+        return;
+      }
+    } else {
+      if (perm.getOtherAction().implies(mode)) {
+        return;
+      }
+    }
+    throw new AccessControlException(String.format(
+        "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat
+            .getPath(), stat.getOwner(), stat.getGroup(),
+        stat.isDirectory() ? "d" : "-", perm));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db41e6d2/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..8d3e509 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
@@ -19,14 +19,11 @@ package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
@@ -34,521 +31,157 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Time;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestAsyncDFSRename {
   public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
-  private final long seed = Time.now();
-  private final Random r = new Random(seed);
-  private final PermissionGenerator permGenerator = new PermissionGenerator(r);
-  private final short replFactor = 2;
+  private final short replFactor = 1;
   private final long blockSize = 512;
   private long fileLen = blockSize * 3;
-
-  /**
-   * Check the blocks of dst file are cleaned after rename with overwrite
-   * Restart NN to check the rename successfully
-   */
-  @Test(timeout = 60000)
-  public void testAsyncRenameWithOverwrite() throws Exception {
-    Configuration conf = new Configuration();
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
-        replFactor).build();
+  private static final int NUM_TESTS = 50;
+  private static final int NUM_NN_HANDLER = 10;
+  private static final int ASYNC_CALL_LIMIT = 1000;
+
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private FileSystem fs;
+  private AsyncDistributedFileSystem adfs;
+
+  @Before
+  public void setup() throws IOException {
+    conf = new HdfsConfiguration();
+    // 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(1).build();
     cluster.waitActive();
-    DistributedFileSystem dfs = cluster.getFileSystem();
-    AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
-
-    try {
-      String src = "/foo/src";
-      String dst = "/foo/dst";
-      String src2 = "/foo/src2";
-      String dst2 = "/foo/dst2";
-      Path srcPath = new Path(src);
-      Path dstPath = new Path(dst);
-      Path srcPath2 = new Path(src2);
-      Path dstPath2 = new Path(dst2);
-
-      DFSTestUtil.createFile(dfs, srcPath, fileLen, replFactor, 1);
-      DFSTestUtil.createFile(dfs, dstPath, fileLen, replFactor, 1);
-      DFSTestUtil.createFile(dfs, srcPath2, fileLen, replFactor, 1);
-      DFSTestUtil.createFile(dfs, dstPath2, fileLen, replFactor, 1);
-
-      LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(
-          cluster.getNameNode(), dst, 0, fileLen);
-      LocatedBlocks lbs2 = NameNodeAdapter.getBlockLocations(
-          cluster.getNameNode(), dst2, 0, fileLen);
-      BlockManager bm = NameNodeAdapter.getNamesystem(cluster.getNameNode())
-          .getBlockManager();
-      assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock()
-          .getLocalBlock()) != null);
-      assertTrue(bm.getStoredBlock(lbs2.getLocatedBlocks().get(0).getBlock()
-          .getLocalBlock()) != null);
-
-      Future<Void> retVal1 = adfs.rename(srcPath, dstPath, Rename.OVERWRITE);
-      Future<Void> retVal2 = adfs.rename(srcPath2, dstPath2, Rename.OVERWRITE);
-      retVal1.get();
-      retVal2.get();
-
-      assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock()
-          .getLocalBlock()) == null);
-      assertTrue(bm.getStoredBlock(lbs2.getLocatedBlocks().get(0).getBlock()
-          .getLocalBlock()) == null);
+    fs = FileSystem.get(conf);
+    adfs = cluster.getFileSystem().getAsyncDistributedFileSystem();
+  }
 
-      // Restart NN and check the rename successfully
-      cluster.restartNameNodes();
-      assertFalse(dfs.exists(srcPath));
-      assertTrue(dfs.exists(dstPath));
-      assertFalse(dfs.exists(srcPath2));
-      assertTrue(dfs.exists(dstPath2));
-    } finally {
-      if (dfs != null) {
-        dfs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
+  @After
+  public void tearDown() throws IOException {
+    if (fs != null) {
+      fs.close();
+      fs = null;
+    }
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
     }
   }
 
   @Test(timeout = 60000)
   public void testCallGetReturnValueMultipleTimes() throws Exception {
-    final Path renameDir = new Path(
-        "/test/testCallGetReturnValueMultipleTimes/");
-    final Configuration conf = new HdfsConfiguration();
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY, 200);
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(2).build();
-    cluster.waitActive();
-    final DistributedFileSystem dfs = cluster.getFileSystem();
-    final AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
-    final int count = 100;
-    final Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
+    final Path parent = new Path("/test/testCallGetReturnValueMultipleTimes/");
+    assertTrue(fs.mkdirs(parent));
 
-    assertTrue(dfs.mkdirs(renameDir));
-
-    try {
-      // concurrently invoking many rename
-      for (int i = 0; i < count; i++) {
-        Path src = new Path(renameDir, "src" + i);
-        Path dst = new Path(renameDir, "dst" + i);
-        DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
-        DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
-        Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-        returnFutures.put(i, returnFuture);
-      }
-
-      for (int i = 0; i < 5; i++) {
-        verifyCallGetReturnValueMultipleTimes(returnFutures, count, cluster,
-            renameDir, dfs);
-      }
-    } finally {
-      if (dfs != null) {
-        dfs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
+    // prepare test
+    final Path[] srcs = new Path[NUM_TESTS];
+    final Path[] dsts = new Path[NUM_TESTS];
+    for (int i = 0; i < NUM_TESTS; i++) {
+      srcs[i] = new Path(parent, "src" + i);
+      dsts[i] = new Path(parent, "dst" + i);
+      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
+      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
     }
-  }
 
-  private void verifyCallGetReturnValueMultipleTimes(
-      Map<Integer, Future<Void>> returnFutures, int count,
-      MiniDFSCluster cluster, Path renameDir, DistributedFileSystem dfs)
-      throws InterruptedException, ExecutionException, IOException {
-    // wait for completing the calls
-    for (int i = 0; i < count; i++) {
-      returnFutures.get(i).get();
+    // concurrently invoking many rename
+    final Map<Integer, Future<Void>> reFutures =
+        new HashMap<Integer, Future<Void>>();
+    for (int i = 0; i < NUM_TESTS; i++) {
+      Future<Void> retFuture = adfs.rename(srcs[i], dsts[i],
+          Rename.OVERWRITE);
+      reFutures.put(i, retFuture);
     }
 
-    // Restart NN and check the rename successfully
-    cluster.restartNameNodes();
+    assertEquals(NUM_TESTS, reFutures.size());
 
-    // very the src dir should not exist, dst should
-    for (int i = 0; i < count; i++) {
-      Path src = new Path(renameDir, "src" + i);
-      Path dst = new Path(renameDir, "dst" + i);
-      assertFalse(dfs.exists(src));
-      assertTrue(dfs.exists(dst));
+    for (int i = 0; i < 5; i++) {
+      verifyCallGetReturnValueMultipleTimes(reFutures, srcs, dsts);
     }
   }
 
-  @Test
-  public void testConservativeConcurrentAsyncRenameWithOverwrite()
-      throws Exception {
-    internalTestConcurrentAsyncRenameWithOverwrite(100,
-        "testAggressiveConcurrentAsyncRenameWithOverwrite");
-  }
-
-  @Test(timeout = 60000)
-  public void testAggressiveConcurrentAsyncRenameWithOverwrite()
-      throws Exception {
-    internalTestConcurrentAsyncRenameWithOverwrite(10000,
-        "testConservativeConcurrentAsyncRenameWithOverwrite");
-  }
-
-  private void internalTestConcurrentAsyncRenameWithOverwrite(
-      final int asyncCallLimit, final String basePath) throws Exception {
-    final Path renameDir = new Path(String.format("/test/%s/", basePath));
-    Configuration conf = new HdfsConfiguration();
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
-        asyncCallLimit);
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
-        .build();
-    cluster.waitActive();
-    DistributedFileSystem dfs = cluster.getFileSystem();
-    AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
-    int count = 1000;
-    int start = 0, end = 0;
-    Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
-
-    assertTrue(dfs.mkdirs(renameDir));
-
-    try {
-      // concurrently invoking many rename
-      for (int i = 0; i < count; i++) {
-        Path src = new Path(renameDir, "src" + i);
-        Path dst = new Path(renameDir, "dst" + i);
-        DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
-        DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
-        for (;;) {
-          try {
-            LOG.info("rename #" + i);
-            Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-            returnFutures.put(i, returnFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            /**
-             * reached limit of async calls, fetch results of finished async
-             * calls to let follow-on calls go
-             */
-            LOG.error(e);
-            start = end;
-            end = i;
-            LOG.info(String.format("start=%d, end=%d, i=%d", start, end, i));
-            waitForReturnValues(returnFutures, start, end);
-          }
-        }
-      }
-
-      // wait for completing the calls
-      for (int i = start; i < count; i++) {
-        returnFutures.get(i).get();
-      }
-
-      // Restart NN and check the rename successfully
-      cluster.restartNameNodes();
-
-      // very the src dir should not exist, dst should
-      for (int i = 0; i < count; i++) {
-        Path src = new Path(renameDir, "src" + i);
-        Path dst = new Path(renameDir, "dst" + i);
-        assertFalse(dfs.exists(src));
-        assertTrue(dfs.exists(dst));
-      }
-    } finally {
-      if (dfs != null) {
-        dfs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
+  private void verifyCallGetReturnValueMultipleTimes(
+      final Map<Integer, Future<Void>> reFutures, final Path[] srcs,
+      final Path[] dsts)
+      throws InterruptedException, ExecutionException, IOException {
 
-  private void waitForReturnValues(
-      final Map<Integer, Future<Void>> returnFutures, final int start,
-      final int end) throws InterruptedException, ExecutionException {
-    LOG.info(String.format("calling waitForReturnValues [%d, %d)", start, end));
-    for (int i = start; i < end; i++) {
-      LOG.info("calling Future#get #" + i);
-      returnFutures.get(i).get();
-    }
-  }
+    // wait for completing the calls
+    waitForReturnValues(reFutures, 0, NUM_TESTS);
 
-  @Test
-  public void testConservativeConcurrentAsyncAPI() throws Exception {
-    internalTestConcurrentAsyncAPI(100, "testConservativeConcurrentAsyncAPI");
+    // verify the src dir should not exist, dst should
+    verifyRenames(srcs, dsts);
   }
 
   @Test(timeout = 60000)
-  public void testAggressiveConcurrentAsyncAPI() throws Exception {
-    internalTestConcurrentAsyncAPI(10000, "testAggressiveConcurrentAsyncAPI");
-  }
-
-  private void internalTestConcurrentAsyncAPI(final int asyncCallLimit,
-      final String basePath) throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    String group1 = "group1";
-    String group2 = "group2";
-    String user1 = "user1";
-    int count = 500;
-
-    // explicitly turn on permission checking
-    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-    // set the limit of max async calls
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
-        asyncCallLimit);
-
-    // 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);
-
-    // start mini cluster
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(3).build();
-    cluster.waitActive();
-    AsyncDistributedFileSystem adfs = cluster.getFileSystem()
-        .getAsyncDistributedFileSystem();
-
-    // prepare for test
-    FileSystem rootFs = FileSystem.get(conf);
-    final Path parent = new Path(String.format("/test/%s/", basePath));
-    final Path[] srcs = new Path[count];
-    final Path[] dsts = new Path[count];
-    short[] permissions = new short[count];
-    for (int i = 0; i < count; i++) {
+  public void testConcurrentAsyncRename() throws Exception {
+    final Path parent = new Path(
+        String.format("/test/%s/", "testConcurrentAsyncRename"));
+    assertTrue(fs.mkdirs(parent));
+
+    // prepare test
+    final Path[] srcs = new Path[NUM_TESTS];
+    final Path[] dsts = new Path[NUM_TESTS];
+    for (int i = 0; i < NUM_TESTS; i++) {
       srcs[i] = new Path(parent, "src" + i);
       dsts[i] = new Path(parent, "dst" + i);
-      DFSTestUtil.createFile(rootFs, srcs[i], fileLen, replFactor, 1);
-      DFSTestUtil.createFile(rootFs, dsts[i], fileLen, replFactor, 1);
-      assertTrue(rootFs.exists(srcs[i]));
-      assertTrue(rootFs.getFileStatus(srcs[i]).isFile());
-      assertTrue(rootFs.exists(dsts[i]));
-      assertTrue(rootFs.getFileStatus(dsts[i]).isFile());
-      permissions[i] = permGenerator.next();
+      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
+      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
     }
 
-    Map<Integer, Future<Void>> renameRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<Void>> permRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<Void>> ownerRetFutures =
-        new HashMap<Integer, Future<Void>>();
+    // concurrently invoking many rename
     int start = 0, end = 0;
-    // test rename
-    for (int i = 0; i < count; i++) {
+    Map<Integer, Future<Void>> retFutures =
+        new HashMap<Integer, Future<Void>>();
+    for (int i = 0; i < NUM_TESTS; i++) {
       for (;;) {
         try {
-          Future<Void> returnFuture = adfs.rename(srcs[i], dsts[i],
+          LOG.info("rename #" + i);
+          Future<Void> retFuture = adfs.rename(srcs[i], dsts[i],
               Rename.OVERWRITE);
-          renameRetFutures.put(i, returnFuture);
+          retFutures.put(i, retFuture);
           break;
         } catch (AsyncCallLimitExceededException e) {
+          /**
+           * reached limit of async calls, fetch results of finished async calls
+           * to let follow-on calls go
+           */
+          LOG.error(e);
           start = end;
           end = i;
-          waitForReturnValues(renameRetFutures, start, end);
+          LOG.info(String.format("start=%d, end=%d, i=%d", start, end, i));
+          waitForReturnValues(retFutures, start, end);
         }
       }
     }
 
     // wait for completing the calls
-    for (int i = start; i < count; i++) {
-      renameRetFutures.get(i).get();
-    }
-
-    // Restart NN and check the rename successfully
-    cluster.restartNameNodes();
-
-    // very the src should not exist, dst should
-    for (int i = 0; i < count; i++) {
-      assertFalse(rootFs.exists(srcs[i]));
-      assertTrue(rootFs.exists(dsts[i]));
-    }
-
-    // test permissions
-    try {
-      for (int i = 0; i < count; i++) {
-        for (;;) {
-          try {
-            Future<Void> retFuture = adfs.setPermission(dsts[i],
-                new FsPermission(permissions[i]));
-            permRetFutures.put(i, retFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            start = end;
-            end = i;
-            waitForReturnValues(permRetFutures, start, end);
-          }
-        }
-      }
-      // wait for completing the calls
-      for (int i = start; i < count; i++) {
-        permRetFutures.get(i).get();
-      }
-
-      // Restart NN and check permission then
-      cluster.restartNameNodes();
-
-      // verify the permission
-      for (int i = 0; i < count; i++) {
-        assertTrue(rootFs.exists(dsts[i]));
-        FsPermission fsPerm = new FsPermission(permissions[i]);
-        checkAccessPermissions(rootFs.getFileStatus(dsts[i]),
-            fsPerm.getUserAction());
-      }
-
-      // test setOwner
-      start = 0;
-      end = 0;
-      for (int i = 0; i < count; i++) {
-        for (;;) {
-          try {
-            Future<Void> retFuture = adfs.setOwner(dsts[i], "user1",
-                "group2");
-            ownerRetFutures.put(i, retFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            start = end;
-            end = i;
-            waitForReturnValues(ownerRetFutures, start, end);
-          }
-        }
-      }
-      // wait for completing the calls
-      for (int i = start; i < count; i++) {
-        ownerRetFutures.get(i).get();
-      }
+    waitForReturnValues(retFutures, end, NUM_TESTS);
 
-      // Restart NN and check owner then
-      cluster.restartNameNodes();
-
-      // verify the owner
-      for (int i = 0; i < count; i++) {
-        assertTrue(rootFs.exists(dsts[i]));
-        assertTrue(
-            "user1".equals(rootFs.getFileStatus(dsts[i]).getOwner()));
-        assertTrue(
-            "group2".equals(rootFs.getFileStatus(dsts[i]).getGroup()));
-      }
-    } catch (AccessControlException ace) {
-      throw ace;
-    } finally {
-      if (rootFs != null) {
-        rootFs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
+    // verify the src dir should not exist, dst should
+    verifyRenames(srcs, dsts);
   }
 
-  static void checkAccessPermissions(FileStatus stat, FsAction mode)
+  private void verifyRenames(final Path[] srcs, final Path[] dsts)
       throws IOException {
-    checkAccessPermissions(UserGroupInformation.getCurrentUser(), stat, mode);
-  }
-
-  static void checkAccessPermissions(final UserGroupInformation ugi,
-      FileStatus stat, FsAction mode) throws IOException {
-    FsPermission perm = stat.getPermission();
-    String user = ugi.getShortUserName();
-    List<String> groups = Arrays.asList(ugi.getGroupNames());
-
-    if (user.equals(stat.getOwner())) {
-      if (perm.getUserAction().implies(mode)) {
-        return;
-      }
-    } else if (groups.contains(stat.getGroup())) {
-      if (perm.getGroupAction().implies(mode)) {
-        return;
-      }
-    } else {
-      if (perm.getOtherAction().implies(mode)) {
-        return;
-      }
+    for (int i = 0; i < NUM_TESTS; i++) {
+      assertFalse(fs.exists(srcs[i]));
+      assertTrue(fs.exists(dsts[i]));
     }
-    throw new AccessControlException(String.format(
-        "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat
-            .getPath(), stat.getOwner(), stat.getGroup(),
-        stat.isDirectory() ? "d" : "-", perm));
   }
 
-  @Test(timeout = 60000)
-  public void testAsyncAPIWithException() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    String group1 = "group1";
-    String group2 = "group2";
-    String user1 = "user1";
-    UserGroupInformation ugi1;
-
-    // explicitly turn on permission checking
-    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-
-    // 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 MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(3).build();
-    cluster.waitActive();
-
-    FileSystem rootFs = FileSystem.get(conf);
-    final Path renameDir = new Path("/test/async_api_exception/");
-    final Path src = new Path(renameDir, "src");
-    final Path dst = new Path(renameDir, "dst");
-    rootFs.mkdirs(src);
-
-    AsyncDistributedFileSystem adfs = ugi1
-        .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
-          @Override
-          public AsyncDistributedFileSystem run() throws Exception {
-            return cluster.getFileSystem().getAsyncDistributedFileSystem();
-          }
-        });
-
-    Future<Void> retFuture;
-    try {
-      retFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-      retFuture.get();
-    } catch (ExecutionException e) {
-      TestAsyncDFS.checkPermissionDenied(e, src, user1);
-      assertTrue("Permission denied messages must carry the path parent", e
-          .getMessage().contains(src.getParent().toUri().getPath()));
-    }
-
-    FsPermission fsPerm = new FsPermission(permGenerator.next());
-    try {
-      retFuture = adfs.setPermission(src, fsPerm);
-      retFuture.get();
-    } catch (ExecutionException e) {
-      TestAsyncDFS.checkPermissionDenied(e, src, user1);
-      assertTrue("Permission denied messages must carry the name of the path",
-          e.getMessage().contains(src.getName()));
-    }
-
-    try {
-      retFuture = adfs.setOwner(src, "user1", "group2");
-      retFuture.get();
-    } catch (ExecutionException e) {
-      TestAsyncDFS.checkPermissionDenied(e, src, user1);
-      assertTrue("Permission denied messages must carry the name of the path",
-          e.getMessage().contains(src.getName()));
-    } finally {
-      if (rootFs != null) {
-        rootFs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
+  void waitForReturnValues(final Map<Integer, Future<Void>> retFutures,
+      final int start, final int end)
+      throws InterruptedException, ExecutionException {
+    TestAsyncDFS.waitForReturnValues(retFutures, start, end);
   }
 }
\ No newline at end of file


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


[06/47] hadoop git commit: HDFS-10367. TestDFSShell.testMoveWithTargetPortEmpty fails with Address bind exception. Contributed by Brahma Reddy Battula.

Posted by ae...@apache.org.
HDFS-10367. TestDFSShell.testMoveWithTargetPortEmpty fails with Address bind exception. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-1312
Commit: aadb77e412ab9d4ad05a0bd8b37d547ba5adad03
Parents: 99675e0
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Thu Jun 2 20:55:04 2016 +0900
Committer: Masatake Iwasaki <iw...@apache.org>
Committed: Thu Jun 2 20:55:04 2016 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/net/ServerSocketUtil.java | 39 ++++++++++++++++++++
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |  4 +-
 2 files changed, 42 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aadb77e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
index a3e1fff..023c1ed 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
@@ -63,4 +63,43 @@ public class ServerSocketUtil {
     }
   }
 
+  /**
+   * Check whether port is available or not.
+   *
+   * @param port given port
+   * @return
+   */
+  private static boolean isPortAvailable(int port) {
+    try (ServerSocket s = new ServerSocket(port)) {
+      return true;
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  /**
+   * Wait till the port available.
+   *
+   * @param port given port
+   * @param retries number of retries for given port
+   * @return
+   * @throws InterruptedException
+   * @throws IOException
+   */
+  public static int waitForPort(int port, int retries)
+      throws InterruptedException, IOException {
+    int tries = 0;
+    while (true) {
+      if (isPortAvailable(port)) {
+        return port;
+      } else {
+        tries++;
+        if (tries >= retries) {
+          throw new IOException(
+              "Port is already in use; giving up after " + tries + " times.");
+        }
+        Thread.sleep(1000);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aadb77e4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index fc3de75..e31de13 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.BZip2Codec;
 import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -567,7 +568,8 @@ public class TestDFSShell {
       cluster = new MiniDFSCluster.Builder(conf)
           .format(true)
           .numDataNodes(2)
-          .nameNodePort(HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT)
+          .nameNodePort(ServerSocketUtil.waitForPort(
+              HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT, 10))
           .waitSafeMode(true)
           .build();
       FileSystem srcFs = cluster.getFileSystem();


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


[02/47] hadoop git commit: YARN-5088. Improve "yarn log" command-line to read the last K bytes for the log files. Contributed by Xuan Gong

Posted by ae...@apache.org.
YARN-5088. Improve "yarn log" command-line to read the last K bytes for the log files. Contributed by Xuan Gong


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

Branch: refs/heads/HDFS-1312
Commit: 0bc05e40fa7e183efe8463ada459c621da3ce3bf
Parents: 35356de
Author: Xuan <xg...@apache.org>
Authored: Wed Jun 1 13:44:21 2016 -0700
Committer: Xuan <xg...@apache.org>
Committed: Wed Jun 1 13:44:21 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  | 17 +++-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     | 70 ++++++++++++++-
 .../logaggregation/AggregatedLogFormat.java     | 89 ++++++++++++++++++--
 .../logaggregation/ContainerLogsRequest.java    | 13 ++-
 .../yarn/logaggregation/LogCLIHelpers.java      | 25 +++---
 .../webapp/AHSWebServices.java                  | 50 ++++++++---
 .../webapp/TestAHSWebServices.java              | 66 +++++++++++++++
 .../nodemanager/webapp/NMWebServices.java       | 49 +++++++++--
 .../nodemanager/webapp/TestNMWebServices.java   | 48 ++++++++++-
 9 files changed, 384 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index 2127006..bbe636f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -88,6 +88,7 @@ public class LogsCLI extends Configured implements Tool {
   private static final String SHOW_META_INFO = "show_meta_info";
   private static final String LIST_NODES_OPTION = "list_nodes";
   private static final String OUT_OPTION = "out";
+  private static final String SIZE_OPTION = "size";
   public static final String HELP_CMD = "help";
 
   @Override
@@ -113,6 +114,7 @@ public class LogsCLI extends Configured implements Tool {
     String[] logFiles = null;
     List<String> amContainersList = new ArrayList<String>();
     String localDir = null;
+    long bytes = Long.MAX_VALUE;
     try {
       CommandLine commandLine = parser.parse(opts, args, true);
       appIdStr = commandLine.getOptionValue(APPLICATION_ID_OPTION);
@@ -134,6 +136,9 @@ public class LogsCLI extends Configured implements Tool {
       if (commandLine.hasOption(CONTAINER_LOG_FILES)) {
         logFiles = commandLine.getOptionValues(CONTAINER_LOG_FILES);
       }
+      if (commandLine.hasOption(SIZE_OPTION)) {
+        bytes = Long.parseLong(commandLine.getOptionValue(SIZE_OPTION));
+      }
     } catch (ParseException e) {
       System.err.println("options parsing failed: " + e.getMessage());
       printHelpMessage(printOpts);
@@ -195,7 +200,7 @@ public class LogsCLI extends Configured implements Tool {
 
     ContainerLogsRequest request = new ContainerLogsRequest(appId,
         isApplicationFinished(appState), appOwner, nodeAddress, null,
-        containerIdStr, localDir, logs);
+        containerIdStr, localDir, logs, bytes);
 
     if (showMetaInfo) {
       return showMetaInfo(request, logCliHelper);
@@ -402,6 +407,7 @@ public class LogsCLI extends Configured implements Tool {
           ClientResponse response =
               webResource.path("ws").path("v1").path("node")
                 .path("containerlogs").path(containerIdStr).path(logFile)
+                .queryParam("size", Long.toString(request.getBytes()))
                 .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
           out.println(response.getEntity(String.class));
           out.println("End of LogType:" + logFile);
@@ -442,7 +448,9 @@ public class LogsCLI extends Configured implements Tool {
         newOptions);
   }
 
-  private ContainerReport getContainerReport(String containerIdStr)
+  @Private
+  @VisibleForTesting
+  public ContainerReport getContainerReport(String containerIdStr)
       throws YarnException, IOException {
     YarnClient yarnClient = createYarnClient();
     try {
@@ -636,12 +644,16 @@ public class LogsCLI extends Configured implements Tool {
     opts.addOption(OUT_OPTION, true, "Local directory for storing individual "
         + "container logs. The container logs will be stored based on the "
         + "node the container ran on.");
+    opts.addOption(SIZE_OPTION, true, "Prints the log file's first 'n' bytes "
+        + "or the last 'n' bytes. Use negative values as bytes to read from "
+        + "the end and positive values as bytes to read from the beginning.");
     opts.getOption(APPLICATION_ID_OPTION).setArgName("Application ID");
     opts.getOption(CONTAINER_ID_OPTION).setArgName("Container ID");
     opts.getOption(NODE_ADDRESS_OPTION).setArgName("Node Address");
     opts.getOption(APP_OWNER_OPTION).setArgName("Application Owner");
     opts.getOption(AM_CONTAINER_OPTION).setArgName("AM Containers");
     opts.getOption(OUT_OPTION).setArgName("Local Directory");
+    opts.getOption(SIZE_OPTION).setArgName("size");
     return opts;
   }
 
@@ -656,6 +668,7 @@ public class LogsCLI extends Configured implements Tool {
     printOpts.addOption(commandOpts.getOption(SHOW_META_INFO));
     printOpts.addOption(commandOpts.getOption(LIST_NODES_OPTION));
     printOpts.addOption(commandOpts.getOption(OUT_OPTION));
+    printOpts.addOption(commandOpts.getOption(SIZE_OPTION));
     return printOpts;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index 2639a0a..db40b50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -203,6 +203,11 @@ public class TestLogsCLI {
     pw.println("                                 for all the containers on the specific");
     pw.println("                                 NodeManager. Currently, this option can");
     pw.println("                                 only be used for finished applications.");
+    pw.println(" -size <size>                    Prints the log file's first 'n' bytes or");
+    pw.println("                                 the last 'n' bytes. Use negative values");
+    pw.println("                                 as bytes to read from the end and");
+    pw.println("                                 positive values as bytes to read from the");
+    pw.println("                                 beginning.");
     pw.close();
     String appReportStr = baos.toString("UTF-8");
     Assert.assertEquals(appReportStr, sysOutStream.toString());
@@ -227,7 +232,7 @@ public class TestLogsCLI {
     ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1);
     ContainerId containerId2 = ContainerId.newContainerId(appAttemptId, 2);
     ContainerId containerId3 = ContainerId.newContainerId(appAttemptId, 3);
-    NodeId nodeId = NodeId.newInstance("localhost", 1234);
+    final NodeId nodeId = NodeId.newInstance("localhost", 1234);
 
     // create local logs
     String rootLogDir = "target/LocalLogs";
@@ -281,7 +286,16 @@ public class TestLogsCLI {
     YarnClient mockYarnClient =
         createMockYarnClient(
             YarnApplicationState.FINISHED, ugi.getShortUserName());
-    LogsCLI cli = new LogsCLIForTest(mockYarnClient);
+    LogsCLI cli = new LogsCLIForTest(mockYarnClient) {
+      @Override
+      public ContainerReport getContainerReport(String containerIdStr)
+          throws YarnException, IOException {
+        ContainerReport mockReport = mock(ContainerReport.class);
+        doReturn(nodeId).when(mockReport).getAssignedNode();
+        doReturn("http://localhost:2345").when(mockReport).getNodeHttpAddress();
+        return mockReport;
+      }
+    };
     cli.setConf(configuration);
 
     int exitCode = cli.run(new String[] { "-applicationId", appId.toString() });
@@ -307,6 +321,7 @@ public class TestLogsCLI {
         "Hello container_0_0001_01_000003 in syslog!"));
     assertTrue(sysOutStream.toString().contains(
         "Hello container_0_0001_01_000003 in stdout!"));
+    int fullSize = sysOutStream.toByteArray().length;
     sysOutStream.reset();
 
     exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
@@ -329,6 +344,14 @@ public class TestLogsCLI {
         "Can not find any log file matching the pattern: [123]"));
     sysErrStream.reset();
 
+    // specify the bytes which is larger than the actual file size,
+    // we would get the full logs
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-logFiles", ".*", "-size", "10000" });
+    assertTrue(exitCode == 0);
+    assertTrue(sysOutStream.toByteArray().length == fullSize);
+    sysOutStream.reset();
+
     // uploaded two logs for container1. The first log is empty.
     // The second one is not empty.
     // We can still successfully read logs for container1.
@@ -345,6 +368,49 @@ public class TestLogsCLI {
           + " are not present in this log-file."));
     sysOutStream.reset();
 
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-containerId", containerId3.toString(), "-logFiles", "stdout" });
+    assertTrue(exitCode == 0);
+    int fullContextSize = sysOutStream.toByteArray().length;
+    String fullContext = sysOutStream.toString();
+    sysOutStream.reset();
+
+    String logMessage = "Hello container_0_0001_01_000003 in stdout!";
+    int fileContentSize = logMessage.getBytes().length;
+    int tailContentSize = "End of LogType:syslog\n\n".getBytes().length;
+
+    // specify how many bytes we should get from logs
+    // specify a position number, it would get the first n bytes from
+    // container log
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-containerId", containerId3.toString(), "-logFiles", "stdout",
+        "-size", "5"});
+    assertTrue(exitCode == 0);
+    Assert.assertEquals(new String(logMessage.getBytes(), 0, 5),
+        new String(sysOutStream.toByteArray(),
+        (fullContextSize - fileContentSize - tailContentSize), 5));
+    sysOutStream.reset();
+
+    // specify a negative number, it would get the last n bytes from
+    // container log
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-containerId", containerId3.toString(), "-logFiles", "stdout",
+        "-size", "-5"});
+    assertTrue(exitCode == 0);
+    Assert.assertEquals(new String(logMessage.getBytes(),
+        logMessage.getBytes().length - 5, 5),
+        new String(sysOutStream.toByteArray(),
+        (fullContextSize - fileContentSize - tailContentSize), 5));
+    sysOutStream.reset();
+
+    long negative = (fullContextSize + 1000) * (-1);
+    exitCode = cli.run(new String[] {"-applicationId", appId.toString(),
+        "-containerId", containerId3.toString(), "-logFiles", "stdout",
+        "-size", Long.toString(negative)});
+    assertTrue(exitCode == 0);
+    Assert.assertEquals(fullContext, sysOutStream.toString());
+    sysOutStream.reset();
+
     // Uploaded the empty log for container0.
     // We should see the message showing the log for container0
     // are not present.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index d636200..98ffce1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -733,7 +733,7 @@ public class AggregatedLogFormat {
         ps = new PrintStream(os);
         while (true) {
           try {
-            readContainerLogs(valueStream, ps, logUploadedTime);
+            readContainerLogs(valueStream, ps, logUploadedTime, Long.MAX_VALUE);
           } catch (EOFException e) {
             // EndOfFile
             return;
@@ -757,7 +757,8 @@ public class AggregatedLogFormat {
     }
 
     private static void readContainerLogs(DataInputStream valueStream,
-        PrintStream out, long logUploadedTime) throws IOException {
+        PrintStream out, long logUploadedTime, long bytes)
+        throws IOException {
       byte[] buf = new byte[65535];
 
       String fileType = valueStream.readUTF();
@@ -773,16 +774,35 @@ public class AggregatedLogFormat {
       out.println(fileLengthStr);
       out.println("Log Contents:");
 
+      long toSkip = 0;
+      long totalBytesToRead = fileLength;
+      if (bytes < 0) {
+        long absBytes = Math.abs(bytes);
+        if (absBytes < fileLength) {
+          toSkip = fileLength - absBytes;
+          totalBytesToRead = absBytes;
+        }
+        long skippedBytes = valueStream.skip(toSkip);
+        if (skippedBytes != toSkip) {
+          throw new IOException("The bytes were skipped are "
+              + "different from the caller requested");
+        }
+      } else {
+        if (bytes < fileLength) {
+          totalBytesToRead = bytes;
+        }
+      }
+
       long curRead = 0;
-      long pendingRead = fileLength - curRead;
+      long pendingRead = totalBytesToRead - curRead;
       int toRead =
                 pendingRead > buf.length ? buf.length : (int) pendingRead;
       int len = valueStream.read(buf, 0, toRead);
-      while (len != -1 && curRead < fileLength) {
+      while (len != -1 && curRead < totalBytesToRead) {
         out.write(buf, 0, len);
         curRead += len;
 
-        pendingRead = fileLength - curRead;
+        pendingRead = totalBytesToRead - curRead;
         toRead =
                   pendingRead > buf.length ? buf.length : (int) pendingRead;
         len = valueStream.read(buf, 0, toRead);
@@ -803,7 +823,23 @@ public class AggregatedLogFormat {
     public static void readAContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime)
           throws IOException {
-      readContainerLogs(valueStream, out, logUploadedTime);
+      readContainerLogs(valueStream, out, logUploadedTime, Long.MAX_VALUE);
+    }
+
+    /**
+     * Keep calling this till you get a {@link EOFException} for getting logs of
+     * all types for a single container for the specific bytes.
+     *
+     * @param valueStream
+     * @param out
+     * @param logUploadedTime
+     * @param bytes
+     * @throws IOException
+     */
+    public static void readAContainerLogsForALogType(
+        DataInputStream valueStream, PrintStream out, long logUploadedTime,
+        long bytes) throws IOException {
+      readContainerLogs(valueStream, out, logUploadedTime, bytes);
     }
 
     /**
@@ -832,6 +868,22 @@ public class AggregatedLogFormat {
     public static int readContainerLogsForALogType(
         DataInputStream valueStream, PrintStream out, long logUploadedTime,
         List<String> logType) throws IOException {
+      return readContainerLogsForALogType(valueStream, out, logUploadedTime,
+          logType, Long.MAX_VALUE);
+    }
+
+    /**
+     * Keep calling this till you get a {@link EOFException} for getting logs of
+     * the specific types for a single container.
+     * @param valueStream
+     * @param out
+     * @param logUploadedTime
+     * @param logType
+     * @throws IOException
+     */
+    public static int readContainerLogsForALogType(
+        DataInputStream valueStream, PrintStream out, long logUploadedTime,
+        List<String> logType, long bytes) throws IOException {
       byte[] buf = new byte[65535];
 
       String fileType = valueStream.readUTF();
@@ -848,15 +900,34 @@ public class AggregatedLogFormat {
         out.println(fileLengthStr);
         out.println("Log Contents:");
 
+        long toSkip = 0;
+        long totalBytesToRead = fileLength;
+        if (bytes < 0) {
+          long absBytes = Math.abs(bytes);
+          if (absBytes < fileLength) {
+            toSkip = fileLength - absBytes;
+            totalBytesToRead = absBytes;
+          }
+          long skippedBytes = valueStream.skip(toSkip);
+          if (skippedBytes != toSkip) {
+            throw new IOException("The bytes were skipped are "
+                + "different from the caller requested");
+          }
+        } else {
+          if (bytes < fileLength) {
+            totalBytesToRead = bytes;
+          }
+        }
+
         long curRead = 0;
-        long pendingRead = fileLength - curRead;
+        long pendingRead = totalBytesToRead - curRead;
         int toRead = pendingRead > buf.length ? buf.length : (int) pendingRead;
         int len = valueStream.read(buf, 0, toRead);
-        while (len != -1 && curRead < fileLength) {
+        while (len != -1 && curRead < totalBytesToRead) {
           out.write(buf, 0, len);
           curRead += len;
 
-          pendingRead = fileLength - curRead;
+          pendingRead = totalBytesToRead - curRead;
           toRead = pendingRead > buf.length ? buf.length : (int) pendingRead;
           len = valueStream.read(buf, 0, toRead);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRequest.java
index b0a7fdc..f32285c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogsRequest.java
@@ -30,6 +30,7 @@ public class ContainerLogsRequest {
   private boolean appFinished;
   private String outputLocalDir;
   private List<String> logTypes;
+  private long bytes;
 
   public ContainerLogsRequest() {}
 
@@ -42,12 +43,13 @@ public class ContainerLogsRequest {
     this.setContainerId(request.getContainerId());
     this.setOutputLocalDir(request.getOutputLocalDir());
     this.setLogTypes(request.getLogTypes());
+    this.setBytes(request.getBytes());
   }
 
   public ContainerLogsRequest(ApplicationId applicationId,
       boolean isAppFinished, String owner,
       String address, String httpAddress, String container, String localDir,
-      List<String> logs) {
+      List<String> logs, long bytes) {
     this.setAppId(applicationId);
     this.setAppFinished(isAppFinished);
     this.setAppOwner(owner);
@@ -56,6 +58,7 @@ public class ContainerLogsRequest {
     this.setContainerId(container);
     this.setOutputLocalDir(localDir);
     this.setLogTypes(logs);
+    this.setBytes(bytes);
   }
 
   public ApplicationId getAppId() {
@@ -121,4 +124,12 @@ public class ContainerLogsRequest {
   public void setLogTypes(List<String> logTypes) {
     this.logTypes = logTypes;
   }
+
+  public long getBytes() {
+    return bytes;
+  }
+
+  public void setBytes(long bytes) {
+    this.bytes = bytes;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
index 22147ae..3811054 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogCLIHelpers.java
@@ -65,6 +65,7 @@ public class LogCLIHelpers implements Configurable {
     options.setAppOwner(jobOwner);
     List<String> logs = new ArrayList<String>();
     options.setLogTypes(logs);
+    options.setBytes(Long.MAX_VALUE);
     return dumpAContainersLogsForALogType(options, false);
   }
 
@@ -160,12 +161,13 @@ public class LogCLIHelpers implements Configurable {
                 thisNodeFile.getPath());
           if (logType == null || logType.isEmpty()) {
             if (dumpAContainerLogs(containerId, reader, out,
-                thisNodeFile.getModificationTime()) > -1) {
+                thisNodeFile.getModificationTime(), options.getBytes()) > -1) {
               foundContainerLogs = true;
             }
           } else {
             if (dumpAContainerLogsForALogType(containerId, reader, out,
-                thisNodeFile.getModificationTime(), logType) > -1) {
+                thisNodeFile.getModificationTime(), logType,
+                options.getBytes()) > -1) {
               foundContainerLogs = true;
             }
           }
@@ -222,12 +224,13 @@ public class LogCLIHelpers implements Configurable {
           out.println(StringUtils.repeat("=", containerId.length()));
           if (logType == null || logType.isEmpty()) {
             if (dumpAContainerLogs(containerId, reader, out,
-                thisNodeFile.getModificationTime()) > -1) {
+                thisNodeFile.getModificationTime(), options.getBytes()) > -1) {
               foundContainerLogs = true;
             }
           } else {
             if (dumpAContainerLogsForALogType(containerId, reader, out,
-                thisNodeFile.getModificationTime(), logType) > -1) {
+                thisNodeFile.getModificationTime(), logType,
+                options.getBytes()) > -1) {
               foundContainerLogs = true;
             }
           }
@@ -249,7 +252,7 @@ public class LogCLIHelpers implements Configurable {
   @Private
   public int dumpAContainerLogs(String containerIdStr,
       AggregatedLogFormat.LogReader reader, PrintStream out,
-      long logUploadedTime) throws IOException {
+      long logUploadedTime, long bytes) throws IOException {
     DataInputStream valueStream = getContainerLogsStream(
         containerIdStr, reader);
 
@@ -261,7 +264,7 @@ public class LogCLIHelpers implements Configurable {
     while (true) {
       try {
         LogReader.readAContainerLogsForALogType(valueStream, out,
-            logUploadedTime);
+            logUploadedTime, bytes);
         foundContainerLogs = true;
       } catch (EOFException eof) {
         break;
@@ -290,7 +293,8 @@ public class LogCLIHelpers implements Configurable {
   @Private
   public int dumpAContainerLogsForALogType(String containerIdStr,
       AggregatedLogFormat.LogReader reader, PrintStream out,
-      long logUploadedTime, List<String> logType) throws IOException {
+      long logUploadedTime, List<String> logType, long bytes)
+      throws IOException {
     DataInputStream valueStream = getContainerLogsStream(
         containerIdStr, reader);
     if (valueStream == null) {
@@ -301,7 +305,7 @@ public class LogCLIHelpers implements Configurable {
     while (true) {
       try {
         int result = LogReader.readContainerLogsForALogType(
-            valueStream, out, logUploadedTime, logType);
+            valueStream, out, logUploadedTime, logType, bytes);
         if (result == 0) {
           foundContainerLogs = true;
         }
@@ -361,12 +365,13 @@ public class LogCLIHelpers implements Configurable {
                 try {
                   if (logTypes == null || logTypes.isEmpty()) {
                     LogReader.readAContainerLogsForALogType(valueStream, out,
-                        thisNodeFile.getModificationTime());
+                        thisNodeFile.getModificationTime(),
+                        options.getBytes());
                     foundAnyLogs = true;
                   } else {
                     int result = LogReader.readContainerLogsForALogType(
                         valueStream, out, thisNodeFile.getModificationTime(),
-                        logTypes);
+                        logTypes, options.getBytes());
                     if (result == 0) {
                       foundAnyLogs = true;
                     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
index e422c35..d91ae55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
@@ -213,7 +213,8 @@ public class AHSWebServices extends WebServices {
       @Context HttpServletResponse res,
       @PathParam("containerid") String containerIdStr,
       @PathParam("filename") String filename,
-      @QueryParam("download") String download) {
+      @QueryParam("download") String download,
+      @QueryParam("size") String size) {
     init(res);
     ContainerId containerId;
     try {
@@ -225,6 +226,9 @@ public class AHSWebServices extends WebServices {
 
     boolean downloadFile = parseBooleanParam(download);
 
+
+    final long length = parseLongParam(size);
+
     ApplicationId appId = containerId.getApplicationAttemptId()
         .getApplicationId();
     AppInfo appInfo;
@@ -233,7 +237,7 @@ public class AHSWebServices extends WebServices {
     } catch (Exception ex) {
       // directly find logs from HDFS.
       return sendStreamOutputResponse(appId, null, null, containerIdStr,
-          filename, downloadFile);
+          filename, downloadFile, length);
     }
     String appOwner = appInfo.getUser();
 
@@ -247,7 +251,7 @@ public class AHSWebServices extends WebServices {
       if (isFinishedState(appInfo.getAppState())) {
         // directly find logs from HDFS.
         return sendStreamOutputResponse(appId, appOwner, null, containerIdStr,
-            filename, downloadFile);
+            filename, downloadFile, length);
       }
       return createBadResponse(Status.INTERNAL_SERVER_ERROR,
           "Can not get ContainerInfo for the container: " + containerId);
@@ -267,7 +271,7 @@ public class AHSWebServices extends WebServices {
       return response.build();
     } else if (isFinishedState(appInfo.getAppState())) {
       return sendStreamOutputResponse(appId, appOwner, nodeId,
-          containerIdStr, filename, downloadFile);
+          containerIdStr, filename, downloadFile, length);
     } else {
       return createBadResponse(Status.NOT_FOUND,
           "The application is not at Running or Finished State.");
@@ -296,11 +300,11 @@ public class AHSWebServices extends WebServices {
 
   private Response sendStreamOutputResponse(ApplicationId appId,
       String appOwner, String nodeId, String containerIdStr,
-      String fileName, boolean downloadFile) {
+      String fileName, boolean downloadFile, long bytes) {
     StreamingOutput stream = null;
     try {
       stream = getStreamingOutput(appId, appOwner, nodeId,
-          containerIdStr, fileName);
+          containerIdStr, fileName, bytes);
     } catch (Exception ex) {
       return createBadResponse(Status.INTERNAL_SERVER_ERROR,
           ex.getMessage());
@@ -318,7 +322,7 @@ public class AHSWebServices extends WebServices {
 
   private StreamingOutput getStreamingOutput(ApplicationId appId,
       String appOwner, final String nodeId, final String containerIdStr,
-      final String logFile) throws IOException{
+      final String logFile, final long bytes) throws IOException{
     String suffix = LogAggregationUtils.getRemoteNodeLogDirSuffix(conf);
     org.apache.hadoop.fs.Path remoteRootLogDir = new org.apache.hadoop.fs.Path(
         conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
@@ -391,16 +395,35 @@ public class AHSWebServices extends WebServices {
                   byte[] b = sb.toString().getBytes(Charset.forName("UTF-8"));
                   os.write(b, 0, b.length);
 
+                  long toSkip = 0;
+                  long totalBytesToRead = fileLength;
+                  if (bytes < 0) {
+                    long absBytes = Math.abs(bytes);
+                    if (absBytes < fileLength) {
+                      toSkip = fileLength - absBytes;
+                      totalBytesToRead = absBytes;
+                    }
+                    long skippedBytes = valueStream.skip(toSkip);
+                    if (skippedBytes != toSkip) {
+                      throw new IOException("The bytes were skipped are "
+                          + "different from the caller requested");
+                    }
+                  } else {
+                    if (bytes < fileLength) {
+                      totalBytesToRead = bytes;
+                    }
+                  }
+
                   long curRead = 0;
-                  long pendingRead = fileLength - curRead;
+                  long pendingRead = totalBytesToRead - curRead;
                   int toRead = pendingRead > buf.length ? buf.length
                       : (int) pendingRead;
                   int len = valueStream.read(buf, 0, toRead);
-                  while (len != -1 && curRead < fileLength) {
+                  while (len != -1 && curRead < totalBytesToRead) {
                     os.write(buf, 0, len);
                     curRead += len;
 
-                    pendingRead = fileLength - curRead;
+                    pendingRead = totalBytesToRead - curRead;
                     toRead = pendingRead > buf.length ? buf.length
                         : (int) pendingRead;
                     len = valueStream.read(buf, 0, toRead);
@@ -433,4 +456,11 @@ public class AHSWebServices extends WebServices {
     };
     return stream;
   }
+
+  private long parseLongParam(String bytes) {
+    if (bytes == null || bytes.isEmpty()) {
+      return Long.MAX_VALUE;
+    }
+    return Long.parseLong(bytes);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
index f985fe4..71b0275 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
@@ -601,6 +601,72 @@ public class TestAHSWebServices extends JerseyTestBase {
         .get(ClientResponse.class);
     responseText = response.getEntity(String.class);
     assertTrue(responseText.contains("Hello." + containerId1ForApp100));
+    int fullTextSize = responseText.getBytes().length;
+    int tailTextSize = "\nEnd of LogType:syslog\n".getBytes().length;
+
+    String logMessage = "Hello." + containerId1ForApp100;
+    int fileContentSize = logMessage.getBytes().length;
+    // specify how many bytes we should get from logs
+    // if we specify a position number, it would get the first n bytes from
+    // container log
+    r = resource();
+    response = r.path("ws").path("v1")
+        .path("applicationhistory").path("containerlogs")
+        .path(containerId1ForApp100.toString()).path(fileName)
+        .queryParam("user.name", user)
+        .queryParam("size", "5")
+        .accept(MediaType.TEXT_PLAIN)
+        .get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals(responseText.getBytes().length,
+        (fullTextSize - fileContentSize) + 5);
+    assertTrue(fullTextSize >= responseText.getBytes().length);
+    assertEquals(new String(responseText.getBytes(),
+        (fullTextSize - fileContentSize - tailTextSize), 5),
+        new String(logMessage.getBytes(), 0, 5));
+
+    // specify how many bytes we should get from logs
+    // if we specify a negative number, it would get the last n bytes from
+    // container log
+    r = resource();
+    response = r.path("ws").path("v1")
+        .path("applicationhistory").path("containerlogs")
+        .path(containerId1ForApp100.toString()).path(fileName)
+        .queryParam("user.name", user)
+        .queryParam("size", "-5")
+        .accept(MediaType.TEXT_PLAIN)
+        .get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals(responseText.getBytes().length,
+        (fullTextSize - fileContentSize) + 5);
+    assertTrue(fullTextSize >= responseText.getBytes().length);
+    assertEquals(new String(responseText.getBytes(),
+        (fullTextSize - fileContentSize - tailTextSize), 5),
+        new String(logMessage.getBytes(), fileContentSize - 5, 5));
+
+    // specify the bytes which is larger than the actual file size,
+    // we would get the full logs
+    r = resource();
+    response = r.path("ws").path("v1")
+        .path("applicationhistory").path("containerlogs")
+        .path(containerId1ForApp100.toString()).path(fileName)
+        .queryParam("user.name", user)
+        .queryParam("size", "10000")
+        .accept(MediaType.TEXT_PLAIN)
+        .get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals(responseText.getBytes().length, fullTextSize);
+
+    r = resource();
+    response = r.path("ws").path("v1")
+        .path("applicationhistory").path("containerlogs")
+        .path(containerId1ForApp100.toString()).path(fileName)
+        .queryParam("user.name", user)
+        .queryParam("size", "-10000")
+        .accept(MediaType.TEXT_PLAIN)
+        .get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals(responseText.getBytes().length, fullTextSize);
   }
 
   private static void createContainerLogInLocalDir(Path appLogsDir,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
index 5c66511..e13baa7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
@@ -60,7 +60,6 @@ import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
-
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 
@@ -217,7 +216,8 @@ public class NMWebServices {
   @Unstable
   public Response getLogs(@PathParam("containerid") String containerIdStr,
       @PathParam("filename") String filename,
-      @QueryParam("download") String download) {
+      @QueryParam("download") String download,
+      @QueryParam("size") String size) {
     ContainerId containerId;
     try {
       containerId = ConverterUtils.toContainerId(containerIdStr);
@@ -235,19 +235,51 @@ public class NMWebServices {
       return Response.serverError().entity(ex.getMessage()).build();
     }
     boolean downloadFile = parseBooleanParam(download);
+    final long bytes = parseLongParam(size);
+
     try {
       final FileInputStream fis = ContainerLogsUtils.openLogFileForRead(
           containerIdStr, logFile, nmContext);
-      
+      final long fileLength = logFile.length();
+
       StreamingOutput stream = new StreamingOutput() {
         @Override
         public void write(OutputStream os) throws IOException,
             WebApplicationException {
           int bufferSize = 65536;
           byte[] buf = new byte[bufferSize];
-          int len;
-          while ((len = fis.read(buf, 0, bufferSize)) > 0) {
+          long toSkip = 0;
+          long totalBytesToRead = fileLength;
+          if (bytes < 0) {
+            long absBytes = Math.abs(bytes);
+            if (absBytes < fileLength) {
+              toSkip = fileLength - absBytes;
+              totalBytesToRead = absBytes;
+            }
+            long skippedBytes = fis.skip(toSkip);
+            if (skippedBytes != toSkip) {
+              throw new IOException("The bytes were skipped are different "
+                  + "from the caller requested");
+            }
+          } else {
+            if (bytes < fileLength) {
+              totalBytesToRead = bytes;
+            }
+          }
+
+          long curRead = 0;
+          long pendingRead = totalBytesToRead - curRead;
+          int toRead = pendingRead > buf.length ? buf.length
+              : (int) pendingRead;
+          int len = fis.read(buf, 0, toRead);
+          while (len != -1 && curRead < totalBytesToRead) {
             os.write(buf, 0, len);
+            curRead += len;
+
+            pendingRead = totalBytesToRead - curRead;
+            toRead = pendingRead > buf.length ? buf.length
+                : (int) pendingRead;
+            len = fis.read(buf, 0, toRead);
           }
           os.flush();
         }
@@ -268,4 +300,11 @@ public class NMWebServices {
     }
     return false;
   }
+
+  private long parseLongParam(String bytes) {
+    if (bytes == null || bytes.isEmpty()) {
+      return Long.MAX_VALUE;
+    }
+    return Long.parseLong(bytes);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc05e40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
index ce1b309..4e2feee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
@@ -26,17 +26,14 @@ import java.io.File;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringReader;
-
 import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -313,7 +310,7 @@ public class TestNMWebServices extends JerseyTestBase {
     assertEquals("incorrect number of elements", 1, nodes.getLength());
     verifyNodesXML(nodes);
   }
-  
+
   @Test
   public void testContainerLogs() throws IOException {
     WebResource r = resource();
@@ -351,6 +348,49 @@ public class TestNMWebServices extends JerseyTestBase {
         .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
     String responseText = response.getEntity(String.class);
     assertEquals(logMessage, responseText);
+    int fullTextSize = responseText.getBytes().length;
+
+    // specify how many bytes we should get from logs
+    // specify a position number, it would get the first n bytes from
+    // container log
+    response = r.path("ws").path("v1").path("node")
+        .path("containerlogs").path(containerIdStr).path(filename)
+        .queryParam("size", "5")
+        .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals(5, responseText.getBytes().length);
+    assertEquals(new String(logMessage.getBytes(), 0, 5), responseText);
+    assertTrue(fullTextSize >= responseText.getBytes().length);
+
+    // specify the bytes which is larger than the actual file size,
+    // we would get the full logs
+    response = r.path("ws").path("v1").path("node")
+        .path("containerlogs").path(containerIdStr).path(filename)
+        .queryParam("size", "10000")
+        .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals(fullTextSize, responseText.getBytes().length);
+    assertEquals(logMessage, responseText);
+
+    // specify a negative number, it would get the last n bytes from
+    // container log
+    response = r.path("ws").path("v1").path("node")
+        .path("containerlogs").path(containerIdStr).path(filename)
+        .queryParam("size", "-5")
+        .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals(5, responseText.getBytes().length);
+    assertEquals(new String(logMessage.getBytes(),
+        logMessage.getBytes().length - 5, 5), responseText);
+    assertTrue(fullTextSize >= responseText.getBytes().length);
+
+    response = r.path("ws").path("v1").path("node")
+        .path("containerlogs").path(containerIdStr).path(filename)
+        .queryParam("size", "-10000")
+        .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals(fullTextSize, responseText.getBytes().length);
+    assertEquals(logMessage, responseText);
 
     // ask and download it
     response = r.path("ws").path("v1").path("node").path("containerlogs")


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


[46/47] hadoop git commit: HADOOP-10048. LocalDirAllocator should avoid holding locks while accessing the filesystem. Contributed by Jason Lowe.

Posted by ae...@apache.org.
HADOOP-10048. LocalDirAllocator should avoid holding locks while accessing the filesystem. Contributed by Jason Lowe.


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

Branch: refs/heads/HDFS-1312
Commit: c14c1b298e29e799f7c8f15ff24d7eba6e0cd39b
Parents: e620530
Author: Junping Du <ju...@apache.org>
Authored: Tue Jun 7 09:18:58 2016 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Tue Jun 7 09:18:58 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/LocalDirAllocator.java | 153 ++++++++++++-------
 1 file changed, 94 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c14c1b29/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
index 70cf87d..b14e1f0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
@@ -20,9 +20,10 @@ package org.apache.hadoop.fs;
 
 import java.io.*;
 import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.logging.*;
-
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -247,74 +248,101 @@ public class LocalDirAllocator {
     private final Log LOG =
       LogFactory.getLog(AllocatorPerContext.class);
 
-    private int dirNumLastAccessed;
     private Random dirIndexRandomizer = new Random();
-    private FileSystem localFS;
-    private DF[] dirDF = new DF[0];
     private String contextCfgItemName;
-    private String[] localDirs = new String[0];
-    private String savedLocalDirs = "";
+
+    // NOTE: the context must be accessed via a local reference as it
+    //       may be updated at any time to reference a different context
+    private AtomicReference<Context> currentContext;
+
+    private static class Context {
+      private AtomicInteger dirNumLastAccessed = new AtomicInteger(0);
+      private FileSystem localFS;
+      private DF[] dirDF;
+      private Path[] localDirs;
+      private String savedLocalDirs;
+
+      public int getAndIncrDirNumLastAccessed() {
+        return getAndIncrDirNumLastAccessed(1);
+      }
+
+      public int getAndIncrDirNumLastAccessed(int delta) {
+        if (localDirs.length < 2 || delta == 0) {
+          return dirNumLastAccessed.get();
+        }
+        int oldval, newval;
+        do {
+          oldval = dirNumLastAccessed.get();
+          newval = (oldval + delta) % localDirs.length;
+        } while (!dirNumLastAccessed.compareAndSet(oldval, newval));
+        return oldval;
+      }
+    }
 
     public AllocatorPerContext(String contextCfgItemName) {
       this.contextCfgItemName = contextCfgItemName;
+      this.currentContext = new AtomicReference<Context>(new Context());
     }
 
     /** This method gets called everytime before any read/write to make sure
      * that any change to localDirs is reflected immediately.
      */
-    private synchronized void confChanged(Configuration conf) 
+    private Context confChanged(Configuration conf)
         throws IOException {
+      Context ctx = currentContext.get();
       String newLocalDirs = conf.get(contextCfgItemName);
       if (null == newLocalDirs) {
         throw new IOException(contextCfgItemName + " not configured");
       }
-      if (!newLocalDirs.equals(savedLocalDirs)) {
-        localDirs = StringUtils.getTrimmedStrings(newLocalDirs);
-        localFS = FileSystem.getLocal(conf);
-        int numDirs = localDirs.length;
-        ArrayList<String> dirs = new ArrayList<String>(numDirs);
+      if (!newLocalDirs.equals(ctx.savedLocalDirs)) {
+        ctx = new Context();
+        String[] dirStrings = StringUtils.getTrimmedStrings(newLocalDirs);
+        ctx.localFS = FileSystem.getLocal(conf);
+        int numDirs = dirStrings.length;
+        ArrayList<Path> dirs = new ArrayList<Path>(numDirs);
         ArrayList<DF> dfList = new ArrayList<DF>(numDirs);
         for (int i = 0; i < numDirs; i++) {
           try {
             // filter problematic directories
-            Path tmpDir = new Path(localDirs[i]);
-            if(localFS.mkdirs(tmpDir)|| localFS.exists(tmpDir)) {
+            Path tmpDir = new Path(dirStrings[i]);
+            if(ctx.localFS.mkdirs(tmpDir)|| ctx.localFS.exists(tmpDir)) {
               try {
-
                 File tmpFile = tmpDir.isAbsolute()
-                  ? new File(localFS.makeQualified(tmpDir).toUri())
-                  : new File(localDirs[i]);
+                    ? new File(ctx.localFS.makeQualified(tmpDir).toUri())
+                    : new File(dirStrings[i]);
 
                 DiskChecker.checkDir(tmpFile);
-                dirs.add(tmpFile.getPath());
+                dirs.add(new Path(tmpFile.getPath()));
                 dfList.add(new DF(tmpFile, 30000));
-
               } catch (DiskErrorException de) {
-                LOG.warn( localDirs[i] + " is not writable\n", de);
+                LOG.warn(dirStrings[i] + " is not writable\n", de);
               }
             } else {
-              LOG.warn( "Failed to create " + localDirs[i]);
+              LOG.warn("Failed to create " + dirStrings[i]);
             }
           } catch (IOException ie) { 
-            LOG.warn( "Failed to create " + localDirs[i] + ": " +
+            LOG.warn("Failed to create " + dirStrings[i] + ": " +
                 ie.getMessage() + "\n", ie);
           } //ignore
         }
-        localDirs = dirs.toArray(new String[dirs.size()]);
-        dirDF = dfList.toArray(new DF[dirs.size()]);
-        savedLocalDirs = newLocalDirs;
-        
+        ctx.localDirs = dirs.toArray(new Path[dirs.size()]);
+        ctx.dirDF = dfList.toArray(new DF[dirs.size()]);
+        ctx.savedLocalDirs = newLocalDirs;
+
         if (dirs.size() > 0) {
           // randomize the first disk picked in the round-robin selection
-          dirNumLastAccessed = dirIndexRandomizer.nextInt(dirs.size());
+          ctx.dirNumLastAccessed.set(dirIndexRandomizer.nextInt(dirs.size()));
         }
+
+        currentContext.set(ctx);
       }
+
+      return ctx;
     }
 
-    private Path createPath(String path, 
+    private Path createPath(Path dir, String path,
         boolean checkWrite) throws IOException {
-      Path file = new Path(new Path(localDirs[dirNumLastAccessed]),
-                                    path);
+      Path file = new Path(dir, path);
       if (checkWrite) {
         //check whether we are able to create a directory here. If the disk
         //happens to be RDONLY we will fail
@@ -334,7 +362,7 @@ public class LocalDirAllocator {
      * @return the current directory index.
      */
     int getCurrentDirectoryIndex() {
-      return dirNumLastAccessed;
+      return currentContext.get().dirNumLastAccessed.get();
     }
 
     /** Get a path from the local FS. If size is known, we go
@@ -344,10 +372,10 @@ public class LocalDirAllocator {
      *  If size is not known, use roulette selection -- pick directories
      *  with probability proportional to their available space.
      */
-    public synchronized Path getLocalPathForWrite(String pathStr, long size, 
+    public Path getLocalPathForWrite(String pathStr, long size,
         Configuration conf, boolean checkWrite) throws IOException {
-      confChanged(conf);
-      int numDirs = localDirs.length;
+      Context ctx = confChanged(conf);
+      int numDirs = ctx.localDirs.length;
       int numDirsSearched = 0;
       //remove the leading slash from the path (to make sure that the uri
       //resolution results in a valid path on the dir being checked)
@@ -358,12 +386,12 @@ public class LocalDirAllocator {
       
       if(size == SIZE_UNKNOWN) {  //do roulette selection: pick dir with probability 
                     //proportional to available size
-        long[] availableOnDisk = new long[dirDF.length];
+        long[] availableOnDisk = new long[ctx.dirDF.length];
         long totalAvailable = 0;
         
             //build the "roulette wheel"
-        for(int i =0; i < dirDF.length; ++i) {
-          availableOnDisk[i] = dirDF[i].getAvailable();
+        for(int i =0; i < ctx.dirDF.length; ++i) {
+          availableOnDisk[i] = ctx.dirDF[i].getAvailable();
           totalAvailable += availableOnDisk[i];
         }
 
@@ -380,8 +408,8 @@ public class LocalDirAllocator {
             randomPosition -= availableOnDisk[dir];
             dir++;
           }
-          dirNumLastAccessed = dir;
-          returnPath = createPath(pathStr, checkWrite);
+          ctx.dirNumLastAccessed.set(dir);
+          returnPath = createPath(ctx.localDirs[dir], pathStr, checkWrite);
           if (returnPath == null) {
             totalAvailable -= availableOnDisk[dir];
             availableOnDisk[dir] = 0; // skip this disk
@@ -389,15 +417,21 @@ public class LocalDirAllocator {
           }
         }
       } else {
-        while (numDirsSearched < numDirs && returnPath == null) {
-          long capacity = dirDF[dirNumLastAccessed].getAvailable();
+        int dirNum = ctx.getAndIncrDirNumLastAccessed();
+        while (numDirsSearched < numDirs) {
+          long capacity = ctx.dirDF[dirNum].getAvailable();
           if (capacity > size) {
-            returnPath = createPath(pathStr, checkWrite);
+            returnPath =
+                createPath(ctx.localDirs[dirNum], pathStr, checkWrite);
+            if (returnPath != null) {
+              ctx.getAndIncrDirNumLastAccessed(numDirsSearched);
+              break;
+            }
           }
-          dirNumLastAccessed++;
-          dirNumLastAccessed = dirNumLastAccessed % numDirs; 
+          dirNum++;
+          dirNum = dirNum % numDirs;
           numDirsSearched++;
-        } 
+        }
       }
       if (returnPath != null) {
         return returnPath;
@@ -432,10 +466,10 @@ public class LocalDirAllocator {
      *  configured dirs for the file's existence and return the complete
      *  path to the file when we find one 
      */
-    public synchronized Path getLocalPathToRead(String pathStr, 
+    public Path getLocalPathToRead(String pathStr,
         Configuration conf) throws IOException {
-      confChanged(conf);
-      int numDirs = localDirs.length;
+      Context ctx = confChanged(conf);
+      int numDirs = ctx.localDirs.length;
       int numDirsSearched = 0;
       //remove the leading slash from the path (to make sure that the uri
       //resolution results in a valid path on the dir being checked)
@@ -443,8 +477,8 @@ public class LocalDirAllocator {
         pathStr = pathStr.substring(1);
       }
       while (numDirsSearched < numDirs) {
-        Path file = new Path(localDirs[numDirsSearched], pathStr);
-        if (localFS.exists(file)) {
+        Path file = new Path(ctx.localDirs[numDirsSearched], pathStr);
+        if (ctx.localFS.exists(file)) {
           return file;
         }
         numDirsSearched++;
@@ -459,10 +493,10 @@ public class LocalDirAllocator {
       private final FileSystem fs;
       private final String pathStr;
       private int i = 0;
-      private final String[] rootDirs;
+      private final Path[] rootDirs;
       private Path next = null;
 
-      private PathIterator(FileSystem fs, String pathStr, String[] rootDirs)
+      private PathIterator(FileSystem fs, String pathStr, Path[] rootDirs)
           throws IOException {
         this.fs = fs;
         this.pathStr = pathStr;
@@ -517,21 +551,22 @@ public class LocalDirAllocator {
      * @return all of the paths that exist under any of the roots
      * @throws IOException
      */
-    synchronized Iterable<Path> getAllLocalPathsToRead(String pathStr,
+    Iterable<Path> getAllLocalPathsToRead(String pathStr,
         Configuration conf) throws IOException {
-      confChanged(conf);
+      Context ctx = confChanged(conf);
       if (pathStr.startsWith("/")) {
         pathStr = pathStr.substring(1);
       }
-      return new PathIterator(localFS, pathStr, localDirs);
+      return new PathIterator(ctx.localFS, pathStr, ctx.localDirs);
     }
 
     /** We search through all the configured dirs for the file's existence
      *  and return true when we find one 
      */
-    public synchronized boolean ifExists(String pathStr,Configuration conf) {
+    public boolean ifExists(String pathStr, Configuration conf) {
+      Context ctx = currentContext.get();
       try {
-        int numDirs = localDirs.length;
+        int numDirs = ctx.localDirs.length;
         int numDirsSearched = 0;
         //remove the leading slash from the path (to make sure that the uri
         //resolution results in a valid path on the dir being checked)
@@ -539,8 +574,8 @@ public class LocalDirAllocator {
           pathStr = pathStr.substring(1);
         }
         while (numDirsSearched < numDirs) {
-          Path file = new Path(localDirs[numDirsSearched], pathStr);
-          if (localFS.exists(file)) {
+          Path file = new Path(ctx.localDirs[numDirsSearched], pathStr);
+          if (ctx.localFS.exists(file)) {
             return true;
           }
           numDirsSearched++;


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


[42/47] hadoop git commit: YARN-5185. StageAllocaterGreedyRLE: Fix NPE in corner case. (Carlo Curino via asuresh)

Posted by ae...@apache.org.
YARN-5185. StageAllocaterGreedyRLE: Fix NPE in corner case. (Carlo Curino via asuresh)


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

Branch: refs/heads/HDFS-1312
Commit: 7a9b7372a1a917c7b5e1beca7e13c0419e3dbfef
Parents: 6de9213
Author: Arun Suresh <as...@apache.org>
Authored: Mon Jun 6 21:06:52 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Jun 6 21:06:52 2016 -0700

----------------------------------------------------------------------
 .../planning/StageAllocatorGreedyRLE.java           | 16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a9b7372/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
index c5a3192..5e748fc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/planning/StageAllocatorGreedyRLE.java
@@ -168,12 +168,20 @@ public class StageAllocatorGreedyRLE implements StageAllocator {
       if (allocateLeft) {
         // set earliest start to the min of the constraining "range" or my the
         // end of this allocation
-        stageEarliestStart =
-            Math.min(partialMap.higherKey(minPoint), stageEarliestStart + dur);
+        if(partialMap.higherKey(minPoint) == null){
+          stageEarliestStart = stageEarliestStart + dur;
+        } else {
+          stageEarliestStart =
+             Math.min(partialMap.higherKey(minPoint), stageEarliestStart + dur);
+        }
       } else {
         // same as above moving right-to-left
-        stageDeadline =
-            Math.max(partialMap.higherKey(minPoint), stageDeadline - dur);
+        if(partialMap.higherKey(minPoint) == null){
+          stageDeadline = stageDeadline - dur;
+        } else {
+          stageDeadline =
+              Math.max(partialMap.higherKey(minPoint), stageDeadline - dur);
+        }
       }
     }
 


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


[44/47] hadoop git commit: YARN-5118. Tests fails with localizer port bind exception. Contributed by Brahma Reddy Battula.

Posted by ae...@apache.org.
YARN-5118. Tests fails with localizer port bind exception. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-1312
Commit: bddea5fe5fe72eee8e2ecfcec616bd8ceb4d72e7
Parents: 3a154f7
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Tue Jun 7 11:20:15 2016 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Tue Jun 7 11:20:15 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/yarn/server/nodemanager/TestEventFlow.java  | 3 +++
 .../server/nodemanager/TestNodeStatusUpdaterForLabels.java    | 7 +++++++
 .../containermanager/BaseContainerManagerTest.java            | 3 +++
 3 files changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bddea5fe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
index f126080..a9ff83c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
@@ -25,6 +25,7 @@ import java.util.List;
 
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
@@ -91,6 +92,8 @@ public class TestEventFlow {
     conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, 
         remoteLogDir.getAbsolutePath());
+    conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "0.0.0.0:"
+        + ServerSocketUtil.getPort(8040, 10));
 
     ContainerExecutor exec = new DefaultContainerExecutor();
     exec.setConf(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bddea5fe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
index 563104e..257e18c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
@@ -28,6 +28,7 @@ import java.nio.ByteBuffer;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -233,6 +234,9 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
 
     YarnConfiguration conf = createNMConfigForDistributeNodeLabels();
     conf.setLong(YarnConfiguration.NM_NODE_LABELS_RESYNC_INTERVAL, 2000);
+    conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "0.0.0.0:"
+        + ServerSocketUtil.getPort(8040, 10));
+
     nm.init(conf);
     resourceTracker.resetNMHeartbeatReceiveFlag();
     nm.start();
@@ -329,6 +333,9 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
     };
     dummyLabelsProviderRef.setNodeLabels(toNodeLabelSet("P"));
     YarnConfiguration conf = createNMConfigForDistributeNodeLabels();
+    conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "0.0.0.0:"
+        + ServerSocketUtil.getPort(8040, 10));
+
     nm.init(conf);
     resourceTracker.resetNMHeartbeatReceiveFlag();
     nm.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bddea5fe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index 638e51f..ab60288 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -172,6 +172,9 @@ public abstract class BaseContainerManagerTest {
     conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath());
+    conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "0.0.0.0:"
+        + ServerSocketUtil.getPort(8040, 10));
+
 
     conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
     // Default delSrvc


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


[28/47] hadoop git commit: Revert "HDFS-10346. Implement asynchronous setPermission/setOwner for DistributedFileSystem. Contributed by Xiaobing Zhou"

Posted by ae...@apache.org.
Revert "HDFS-10346. Implement asynchronous setPermission/setOwner for DistributedFileSystem.  Contributed by  Xiaobing Zhou"

This reverts commit 7251bb922b20dae49c8c6854864095fb16d8cbd5.


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

Branch: refs/heads/HDFS-1312
Commit: f23d5dfc60a017187ae57f3667ac0e688877c2dd
Parents: e4450d4
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 18:09:17 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 18:09:17 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |  59 ----
 .../ClientNamenodeProtocolTranslatorPB.java     |  39 +--
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 267 ++-----------------
 .../apache/hadoop/hdfs/TestDFSPermission.java   |  29 +-
 4 files changed, 43 insertions(+), 351 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f23d5dfc/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 4fe0861..356ae3f 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
@@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 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.FsPermission;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
 import org.apache.hadoop.ipc.Client;
 
@@ -38,9 +37,6 @@ import com.google.common.util.concurrent.AbstractFuture;
  * This instance of this class is the way end-user code interacts
  * with a Hadoop DistributedFileSystem in an asynchronous manner.
  *
- * This class is unstable, so no guarantee is provided as to reliability,
- * stability or compatibility across any level of release granularity.
- *
  *****************************************************************/
 @Unstable
 public class AsyncDistributedFileSystem {
@@ -115,59 +111,4 @@ public class AsyncDistributedFileSystem {
       Client.setAsynchronousMode(isAsync);
     }
   }
-
-  /**
-   * Set permission of a path.
-   *
-   * @param p
-   *          the path the permission is set to
-   * @param permission
-   *          the permission that is set to a path.
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<Void> setPermission(Path p, final FsPermission permission)
-      throws IOException {
-    dfs.getFsStatistics().incrementWriteOps(1);
-    final Path absPath = dfs.fixRelativePart(p);
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().setPermission(dfs.getPathName(absPath), permission);
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
-
-  /**
-   * Set owner of a path (i.e. a file or a directory). The parameters username
-   * and groupname cannot both be null.
-   *
-   * @param p
-   *          The path
-   * @param username
-   *          If it is null, the original username remains unchanged.
-   * @param groupname
-   *          If it is null, the original groupname remains unchanged.
-   * @return an instance of Future, #get of which is invoked to wait for
-   *         asynchronous call being finished.
-   */
-  public Future<Void> setOwner(Path p, String username, String groupname)
-      throws IOException {
-    if (username == null && groupname == null) {
-      throw new IOException("username == null && groupname == null");
-    }
-
-    dfs.getFsStatistics().incrementWriteOps(1);
-    final Path absPath = dfs.fixRelativePart(p);
-    final boolean isAsync = Client.isAsynchronousMode();
-    Client.setAsynchronousMode(true);
-    try {
-      dfs.getClient().setOwner(dfs.getPathName(absPath), username, groupname);
-      return getReturnValue();
-    } finally {
-      Client.setAsynchronousMode(isAsync);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f23d5dfc/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 faa925c..f4074b6 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
@@ -367,30 +367,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setPermission(PBHelperClient.convert(permission))
         .build();
     try {
-      if (Client.isAsynchronousMode()) {
-        rpcProxy.setPermission(null, req);
-        setReturnValueCallback();
-      } else {
-        rpcProxy.setPermission(null, req);
-      }
+      rpcProxy.setPermission(null, req);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
-  private void setReturnValueCallback() {
-    final Callable<Message> returnMessageCallback = ProtobufRpcEngine
-        .getReturnMessageCallback();
-    Callable<Void> callBack = new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        returnMessageCallback.call();
-        return null;
-      }
-    };
-    RETURN_VALUE_CALLBACK.set(callBack);
-  }
-
   @Override
   public void setOwner(String src, String username, String groupname)
       throws IOException {
@@ -401,12 +383,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     if (groupname != null)
       req.setGroupname(groupname);
     try {
-      if (Client.isAsynchronousMode()) {
-        rpcProxy.setOwner(null, req.build());
-        setReturnValueCallback();
-      } else {
-        rpcProxy.setOwner(null, req.build());
-      }
+      rpcProxy.setOwner(null, req.build());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -536,7 +513,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       if (Client.isAsynchronousMode()) {
         rpcProxy.rename2(null, req);
-        setReturnValueCallback();
+
+        final Callable<Message> returnMessageCallback = ProtobufRpcEngine
+            .getReturnMessageCallback();
+        Callable<Void> callBack = new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            returnMessageCallback.call();
+            return null;
+          }
+        };
+        RETURN_VALUE_CALLBACK.set(callBack);
       } else {
         rpcProxy.rename2(null, req);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f23d5dfc/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 7539fbd..d129299 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
@@ -22,11 +22,8 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
@@ -34,30 +31,18 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 public class TestAsyncDFSRename {
   public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
-  private final long seed = Time.now();
-  private final Random r = new Random(seed);
-  private final PermissionGenerator permGenerator = new PermissionGenerator(r);
-  private final short replFactor = 2;
-  private final long blockSize = 512;
-  private long fileLen = blockSize * 3;
 
   /**
    * Check the blocks of dst file are cleaned after rename with overwrite
@@ -65,6 +50,8 @@ public class TestAsyncDFSRename {
    */
   @Test(timeout = 60000)
   public void testAsyncRenameWithOverwrite() throws Exception {
+    final short replFactor = 2;
+    final long blockSize = 512;
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
         replFactor).build();
@@ -73,6 +60,8 @@ public class TestAsyncDFSRename {
     AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
 
     try {
+
+      long fileLen = blockSize * 3;
       String src = "/foo/src";
       String dst = "/foo/dst";
       String src2 = "/foo/src2";
@@ -126,6 +115,8 @@ public class TestAsyncDFSRename {
 
   @Test(timeout = 60000)
   public void testCallGetReturnValueMultipleTimes() throws Exception {
+    final short replFactor = 2;
+    final long blockSize = 512;
     final Path renameDir = new Path(
         "/test/testCallGetReturnValueMultipleTimes/");
     final Configuration conf = new HdfsConfiguration();
@@ -136,6 +127,7 @@ public class TestAsyncDFSRename {
     final DistributedFileSystem dfs = cluster.getFileSystem();
     final AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
     final int count = 100;
+    long fileLen = blockSize * 3;
     final Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
 
     assertTrue(dfs.mkdirs(renameDir));
@@ -186,15 +178,15 @@ public class TestAsyncDFSRename {
     }
   }
 
-  @Test
-  public void testConservativeConcurrentAsyncRenameWithOverwrite()
+  @Test(timeout = 120000)
+  public void testAggressiveConcurrentAsyncRenameWithOverwrite()
       throws Exception {
     internalTestConcurrentAsyncRenameWithOverwrite(100,
         "testAggressiveConcurrentAsyncRenameWithOverwrite");
   }
 
   @Test(timeout = 60000)
-  public void testAggressiveConcurrentAsyncRenameWithOverwrite()
+  public void testConservativeConcurrentAsyncRenameWithOverwrite()
       throws Exception {
     internalTestConcurrentAsyncRenameWithOverwrite(10000,
         "testConservativeConcurrentAsyncRenameWithOverwrite");
@@ -202,6 +194,8 @@ public class TestAsyncDFSRename {
 
   private void internalTestConcurrentAsyncRenameWithOverwrite(
       final int asyncCallLimit, final String basePath) throws Exception {
+    final short replFactor = 2;
+    final long blockSize = 512;
     final Path renameDir = new Path(String.format("/test/%s/", basePath));
     Configuration conf = new HdfsConfiguration();
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
@@ -212,6 +206,7 @@ public class TestAsyncDFSRename {
     DistributedFileSystem dfs = cluster.getFileSystem();
     AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
     int count = 1000;
+    long fileLen = blockSize * 3;
     int start = 0, end = 0;
     Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
 
@@ -279,206 +274,8 @@ public class TestAsyncDFSRename {
     }
   }
 
-  @Test
-  public void testConservativeConcurrentAsyncAPI() throws Exception {
-    internalTestConcurrentAsyncAPI(100, "testConservativeConcurrentAsyncAPI");
-  }
-
-  @Test(timeout = 60000)
-  public void testAggressiveConcurrentAsyncAPI() throws Exception {
-    internalTestConcurrentAsyncAPI(10000, "testAggressiveConcurrentAsyncAPI");
-  }
-
-  private void internalTestConcurrentAsyncAPI(final int asyncCallLimit,
-      final String basePath) throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    String group1 = "group1";
-    String group2 = "group2";
-    String user1 = "user1";
-    int count = 500;
-
-    // explicitly turn on permission checking
-    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-    // set the limit of max async calls
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
-        asyncCallLimit);
-
-    // 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);
-
-    // start mini cluster
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(3).build();
-    cluster.waitActive();
-    AsyncDistributedFileSystem adfs = cluster.getFileSystem()
-        .getAsyncDistributedFileSystem();
-
-    // prepare for test
-    FileSystem rootFs = FileSystem.get(conf);
-    final Path parent = new Path(String.format("/test/%s/", basePath));
-    final Path[] srcs = new Path[count];
-    final Path[] dsts = new Path[count];
-    short[] permissions = new short[count];
-    for (int i = 0; i < count; i++) {
-      srcs[i] = new Path(parent, "src" + i);
-      dsts[i] = new Path(parent, "dst" + i);
-      DFSTestUtil.createFile(rootFs, srcs[i], fileLen, replFactor, 1);
-      DFSTestUtil.createFile(rootFs, dsts[i], fileLen, replFactor, 1);
-      assertTrue(rootFs.exists(srcs[i]));
-      assertTrue(rootFs.getFileStatus(srcs[i]).isFile());
-      assertTrue(rootFs.exists(dsts[i]));
-      assertTrue(rootFs.getFileStatus(dsts[i]).isFile());
-      permissions[i] = permGenerator.next();
-    }
-
-    Map<Integer, Future<Void>> renameRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<Void>> permRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<Void>> ownerRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    int start = 0, end = 0;
-    // test rename
-    for (int i = 0; i < count; i++) {
-      for (;;) {
-        try {
-          Future<Void> returnFuture = adfs.rename(srcs[i], dsts[i],
-              Rename.OVERWRITE);
-          renameRetFutures.put(i, returnFuture);
-          break;
-        } catch (AsyncCallLimitExceededException e) {
-          start = end;
-          end = i;
-          waitForReturnValues(renameRetFutures, start, end);
-        }
-      }
-    }
-
-    // wait for completing the calls
-    for (int i = start; i < count; i++) {
-      renameRetFutures.get(i).get();
-    }
-
-    // Restart NN and check the rename successfully
-    cluster.restartNameNodes();
-
-    // very the src should not exist, dst should
-    for (int i = 0; i < count; i++) {
-      assertFalse(rootFs.exists(srcs[i]));
-      assertTrue(rootFs.exists(dsts[i]));
-    }
-
-    // test permissions
-    try {
-      for (int i = 0; i < count; i++) {
-        for (;;) {
-          try {
-            Future<Void> retFuture = adfs.setPermission(dsts[i],
-                new FsPermission(permissions[i]));
-            permRetFutures.put(i, retFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            start = end;
-            end = i;
-            waitForReturnValues(permRetFutures, start, end);
-          }
-        }
-      }
-      // wait for completing the calls
-      for (int i = start; i < count; i++) {
-        permRetFutures.get(i).get();
-      }
-
-      // Restart NN and check permission then
-      cluster.restartNameNodes();
-
-      // verify the permission
-      for (int i = 0; i < count; i++) {
-        assertTrue(rootFs.exists(dsts[i]));
-        FsPermission fsPerm = new FsPermission(permissions[i]);
-        checkAccessPermissions(rootFs.getFileStatus(dsts[i]),
-            fsPerm.getUserAction());
-      }
-
-      // test setOwner
-      start = 0;
-      end = 0;
-      for (int i = 0; i < count; i++) {
-        for (;;) {
-          try {
-            Future<Void> retFuture = adfs.setOwner(dsts[i], "user1",
-                "group2");
-            ownerRetFutures.put(i, retFuture);
-            break;
-          } catch (AsyncCallLimitExceededException e) {
-            start = end;
-            end = i;
-            waitForReturnValues(ownerRetFutures, start, end);
-          }
-        }
-      }
-      // wait for completing the calls
-      for (int i = start; i < count; i++) {
-        ownerRetFutures.get(i).get();
-      }
-
-      // Restart NN and check owner then
-      cluster.restartNameNodes();
-
-      // verify the owner
-      for (int i = 0; i < count; i++) {
-        assertTrue(rootFs.exists(dsts[i]));
-        assertTrue(
-            "user1".equals(rootFs.getFileStatus(dsts[i]).getOwner()));
-        assertTrue(
-            "group2".equals(rootFs.getFileStatus(dsts[i]).getGroup()));
-      }
-    } catch (AccessControlException ace) {
-      throw ace;
-    } finally {
-      if (rootFs != null) {
-        rootFs.close();
-      }
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-
-  static void checkAccessPermissions(FileStatus stat, FsAction mode)
-      throws IOException {
-    checkAccessPermissions(UserGroupInformation.getCurrentUser(), stat, mode);
-  }
-
-  static void checkAccessPermissions(final UserGroupInformation ugi,
-      FileStatus stat, FsAction mode) throws IOException {
-    FsPermission perm = stat.getPermission();
-    String user = ugi.getShortUserName();
-    List<String> groups = Arrays.asList(ugi.getGroupNames());
-
-    if (user.equals(stat.getOwner())) {
-      if (perm.getUserAction().implies(mode)) {
-        return;
-      }
-    } else if (groups.contains(stat.getGroup())) {
-      if (perm.getGroupAction().implies(mode)) {
-        return;
-      }
-    } else {
-      if (perm.getOtherAction().implies(mode)) {
-        return;
-      }
-    }
-    throw new AccessControlException(String.format(
-        "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat
-            .getPath(), stat.getOwner(), stat.getGroup(),
-        stat.isDirectory() ? "d" : "-", perm));
-  }
-
   @Test(timeout = 60000)
-  public void testAsyncAPIWithException() throws Exception {
+  public void testAsyncRenameWithException() throws Exception {
     Configuration conf = new HdfsConfiguration();
     String group1 = "group1";
     String group2 = "group2";
@@ -489,9 +286,9 @@ public class TestAsyncDFSRename {
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
 
     // 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);
+    Map<String, String[]> u2g_map = new HashMap<String, String[]>(1);
+    u2g_map.put(user1, new String[] { group1, group2 });
+    DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
 
     // Initiate all four users
     ugi1 = UserGroupInformation.createUserForTesting(user1, new String[] {
@@ -502,7 +299,7 @@ public class TestAsyncDFSRename {
     cluster.waitActive();
 
     FileSystem rootFs = FileSystem.get(conf);
-    final Path renameDir = new Path("/test/async_api_exception/");
+    final Path renameDir = new Path("/test/async_rename_exception/");
     final Path src = new Path(renameDir, "src");
     final Path dst = new Path(renameDir, "dst");
     rootFs.mkdirs(src);
@@ -515,33 +312,11 @@ public class TestAsyncDFSRename {
           }
         });
 
-    Future<Void> retFuture;
-    try {
-      retFuture = adfs.rename(src, dst, Rename.OVERWRITE);
-      retFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-      assertTrue("Permission denied messages must carry the path parent", e
-          .getMessage().contains(src.getParent().toUri().getPath()));
-    }
-
-    FsPermission fsPerm = new FsPermission(permGenerator.next());
-    try {
-      retFuture = adfs.setPermission(src, fsPerm);
-      retFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-      assertTrue("Permission denied messages must carry the name of the path",
-          e.getMessage().contains(src.getName()));
-    }
-
     try {
-      retFuture = adfs.setOwner(src, "user1", "group2");
-      retFuture.get();
+      Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+      returnFuture.get();
     } catch (ExecutionException e) {
       checkPermissionDenied(e, src, user1);
-      assertTrue("Permission denied messages must carry the name of the path",
-          e.getMessage().contains(src.getName()));
     } finally {
       if (rootFs != null) {
         rootFs.close();
@@ -559,5 +334,7 @@ public class TestAsyncDFSRename {
         e.getMessage().contains("AccessControlException"));
     assertTrue("Permission denied messages must carry the username", e
         .getMessage().contains(user));
+    assertTrue("Permission denied messages must carry the path parent", e
+        .getMessage().contains(dir.getParent().toUri().getPath()));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f23d5dfc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
index 66a0380..aa204cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
@@ -196,35 +196,22 @@ public class TestDFSPermission {
     return fs.getFileStatus(path).getPermission().toShort();
   }
 
-  private void create(OpType op, Path name) throws IOException {
-    create(fs, conf, op, name);
-  }
-
   /* create a file/directory with the default umask and permission */
-  static void create(final FileSystem fs, final Configuration fsConf,
-      OpType op, Path name) throws IOException {
-    create(fs, fsConf, op, name, DEFAULT_UMASK, new FsPermission(
-        DEFAULT_PERMISSION));
-  }
-
-  private void create(OpType op, Path name, short umask,
-      FsPermission permission)
-      throws IOException {
-    create(fs, conf, op, name, umask, permission);
+  private void create(OpType op, Path name) throws IOException {
+    create(op, name, DEFAULT_UMASK, new FsPermission(DEFAULT_PERMISSION));
   }
 
   /* create a file/directory with the given umask and permission */
-  static void create(final FileSystem fs, final Configuration fsConf,
-      OpType op, Path name, short umask, FsPermission permission)
-      throws IOException {
+  private void create(OpType op, Path name, short umask, 
+      FsPermission permission) throws IOException {
     // set umask in configuration, converting to padded octal
-    fsConf.set(FsPermission.UMASK_LABEL, String.format("%1$03o", umask));
+    conf.set(FsPermission.UMASK_LABEL, String.format("%1$03o", umask));
 
     // create the file/directory
     switch (op) {
     case CREATE:
       FSDataOutputStream out = fs.create(name, permission, true, 
-          fsConf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+          conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
           fs.getDefaultReplication(name), fs.getDefaultBlockSize(name), null);
       out.close();
       break;
@@ -372,7 +359,7 @@ public class TestDFSPermission {
   final static private String DIR_NAME = "dir";
   final static private String FILE_DIR_NAME = "filedir";
 
-  enum OpType {CREATE, MKDIRS, OPEN, SET_REPLICATION,
+  private enum OpType {CREATE, MKDIRS, OPEN, SET_REPLICATION,
     GET_FILEINFO, IS_DIR, EXISTS, GET_CONTENT_LENGTH, LIST, RENAME, DELETE
   };
 
@@ -628,7 +615,7 @@ public class TestDFSPermission {
   /* A random permission generator that guarantees that each permission
    * value is generated only once.
    */
-  static class PermissionGenerator {
+  static private class PermissionGenerator {
     private final Random r;
     private final short[] permissions = new short[MAX_PERMISSION + 1];
     private int numLeft = MAX_PERMISSION + 1;


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


[38/47] hadoop git commit: Revert "Revert "HADOOP-13226 Support async call retry and failover.""

Posted by ae...@apache.org.
Revert "Revert "HADOOP-13226 Support async call retry and failover.""

This reverts commit 5360da8bd9f720384860f411bee081aef13b4bd4.


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

Branch: refs/heads/HDFS-1312
Commit: 35f255b03b1bb5c94063ec1818af1d253ceee991
Parents: 7e7b1ae
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 6 16:31:43 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 6 16:31:43 2016 +0800

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   8 +-
 .../hadoop/io/retry/AsyncCallHandler.java       | 321 +++++++++++++++++++
 .../org/apache/hadoop/io/retry/CallReturn.java  |  75 +++++
 .../hadoop/io/retry/RetryInvocationHandler.java | 134 ++++++--
 .../apache/hadoop/io/retry/RetryPolicies.java   |   4 +-
 .../main/java/org/apache/hadoop/ipc/Client.java |  25 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |  13 +-
 .../apache/hadoop/util/concurrent/AsyncGet.java |  17 +-
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     |  10 +-
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |   7 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  42 +--
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    |  43 +--
 .../apache/hadoop/hdfs/TestAsyncHDFSWithHA.java | 181 +++++++++++
 .../hdfs/server/namenode/ha/HATestUtil.java     |   9 +-
 14 files changed, 775 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
index ab8673b..a644aa5 100644
--- a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
@@ -345,7 +345,13 @@
        <Bug pattern="SF_SWITCH_FALLTHROUGH" />
      </Match>
 
-     <!-- Synchronization performed on util.concurrent instance. -->
+     <!-- WA_NOT_IN_LOOP is invalid in util.concurrent.AsyncGet$Util.wait. -->
+     <Match>
+       <Class name="org.apache.hadoop.util.concurrent.AsyncGet$Util" />
+       <Method name="wait" />
+       <Bug pattern="WA_NOT_IN_LOOP" />
+     </Match>
+
      <Match>
        <Class name="org.apache.hadoop.service.AbstractService" />
        <Method name="stop" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java
new file mode 100644
index 0000000..5a03b03
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java
@@ -0,0 +1,321 @@
+/*
+ * 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.io.retry;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.concurrent.AsyncGet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** Handle async calls. */
+@InterfaceAudience.Private
+public class AsyncCallHandler {
+  static final Logger LOG = LoggerFactory.getLogger(AsyncCallHandler.class);
+
+  private static final ThreadLocal<AsyncGet<?, Exception>>
+      LOWER_LAYER_ASYNC_RETURN = new ThreadLocal<>();
+  private static final ThreadLocal<AsyncGet<Object, Throwable>>
+      ASYNC_RETURN = new ThreadLocal<>();
+
+  /** @return the async return value from {@link AsyncCallHandler}. */
+  @InterfaceStability.Unstable
+  @SuppressWarnings("unchecked")
+  public static <R, T extends  Throwable> AsyncGet<R, T> getAsyncReturn() {
+    final AsyncGet<R, T> asyncGet = (AsyncGet<R, T>)ASYNC_RETURN.get();
+    if (asyncGet != null) {
+      ASYNC_RETURN.set(null);
+      return asyncGet;
+    } else {
+      return (AsyncGet<R, T>) getLowerLayerAsyncReturn();
+    }
+  }
+
+  /** For the lower rpc layers to set the async return value. */
+  @InterfaceStability.Unstable
+  public static void setLowerLayerAsyncReturn(
+      AsyncGet<?, Exception> asyncReturn) {
+    LOWER_LAYER_ASYNC_RETURN.set(asyncReturn);
+  }
+
+  private static AsyncGet<?, Exception> getLowerLayerAsyncReturn() {
+    final AsyncGet<?, Exception> asyncGet = LOWER_LAYER_ASYNC_RETURN.get();
+    Preconditions.checkNotNull(asyncGet);
+    LOWER_LAYER_ASYNC_RETURN.set(null);
+    return asyncGet;
+  }
+
+  /** A simple concurrent queue which keeping track the empty start time. */
+  static class ConcurrentQueue<T> {
+    private final Queue<T> queue = new LinkedList<>();
+    private long emptyStartTime = Time.monotonicNow();
+
+    synchronized int size() {
+      return queue.size();
+    }
+
+    /** Is the queue empty for more than the given time in millisecond? */
+    synchronized boolean isEmpty(long time) {
+      return queue.isEmpty() && Time.monotonicNow() - emptyStartTime > time;
+    }
+
+    synchronized void offer(T c) {
+      final boolean added = queue.offer(c);
+      Preconditions.checkState(added);
+    }
+
+    synchronized T poll() {
+      Preconditions.checkState(!queue.isEmpty());
+      final T t = queue.poll();
+      if (queue.isEmpty()) {
+        emptyStartTime = Time.monotonicNow();
+      }
+      return t;
+    }
+  }
+
+  /** A queue for handling async calls. */
+  static class AsyncCallQueue {
+    private final ConcurrentQueue<AsyncCall> queue = new ConcurrentQueue<>();
+    private final Processor processor = new Processor();
+
+    void addCall(AsyncCall call) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("add " + call);
+      }
+      queue.offer(call);
+      processor.tryStart();
+    }
+
+    void checkCalls() {
+      final int size = queue.size();
+      for (int i = 0; i < size; i++) {
+        final AsyncCall c = queue.poll();
+        if (!c.isDone()) {
+          queue.offer(c); // the call is not done yet, add it back.
+        }
+      }
+    }
+
+    /** Process the async calls in the queue. */
+    private class Processor {
+      static final long GRACE_PERIOD = 10*1000L;
+      static final long SLEEP_PERIOD = 100L;
+
+      private final AtomicReference<Thread> running = new AtomicReference<>();
+
+      boolean isRunning(Daemon d) {
+        return d == running.get();
+      }
+
+      void tryStart() {
+        final Thread current = Thread.currentThread();
+        if (running.compareAndSet(null, current)) {
+          final Daemon daemon = new Daemon() {
+            @Override
+            public void run() {
+              for (; isRunning(this);) {
+                try {
+                  Thread.sleep(SLEEP_PERIOD);
+                } catch (InterruptedException e) {
+                  kill(this);
+                  return;
+                }
+
+                checkCalls();
+                tryStop(this);
+              }
+            }
+          };
+
+          final boolean set = running.compareAndSet(current, daemon);
+          Preconditions.checkState(set);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Starting AsyncCallQueue.Processor " + daemon);
+          }
+          daemon.start();
+        }
+      }
+
+      void tryStop(Daemon d) {
+        if (queue.isEmpty(GRACE_PERIOD)) {
+          kill(d);
+        }
+      }
+
+      void kill(Daemon d) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Killing " + d);
+        }
+        final boolean set = running.compareAndSet(d, null);
+        Preconditions.checkState(set);
+      }
+    }
+  }
+
+  static class AsyncValue<V> {
+    private V value;
+
+    synchronized V waitAsyncValue(long timeout, TimeUnit unit)
+        throws InterruptedException, TimeoutException {
+      if (value != null) {
+        return value;
+      }
+      AsyncGet.Util.wait(this, timeout, unit);
+      if (value != null) {
+        return value;
+      }
+
+      throw new TimeoutException("waitCallReturn timed out "
+          + timeout + " " + unit);
+    }
+
+    synchronized void set(V v) {
+      Preconditions.checkNotNull(v);
+      Preconditions.checkState(value == null);
+      value = v;
+      notify();
+    }
+
+    synchronized boolean isDone() {
+      return value != null;
+    }
+  }
+
+  static class AsyncCall extends RetryInvocationHandler.Call {
+    private final AsyncCallHandler asyncCallHandler;
+
+    private final AsyncValue<CallReturn> asyncCallReturn = new AsyncValue<>();
+    private AsyncGet<?, Exception> lowerLayerAsyncGet;
+
+    AsyncCall(Method method, Object[] args, boolean isRpc, int callId,
+              RetryInvocationHandler.Counters counters,
+              RetryInvocationHandler<?> retryInvocationHandler,
+              AsyncCallHandler asyncCallHandler) {
+      super(method, args, isRpc, callId, counters, retryInvocationHandler);
+
+      this.asyncCallHandler = asyncCallHandler;
+    }
+
+    /** @return true if the call is done; otherwise, return false. */
+    boolean isDone() {
+      final CallReturn r = invokeOnce();
+      switch (r.getState()) {
+        case RETURNED:
+        case EXCEPTION:
+          asyncCallReturn.set(r); // the async call is done
+          return true;
+        case RETRY:
+          invokeOnce();
+          break;
+        case ASYNC_CALL_IN_PROGRESS:
+        case ASYNC_INVOKED:
+          // nothing to do
+          break;
+        default:
+          Preconditions.checkState(false);
+      }
+      return false;
+    }
+
+    @Override
+    CallReturn invoke() throws Throwable {
+      LOG.debug("{}.invoke {}", getClass().getSimpleName(), this);
+      if (lowerLayerAsyncGet != null) {
+        // async call was submitted early, check the lower level async call
+        final boolean isDone = lowerLayerAsyncGet.isDone();
+        LOG.trace("invoke: lowerLayerAsyncGet.isDone()? {}", isDone);
+        if (!isDone) {
+          return CallReturn.ASYNC_CALL_IN_PROGRESS;
+        }
+        try {
+          return new CallReturn(lowerLayerAsyncGet.get(0, TimeUnit.SECONDS));
+        } finally {
+          lowerLayerAsyncGet = null;
+        }
+      }
+
+      // submit a new async call
+      LOG.trace("invoke: ASYNC_INVOKED");
+      final boolean mode = Client.isAsynchronousMode();
+      try {
+        Client.setAsynchronousMode(true);
+        final Object r = invokeMethod();
+        // invokeMethod should set LOWER_LAYER_ASYNC_RETURN and return null.
+        Preconditions.checkState(r == null);
+        lowerLayerAsyncGet = getLowerLayerAsyncReturn();
+
+        if (counters.isZeros()) {
+          // first async attempt, initialize
+          LOG.trace("invoke: initAsyncCall");
+          asyncCallHandler.initAsyncCall(this, asyncCallReturn);
+        }
+        return CallReturn.ASYNC_INVOKED;
+      } finally {
+        Client.setAsynchronousMode(mode);
+      }
+    }
+  }
+
+  private final AsyncCallQueue asyncCalls = new AsyncCallQueue();
+  private volatile boolean hasSuccessfulCall = false;
+
+  AsyncCall newAsyncCall(Method method, Object[] args, boolean isRpc,
+                         int callId, RetryInvocationHandler.Counters counters,
+                         RetryInvocationHandler<?> retryInvocationHandler) {
+    return new AsyncCall(method, args, isRpc, callId, counters,
+        retryInvocationHandler, this);
+  }
+
+  boolean hasSuccessfulCall() {
+    return hasSuccessfulCall;
+  }
+
+  private void initAsyncCall(final AsyncCall asyncCall,
+                             final AsyncValue<CallReturn> asyncCallReturn) {
+    asyncCalls.addCall(asyncCall);
+
+    final AsyncGet<Object, Throwable> asyncGet
+        = new AsyncGet<Object, Throwable>() {
+      @Override
+      public Object get(long timeout, TimeUnit unit) throws Throwable {
+        final CallReturn c = asyncCallReturn.waitAsyncValue(timeout, unit);
+        final Object r = c.getReturnValue();
+        hasSuccessfulCall = true;
+        return r;
+      }
+
+      @Override
+      public boolean isDone() {
+        return asyncCallReturn.isDone();
+      }
+    };
+    ASYNC_RETURN.set(asyncGet);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/CallReturn.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/CallReturn.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/CallReturn.java
new file mode 100644
index 0000000..943725c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/CallReturn.java
@@ -0,0 +1,75 @@
+/*
+ * 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.io.retry;
+
+import com.google.common.base.Preconditions;
+
+/** The call return from a method invocation. */
+class CallReturn {
+  /** The return state. */
+  enum State {
+    /** Call is returned successfully. */
+    RETURNED,
+    /** Call throws an exception. */
+    EXCEPTION,
+    /** Call should be retried according to the {@link RetryPolicy}. */
+    RETRY,
+    /** Call, which is async, is still in progress. */
+    ASYNC_CALL_IN_PROGRESS,
+    /** Call, which is async, just has been invoked. */
+    ASYNC_INVOKED
+  }
+
+  static final CallReturn ASYNC_CALL_IN_PROGRESS = new CallReturn(
+      State.ASYNC_CALL_IN_PROGRESS);
+  static final CallReturn ASYNC_INVOKED = new CallReturn(State.ASYNC_INVOKED);
+  static final CallReturn RETRY = new CallReturn(State.RETRY);
+
+  private final Object returnValue;
+  private final Throwable thrown;
+  private final State state;
+
+  CallReturn(Object r) {
+    this(r, null, State.RETURNED);
+  }
+  CallReturn(Throwable t) {
+    this(null, t, State.EXCEPTION);
+    Preconditions.checkNotNull(t);
+  }
+  private CallReturn(State s) {
+    this(null, null, s);
+  }
+  private CallReturn(Object r, Throwable t, State s) {
+    Preconditions.checkArgument(r == null || t == null);
+    returnValue = r;
+    thrown = t;
+    state = s;
+  }
+
+  State getState() {
+    return state;
+  }
+
+  Object getReturnValue() throws Throwable {
+    if (state == State.EXCEPTION) {
+      throw thrown;
+    }
+    Preconditions.checkState(state == State.RETURNED, "state == %s", state);
+    return returnValue;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
index 300d0c2..f2b2c99 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
@@ -42,11 +42,83 @@ import java.util.Map;
 public class RetryInvocationHandler<T> implements RpcInvocationHandler {
   public static final Log LOG = LogFactory.getLog(RetryInvocationHandler.class);
 
-  private static class Counters {
+  static class Call {
+    private final Method method;
+    private final Object[] args;
+    private final boolean isRpc;
+    private final int callId;
+    final Counters counters;
+
+    private final RetryPolicy retryPolicy;
+    private final RetryInvocationHandler<?> retryInvocationHandler;
+
+    Call(Method method, Object[] args, boolean isRpc, int callId,
+         Counters counters, RetryInvocationHandler<?> retryInvocationHandler) {
+      this.method = method;
+      this.args = args;
+      this.isRpc = isRpc;
+      this.callId = callId;
+      this.counters = counters;
+
+      this.retryPolicy = retryInvocationHandler.getRetryPolicy(method);
+      this.retryInvocationHandler = retryInvocationHandler;
+    }
+
+    /** Invoke the call once without retrying. */
+    synchronized CallReturn invokeOnce() {
+      try {
+        // The number of times this invocation handler has ever been failed over
+        // before this method invocation attempt. Used to prevent concurrent
+        // failed method invocations from triggering multiple failover attempts.
+        final long failoverCount = retryInvocationHandler.getFailoverCount();
+        try {
+          return invoke();
+        } catch (Exception e) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace(this, e);
+          }
+          if (Thread.currentThread().isInterrupted()) {
+            // If interrupted, do not retry.
+            throw e;
+          }
+          retryInvocationHandler.handleException(
+              method, retryPolicy, failoverCount, counters, e);
+          return CallReturn.RETRY;
+        }
+      } catch(Throwable t) {
+        return new CallReturn(t);
+      }
+    }
+
+    CallReturn invoke() throws Throwable {
+      return new CallReturn(invokeMethod());
+    }
+
+    Object invokeMethod() throws Throwable {
+      if (isRpc) {
+        Client.setCallIdAndRetryCount(callId, counters.retries);
+      }
+      return retryInvocationHandler.invokeMethod(method, args);
+    }
+
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "#" + callId + ": "
+          + method.getDeclaringClass().getSimpleName() + "." + method.getName()
+          + "(" + (args == null || args.length == 0? "": Arrays.toString(args))
+          +  ")";
+    }
+  }
+
+  static class Counters {
     /** Counter for retries. */
     private int retries;
     /** Counter for method invocation has been failed over. */
     private int failovers;
+
+    boolean isZeros() {
+      return retries == 0 && failovers == 0;
+    }
   }
 
   private static class ProxyDescriptor<T> {
@@ -144,11 +216,13 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
 
   private final ProxyDescriptor<T> proxyDescriptor;
 
-  private volatile boolean hasMadeASuccessfulCall = false;
-  
+  private volatile boolean hasSuccessfulCall = false;
+
   private final RetryPolicy defaultPolicy;
   private final Map<String,RetryPolicy> methodNameToPolicyMap;
 
+  private final AsyncCallHandler asyncCallHandler = new AsyncCallHandler();
+
   protected RetryInvocationHandler(FailoverProxyProvider<T> proxyProvider,
       RetryPolicy retryPolicy) {
     this(proxyProvider, retryPolicy, Collections.<String, RetryPolicy>emptyMap());
@@ -167,38 +241,35 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
     return policy != null? policy: defaultPolicy;
   }
 
+  private long getFailoverCount() {
+    return proxyDescriptor.getFailoverCount();
+  }
+
+  private Call newCall(Method method, Object[] args, boolean isRpc, int callId,
+                       Counters counters) {
+    if (Client.isAsynchronousMode()) {
+      return asyncCallHandler.newAsyncCall(method, args, isRpc, callId,
+          counters, this);
+    } else {
+      return new Call(method, args, isRpc, callId, counters, this);
+    }
+  }
+
   @Override
   public Object invoke(Object proxy, Method method, Object[] args)
       throws Throwable {
     final boolean isRpc = isRpcInvocation(proxyDescriptor.getProxy());
     final int callId = isRpc? Client.nextCallId(): RpcConstants.INVALID_CALL_ID;
-    return invoke(method, args, isRpc, callId, new Counters());
-  }
-
-  private Object invoke(final Method method, final Object[] args,
-      final boolean isRpc, final int callId, final Counters counters)
-      throws Throwable {
-    final RetryPolicy policy = getRetryPolicy(method);
+    final Counters counters = new Counters();
 
+    final Call call = newCall(method, args, isRpc, callId, counters);
     while (true) {
-      // The number of times this invocation handler has ever been failed over,
-      // before this method invocation attempt. Used to prevent concurrent
-      // failed method invocations from triggering multiple failover attempts.
-      final long failoverCount = proxyDescriptor.getFailoverCount();
-
-      if (isRpc) {
-        Client.setCallIdAndRetryCount(callId, counters.retries);
-      }
-      try {
-        final Object ret = invokeMethod(method, args);
-        hasMadeASuccessfulCall = true;
-        return ret;
-      } catch (Exception ex) {
-        if (Thread.currentThread().isInterrupted()) {
-          // If interrupted, do not retry.
-          throw ex;
-        }
-        handleException(method, policy, failoverCount, counters, ex);
+      final CallReturn c = call.invokeOnce();
+      final CallReturn.State state = c.getState();
+      if (state == CallReturn.State.ASYNC_INVOKED) {
+        return null; // return null for async calls
+      } else if (c.getState() != CallReturn.State.RETRY) {
+        return c.getReturnValue();
       }
     }
   }
@@ -239,7 +310,8 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
       final int failovers, final long delay, final Exception ex) {
     // log info if this has made some successful calls or
     // this is not the first failover
-    final boolean info = hasMadeASuccessfulCall || failovers != 0;
+    final boolean info = hasSuccessfulCall || failovers != 0
+        || asyncCallHandler.hasSuccessfulCall();
     if (!info && !LOG.isDebugEnabled()) {
       return;
     }
@@ -265,7 +337,9 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
       if (!method.isAccessible()) {
         method.setAccessible(true);
       }
-      return method.invoke(proxyDescriptor.getProxy(), args);
+      final Object r = method.invoke(proxyDescriptor.getProxy(), args);
+      hasSuccessfulCall = true;
+      return r;
     } catch (InvocationTargetException e) {
       throw e.getCause();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
index 131aa8f..c0a14b7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.retry;
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.NoRouteToHostException;
@@ -647,8 +648,9 @@ public class RetryPolicies {
         return new RetryAction(RetryAction.RetryDecision.FAIL, 0, "retries ("
             + retries + ") exceeded maximum allowed (" + maxRetries + ")");
       }
-      
+
       if (e instanceof ConnectException ||
+          e instanceof EOFException ||
           e instanceof NoRouteToHostException ||
           e instanceof UnknownHostException ||
           e instanceof StandbyException ||

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index d1d5b17..ed8d905 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -58,7 +58,6 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.concurrent.AsyncGet;
-import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.Tracer;
 
@@ -94,8 +93,8 @@ public class Client implements AutoCloseable {
 
   private static final ThreadLocal<Integer> callId = new ThreadLocal<Integer>();
   private static final ThreadLocal<Integer> retryCount = new ThreadLocal<Integer>();
-  private static final ThreadLocal<Future<?>> ASYNC_RPC_RESPONSE
-      = new ThreadLocal<>();
+  private static final ThreadLocal<AsyncGet<? extends Writable, IOException>>
+      ASYNC_RPC_RESPONSE = new ThreadLocal<>();
   private static final ThreadLocal<Boolean> asynchronousMode =
       new ThreadLocal<Boolean>() {
         @Override
@@ -106,8 +105,9 @@ public class Client implements AutoCloseable {
 
   @SuppressWarnings("unchecked")
   @Unstable
-  public static <T> Future<T> getAsyncRpcResponse() {
-    return (Future<T>) ASYNC_RPC_RESPONSE.get();
+  public static <T extends Writable> AsyncGet<T, IOException>
+      getAsyncRpcResponse() {
+    return (AsyncGet<T, IOException>) ASYNC_RPC_RESPONSE.get();
   }
 
   /** Set call id and retry count for the next call. */
@@ -1413,9 +1413,16 @@ public class Client implements AutoCloseable {
             }
           }
         }
+
+        @Override
+        public boolean isDone() {
+          synchronized (call) {
+            return call.done;
+          }
+        }
       };
 
-      ASYNC_RPC_RESPONSE.set(new AsyncGetFuture<>(asyncGet));
+      ASYNC_RPC_RESPONSE.set(asyncGet);
       return null;
     } else {
       return getRpcResponse(call, connection, -1, null);
@@ -1460,10 +1467,8 @@ public class Client implements AutoCloseable {
     synchronized (call) {
       while (!call.done) {
         try {
-          final long waitTimeout = AsyncGet.Util.asyncGetTimeout2WaitTimeout(
-              timeout, unit);
-          call.wait(waitTimeout); // wait for the result
-          if (waitTimeout > 0 && !call.done) {
+          AsyncGet.Util.wait(call, timeout, unit);
+          if (timeout >= 0 && !call.done) {
             return null;
           }
         } catch (InterruptedException ie) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 0f43fc6..315ec67 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -54,7 +54,6 @@ import java.lang.reflect.Proxy;
 import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -256,14 +255,18 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
       
       if (Client.isAsynchronousMode()) {
-        final Future<RpcResponseWrapper> frrw = Client.getAsyncRpcResponse();
+        final AsyncGet<RpcResponseWrapper, IOException> arr
+            = Client.getAsyncRpcResponse();
         final AsyncGet<Message, Exception> asyncGet
             = new AsyncGet<Message, Exception>() {
           @Override
           public Message get(long timeout, TimeUnit unit) throws Exception {
-            final RpcResponseWrapper rrw = timeout < 0?
-                frrw.get(): frrw.get(timeout, unit);
-            return getReturnMessage(method, rrw);
+            return getReturnMessage(method, arr.get(timeout, unit));
+          }
+
+          @Override
+          public boolean isDone() {
+            return arr.isDone();
           }
         };
         ASYNC_RETURN_MESSAGE.set(asyncGet);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
index 5eac869..f124890 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
@@ -47,14 +47,19 @@ public interface AsyncGet<R, E extends Throwable> {
   R get(long timeout, TimeUnit unit)
       throws E, TimeoutException, InterruptedException;
 
+  /** @return true if the underlying computation is done; false, otherwise. */
+  boolean isDone();
+
   /** Utility */
   class Util {
-    /**
-     * @return {@link Object#wait(long)} timeout converted
-     *         from {@link #get(long, TimeUnit)} timeout.
-     */
-    public static long asyncGetTimeout2WaitTimeout(long timeout, TimeUnit unit){
-      return timeout < 0? 0: timeout == 0? 1:unit.toMillis(timeout);
+    /** Use {@link #get(long, TimeUnit)} timeout parameters to wait. */
+    public static void wait(Object obj, long timeout, TimeUnit unit)
+        throws InterruptedException {
+      if (timeout < 0) {
+        obj.wait();
+      } else if (timeout > 0) {
+        obj.wait(unit.toMillis(timeout));
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
index 0ad191b..4450c0c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.ipc.TestIPC.TestServer;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,6 +51,11 @@ public class TestAsyncIPC {
   private static Configuration conf;
   private static final Log LOG = LogFactory.getLog(TestAsyncIPC.class);
 
+  static <T extends Writable> AsyncGetFuture<T, IOException>
+      getAsyncRpcResponseFuture() {
+    return new AsyncGetFuture<>(Client.getAsyncRpcResponse());
+  }
+
   @Before
   public void setupConf() {
     conf = new Configuration();
@@ -84,7 +90,7 @@ public class TestAsyncIPC {
         try {
           final long param = TestIPC.RANDOM.nextLong();
           TestIPC.call(client, param, server, conf);
-          returnFutures.put(i, Client.getAsyncRpcResponse());
+          returnFutures.put(i, getAsyncRpcResponseFuture());
           expectedValues.put(i, param);
         } catch (Exception e) {
           failed = true;
@@ -204,7 +210,7 @@ public class TestAsyncIPC {
 
     private void doCall(final int idx, final long param) throws IOException {
       TestIPC.call(client, param, server, conf);
-      returnFutures.put(idx, Client.getAsyncRpcResponse());
+      returnFutures.put(idx, getAsyncRpcResponseFuture());
       expectedValues.put(idx, param);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/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..824336a 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
@@ -29,7 +29,7 @@ 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.io.retry.AsyncCallHandler;
 import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.hadoop.ipc.Client;
 
@@ -51,9 +51,8 @@ public class AsyncDistributedFileSystem {
     this.dfs = dfs;
   }
 
-  static <T> Future<T> getReturnValue() {
-    return new AsyncGetFuture<>(
-        ClientNamenodeProtocolTranslatorPB.getAsyncReturnValue());
+  private static <T> Future<T> getReturnValue() {
+    return new AsyncGetFuture<>(AsyncCallHandler.getAsyncReturn());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/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..bcf5269 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
@@ -29,7 +29,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
@@ -184,6 +183,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.AsyncCallHandler;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -212,8 +212,6 @@ import org.apache.hadoop.util.concurrent.AsyncGet;
 public class ClientNamenodeProtocolTranslatorPB implements
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
-  private static final ThreadLocal<AsyncGet<?, Exception>>
-      ASYNC_RETURN_VALUE = new ThreadLocal<>();
 
   static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
       GetServerDefaultsRequestProto.newBuilder().build();
@@ -247,12 +245,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
     rpcProxy = proxy;
   }
 
-  @SuppressWarnings("unchecked")
-  @Unstable
-  public static <T> AsyncGet<T, Exception> getAsyncReturnValue() {
-    return (AsyncGet<T, Exception>) ASYNC_RETURN_VALUE.get();
-  }
-
   @Override
   public void close() {
     RPC.stopProxy(rpcProxy);
@@ -391,8 +383,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
         asyncReturnMessage.get(timeout, unit);
         return null;
       }
+
+      @Override
+      public boolean isDone() {
+        return asyncReturnMessage.isDone();
+      }
     };
-    ASYNC_RETURN_VALUE.set(asyncGet);
+    AsyncCallHandler.setLowerLayerAsyncReturn(asyncGet);
   }
 
   @Override
@@ -1367,17 +1364,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
         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);
+        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));
+          }
+
+          @Override
+          public boolean isDone() {
+            return asyncReturnMessage.isDone();
+          }
+        };
+        AsyncCallHandler.setLowerLayerAsyncReturn(asyncGet);
         return null;
       } else {
         return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/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
index c7615a9..6a60290 100644
--- 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
 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.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
@@ -70,7 +71,7 @@ public class TestAsyncDFS {
   public static final Log LOG = LogFactory.getLog(TestAsyncDFS.class);
   private final short replFactor = 1;
   private final long blockSize = 512;
-  private long fileLen = blockSize * 3;
+  private long fileLen = 0;
   private final long seed = Time.now();
   private final Random r = new Random(seed);
   private final PermissionGenerator permGenerator = new PermissionGenerator(r);
@@ -80,7 +81,7 @@ public class TestAsyncDFS {
 
   private Configuration conf;
   private MiniDFSCluster cluster;
-  private FileSystem fs;
+  private DistributedFileSystem fs;
   private AsyncDistributedFileSystem adfs;
 
   @Before
@@ -95,10 +96,10 @@ public class TestAsyncDFS {
         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 = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
     cluster.waitActive();
-    fs = FileSystem.get(conf);
-    adfs = cluster.getFileSystem().getAsyncDistributedFileSystem();
+    fs = cluster.getFileSystem();
+    adfs = fs.getAsyncDistributedFileSystem();
   }
 
   @After
@@ -113,31 +114,6 @@ public class TestAsyncDFS {
     }
   }
 
-  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";
@@ -348,7 +324,7 @@ public class TestAsyncDFS {
 
   public static void checkPermissionDenied(final Exception e, final Path dir,
       final String user) {
-    assertTrue(e.getCause() instanceof ExecutionException);
+    assertTrue(e.getCause() instanceof RemoteException);
     assertTrue("Permission denied messages must carry AccessControlException",
         e.getMessage().contains("AccessControlException"));
     assertTrue("Permission denied messages must carry the username", e
@@ -470,4 +446,9 @@ public class TestAsyncDFS {
       assertTrue("group2".equals(fs.getFileStatus(dsts[i]).getGroup()));
     }
   }
+
+  @Test
+  public void testAsyncWithoutRetry() throws Exception {
+    TestAsyncHDFSWithHA.runTestAsyncWithoutRetry(conf, cluster, fs);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
new file mode 100644
index 0000000..9ade8ec
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
@@ -0,0 +1,181 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.io.retry.AsyncCallHandler;
+import org.apache.hadoop.io.retry.RetryInvocationHandler;
+import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.concurrent.AsyncGetFuture;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/** Test async methods with HA setup. */
+public class TestAsyncHDFSWithHA {
+  static final Logger LOG = LoggerFactory.getLogger(TestAsyncHDFSWithHA.class);
+  static {
+    GenericTestUtils.setLogLevel(RetryInvocationHandler.LOG, Level.ALL);
+  }
+
+  private static <T> Future<T> getReturnValue() {
+    return new AsyncGetFuture<>(AsyncCallHandler.getAsyncReturn());
+  }
+
+  static void mkdirs(DistributedFileSystem dfs, String dir, Path[] srcs,
+                     Path[] dsts) throws IOException {
+    for (int i = 0; i < srcs.length; i++) {
+      srcs[i] = new Path(dir, "src" + i);
+      dsts[i] = new Path(dir, "dst" + i);
+      dfs.mkdirs(srcs[i]);
+    }
+  }
+
+  static void runTestAsyncWithoutRetry(Configuration conf,
+      MiniDFSCluster cluster, DistributedFileSystem dfs) throws Exception {
+    final int num = 5;
+
+    final String renameDir = "/testAsyncWithoutRetry/";
+    final Path[] srcs = new Path[num + 1];
+    final Path[] dsts = new Path[num + 1];
+    mkdirs(dfs, renameDir, srcs, dsts);
+
+    // create a proxy without retry.
+    final NameNodeProxiesClient.ProxyAndInfo<ClientProtocol> proxyInfo
+        = NameNodeProxies.createNonHAProxy(conf,
+        cluster.getNameNode(0).getNameNodeAddress(),
+        ClientProtocol.class, UserGroupInformation.getCurrentUser(),
+        false);
+    final ClientProtocol cp = proxyInfo.getProxy();
+
+    // submit async calls
+    Client.setAsynchronousMode(true);
+    final List<Future<Void>> results = new ArrayList<>();
+    for (int i = 0; i < num; i++) {
+      final String src = srcs[i].toString();
+      final String dst = dsts[i].toString();
+      LOG.info(i + ") rename " + src + " -> " + dst);
+      cp.rename2(src, dst);
+      results.add(getReturnValue());
+    }
+    Client.setAsynchronousMode(false);
+
+    // wait for the async calls
+    for (Future<Void> f : results) {
+      f.get();
+    }
+
+    //check results
+    for (int i = 0; i < num; i++) {
+      Assert.assertEquals(false, dfs.exists(srcs[i]));
+      Assert.assertEquals(true, dfs.exists(dsts[i]));
+    }
+  }
+
+  /** Testing HDFS async methods with HA setup. */
+  @Test(timeout = 120000)
+  public void testAsyncWithHAFailover() throws Exception {
+    final int num = 10;
+
+    final Configuration conf = new HdfsConfiguration();
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(0).build();
+
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+
+      final DistributedFileSystem dfs = HATestUtil.configureFailoverFs(
+          cluster, conf);
+      runTestAsyncWithoutRetry(conf, cluster, dfs);
+
+      final String renameDir = "/testAsyncWithHAFailover/";
+      final Path[] srcs = new Path[num + 1];
+      final Path[] dsts = new Path[num + 1];
+      mkdirs(dfs, renameDir, srcs, dsts);
+
+      // submit async calls and trigger failover in the middle.
+      final AsyncDistributedFileSystem adfs
+          = dfs.getAsyncDistributedFileSystem();
+      final ExecutorService executor = Executors.newFixedThreadPool(num + 1);
+
+      final List<Future<Void>> results = new ArrayList<>();
+      final List<IOException> exceptions = new ArrayList<>();
+      final List<Future<?>> futures = new ArrayList<>();
+      final int half = num/2;
+      for(int i = 0; i <= num; i++) {
+        final int id = i;
+        futures.add(executor.submit(new Runnable() {
+          @Override
+          public void run() {
+            try {
+              if (id == half) {
+                // failover
+                cluster.shutdownNameNode(0);
+                cluster.transitionToActive(1);
+              } else {
+                // rename
+                results.add(adfs.rename(srcs[id], dsts[id]));
+              }
+            } catch (IOException e) {
+              exceptions.add(e);
+            }
+          }
+        }));
+      }
+
+      // wait for the tasks
+      Assert.assertEquals(num + 1, futures.size());
+      for(int i = 0; i <= num; i++) {
+        futures.get(i).get();
+      }
+      // wait for the async calls
+      Assert.assertEquals(num, results.size());
+      Assert.assertTrue(exceptions.isEmpty());
+      for(Future<Void> r : results) {
+        r.get();
+      }
+
+      // check results
+      for(int i = 0; i <= num; i++) {
+        final boolean renamed = i != half;
+        Assert.assertEquals(!renamed, dfs.exists(srcs[i]));
+        Assert.assertEquals(renamed, dfs.exists(dsts[i]));
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35f255b0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
index 42cf3d4..169bbee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -135,7 +136,8 @@ public abstract class HATestUtil {
   }
   
   /** Gets the filesystem instance by setting the failover configurations */
-  public static FileSystem configureFailoverFs(MiniDFSCluster cluster, Configuration conf)
+  public static DistributedFileSystem configureFailoverFs(
+      MiniDFSCluster cluster, Configuration conf)
       throws IOException, URISyntaxException {
     return configureFailoverFs(cluster, conf, 0);
   }
@@ -147,13 +149,14 @@ public abstract class HATestUtil {
    * @param nsIndex namespace index starting with zero
    * @throws IOException if an error occurs rolling the edit log
    */
-  public static FileSystem configureFailoverFs(MiniDFSCluster cluster, Configuration conf,
+  public static DistributedFileSystem configureFailoverFs(
+      MiniDFSCluster cluster, Configuration conf,
       int nsIndex) throws IOException, URISyntaxException {
     conf = new Configuration(conf);
     String logicalName = getLogicalHostname(cluster);
     setFailoverConfigurations(cluster, conf, logicalName, nsIndex);
     FileSystem fs = FileSystem.get(new URI("hdfs://" + logicalName), conf);
-    return fs;
+    return (DistributedFileSystem)fs;
   }
   
   public static void setFailoverConfigurations(MiniDFSCluster cluster,


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


[34/47] hadoop git commit: Revert "Revert "HADOOP-13168. Support Future.get with timeout in ipc async calls.""

Posted by ae...@apache.org.
Revert "Revert "HADOOP-13168. Support Future.get with timeout in ipc async calls.""

This reverts commit e4450d47f19131818e1c040b6bd8d85ae8250475.


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

Branch: refs/heads/HDFS-1312
Commit: 574dcd34c0da1903d25e37dc5757642a584dc3d0
Parents: cba9a01
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 6 16:31:23 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jun 6 16:31:23 2016 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/Client.java | 119 ++++++++----------
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |  62 +++++-----
 .../apache/hadoop/util/concurrent/AsyncGet.java |  60 +++++++++
 .../hadoop/util/concurrent/AsyncGetFuture.java  |  73 +++++++++++
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     | 124 +++++++++++--------
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |  24 +---
 .../ClientNamenodeProtocolTranslatorPB.java     |  33 ++---
 7 files changed, 310 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/574dcd34/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 9be4649..d1d5b17 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -18,46 +18,10 @@
 
 package org.apache.hadoop.ipc;
 
-import static org.apache.hadoop.ipc.RpcConstants.*;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.FilterInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketTimeoutException;
-import java.net.UnknownHostException;
-import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
-import java.util.Hashtable;
-import java.util.Iterator;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import javax.net.SocketFactory;
-import javax.security.sasl.Sasl;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.protobuf.CodedOutputStream;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -93,14 +57,25 @@ import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.concurrent.AsyncGet;
+import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.Tracer;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.AbstractFuture;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.google.protobuf.CodedOutputStream;
+import javax.net.SocketFactory;
+import javax.security.sasl.Sasl;
+import java.io.*;
+import java.net.*;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ipc.RpcConstants.CONNECTION_CONTEXT_CALL_ID;
+import static org.apache.hadoop.ipc.RpcConstants.PING_CALL_ID;
 
 /** A client for an IPC service.  IPC calls take a single {@link Writable} as a
  * parameter, and return a {@link Writable} as their value.  A service runs on
@@ -119,8 +94,8 @@ public class Client implements AutoCloseable {
 
   private static final ThreadLocal<Integer> callId = new ThreadLocal<Integer>();
   private static final ThreadLocal<Integer> retryCount = new ThreadLocal<Integer>();
-  private static final ThreadLocal<Future<?>>
-      RETURN_RPC_RESPONSE = new ThreadLocal<>();
+  private static final ThreadLocal<Future<?>> ASYNC_RPC_RESPONSE
+      = new ThreadLocal<>();
   private static final ThreadLocal<Boolean> asynchronousMode =
       new ThreadLocal<Boolean>() {
         @Override
@@ -131,8 +106,8 @@ public class Client implements AutoCloseable {
 
   @SuppressWarnings("unchecked")
   @Unstable
-  public static <T> Future<T> getReturnRpcResponse() {
-    return (Future<T>) RETURN_RPC_RESPONSE.get();
+  public static <T> Future<T> getAsyncRpcResponse() {
+    return (Future<T>) ASYNC_RPC_RESPONSE.get();
   }
 
   /** Set call id and retry count for the next call. */
@@ -379,6 +354,11 @@ public class Client implements AutoCloseable {
       }
     }
 
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + id;
+    }
+
     /** Indicate when the call is complete and the
      * value or error are available.  Notifies by default.  */
     protected synchronized void callComplete() {
@@ -1413,27 +1393,32 @@ public class Client implements AutoCloseable {
     }
 
     if (isAsynchronousMode()) {
-      Future<Writable> returnFuture = new AbstractFuture<Writable>() {
-        private final AtomicBoolean callled = new AtomicBoolean(false);
+      final AsyncGet<Writable, IOException> asyncGet
+          = new AsyncGet<Writable, IOException>() {
         @Override
-        public Writable get() throws InterruptedException, ExecutionException {
-          if (callled.compareAndSet(false, true)) {
-            try {
-              set(getRpcResponse(call, connection));
-            } catch (IOException ie) {
-              setException(ie);
-            } finally {
+        public Writable get(long timeout, TimeUnit unit)
+            throws IOException, TimeoutException{
+          boolean done = true;
+          try {
+            final Writable w = getRpcResponse(call, connection, timeout, unit);
+            if (w == null) {
+              done = false;
+              throw new TimeoutException(call + " timed out "
+                  + timeout + " " + unit);
+            }
+            return w;
+          } finally {
+            if (done) {
               releaseAsyncCall();
             }
           }
-          return super.get();
         }
       };
 
-      RETURN_RPC_RESPONSE.set(returnFuture);
+      ASYNC_RPC_RESPONSE.set(new AsyncGetFuture<>(asyncGet));
       return null;
     } else {
-      return getRpcResponse(call, connection);
+      return getRpcResponse(call, connection, -1, null);
     }
   }
 
@@ -1469,12 +1454,18 @@ public class Client implements AutoCloseable {
     return asyncCallCounter.get();
   }
 
-  private Writable getRpcResponse(final Call call, final Connection connection)
-      throws IOException {
+  /** @return the rpc response or, in case of timeout, null. */
+  private Writable getRpcResponse(final Call call, final Connection connection,
+      final long timeout, final TimeUnit unit) throws IOException {
     synchronized (call) {
       while (!call.done) {
         try {
-          call.wait();                           // wait for the result
+          final long waitTimeout = AsyncGet.Util.asyncGetTimeout2WaitTimeout(
+              timeout, unit);
+          call.wait(waitTimeout); // wait for the result
+          if (waitTimeout > 0 && !call.done) {
+            return null;
+          }
         } catch (InterruptedException ie) {
           Thread.currentThread().interrupt();
           throw new InterruptedIOException("Call interrupted");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/574dcd34/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 8fcdb78..0f43fc6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -18,21 +18,9 @@
 
 package org.apache.hadoop.ipc;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.net.InetSocketAddress;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import javax.net.SocketFactory;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.*;
+import com.google.protobuf.Descriptors.MethodDescriptor;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -52,17 +40,23 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.concurrent.AsyncGet;
 import org.apache.htrace.core.TraceScope;
 import org.apache.htrace.core.Tracer;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.BlockingService;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.GeneratedMessage;
-import com.google.protobuf.Message;
-import com.google.protobuf.ServiceException;
-import com.google.protobuf.TextFormat;
+import javax.net.SocketFactory;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * RPC Engine for for protobuf based RPCs.
@@ -70,8 +64,8 @@ import com.google.protobuf.TextFormat;
 @InterfaceStability.Evolving
 public class ProtobufRpcEngine implements RpcEngine {
   public static final Log LOG = LogFactory.getLog(ProtobufRpcEngine.class);
-  private static final ThreadLocal<Callable<?>>
-      RETURN_MESSAGE_CALLBACK = new ThreadLocal<>();
+  private static final ThreadLocal<AsyncGet<Message, Exception>>
+      ASYNC_RETURN_MESSAGE = new ThreadLocal<>();
 
   static { // Register the rpcRequest deserializer for WritableRpcEngine 
     org.apache.hadoop.ipc.Server.registerProtocolEngine(
@@ -81,10 +75,9 @@ public class ProtobufRpcEngine implements RpcEngine {
 
   private static final ClientCache CLIENTS = new ClientCache();
 
-  @SuppressWarnings("unchecked")
   @Unstable
-  public static <T> Callable<T> getReturnMessageCallback() {
-    return (Callable<T>) RETURN_MESSAGE_CALLBACK.get();
+  public static AsyncGet<Message, Exception> getAsyncReturnMessage() {
+    return ASYNC_RETURN_MESSAGE.get();
   }
 
   public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
@@ -263,14 +256,17 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
       
       if (Client.isAsynchronousMode()) {
-        final Future<RpcResponseWrapper> frrw = Client.getReturnRpcResponse();
-        Callable<Message> callback = new Callable<Message>() {
+        final Future<RpcResponseWrapper> frrw = Client.getAsyncRpcResponse();
+        final AsyncGet<Message, Exception> asyncGet
+            = new AsyncGet<Message, Exception>() {
           @Override
-          public Message call() throws Exception {
-            return getReturnMessage(method, frrw.get());
+          public Message get(long timeout, TimeUnit unit) throws Exception {
+            final RpcResponseWrapper rrw = timeout < 0?
+                frrw.get(): frrw.get(timeout, unit);
+            return getReturnMessage(method, rrw);
           }
         };
-        RETURN_MESSAGE_CALLBACK.set(callback);
+        ASYNC_RETURN_MESSAGE.set(asyncGet);
         return null;
       } else {
         return getReturnMessage(method, val);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/574dcd34/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
new file mode 100644
index 0000000..5eac869
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
@@ -0,0 +1,60 @@
+/**
+ * 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.util.concurrent;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * This interface defines an asynchronous {@link #get(long, TimeUnit)} method.
+ *
+ * When the return value is still being computed, invoking
+ * {@link #get(long, TimeUnit)} will result in a {@link TimeoutException}.
+ * The method should be invoked again and again
+ * until the underlying computation is completed.
+ *
+ * @param <R> The type of the return value.
+ * @param <E> The exception type that the underlying implementation may throw.
+ */
+public interface AsyncGet<R, E extends Throwable> {
+  /**
+   * Get the result.
+   *
+   * @param timeout The maximum time period to wait.
+   *                When timeout == 0, it does not wait at all.
+   *                When timeout < 0, it waits indefinitely.
+   * @param unit The unit of the timeout value
+   * @return the result, which is possibly null.
+   * @throws E an exception thrown by the underlying implementation.
+   * @throws TimeoutException if it cannot return after the given time period.
+   * @throws InterruptedException if the thread is interrupted.
+   */
+  R get(long timeout, TimeUnit unit)
+      throws E, TimeoutException, InterruptedException;
+
+  /** Utility */
+  class Util {
+    /**
+     * @return {@link Object#wait(long)} timeout converted
+     *         from {@link #get(long, TimeUnit)} timeout.
+     */
+    public static long asyncGetTimeout2WaitTimeout(long timeout, TimeUnit unit){
+      return timeout < 0? 0: timeout == 0? 1:unit.toMillis(timeout);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/574dcd34/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGetFuture.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGetFuture.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGetFuture.java
new file mode 100644
index 0000000..d687867
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGetFuture.java
@@ -0,0 +1,73 @@
+/**
+ * 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.util.concurrent;
+
+import com.google.common.util.concurrent.AbstractFuture;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/** A {@link Future} implemented using an {@link AsyncGet} object. */
+public class AsyncGetFuture<T, E extends Throwable> extends AbstractFuture<T> {
+  public static final Log LOG = LogFactory.getLog(AsyncGetFuture.class);
+
+  private final AtomicBoolean called = new AtomicBoolean(false);
+  private final AsyncGet<T, E> asyncGet;
+
+  public AsyncGetFuture(AsyncGet<T, E> asyncGet) {
+    this.asyncGet = asyncGet;
+  }
+
+  private void callAsyncGet(long timeout, TimeUnit unit) {
+    if (!isCancelled() && called.compareAndSet(false, true)) {
+      try {
+        set(asyncGet.get(timeout, unit));
+      } catch (TimeoutException te) {
+        LOG.trace("TRACE", te);
+        called.compareAndSet(true, false);
+      } catch (Throwable e) {
+        LOG.trace("TRACE", e);
+        setException(e);
+      }
+    }
+  }
+
+  @Override
+  public T get() throws InterruptedException, ExecutionException {
+    callAsyncGet(-1, TimeUnit.MILLISECONDS);
+    return super.get();
+  }
+
+  @Override
+  public T get(long timeout, TimeUnit unit)
+      throws InterruptedException, TimeoutException, ExecutionException {
+    callAsyncGet(timeout, unit);
+    return super.get(0, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public boolean isDone() {
+    callAsyncGet(0, TimeUnit.MILLISECONDS);
+    return super.isDone();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/574dcd34/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
index 8ee3a2c..0ad191b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
@@ -18,20 +18,6 @@
 
 package org.apache.hadoop.ipc;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-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;
@@ -48,6 +34,17 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
 public class TestAsyncIPC {
 
   private static Configuration conf;
@@ -87,26 +84,50 @@ public class TestAsyncIPC {
         try {
           final long param = TestIPC.RANDOM.nextLong();
           TestIPC.call(client, param, server, conf);
-          Future<LongWritable> returnFuture = Client.getReturnRpcResponse();
-          returnFutures.put(i, returnFuture);
+          returnFutures.put(i, Client.getAsyncRpcResponse());
           expectedValues.put(i, param);
         } catch (Exception e) {
-          LOG.fatal("Caught: " + StringUtils.stringifyException(e));
           failed = true;
+          throw new RuntimeException(e);
         }
       }
     }
 
-    public void waitForReturnValues() throws InterruptedException,
-        ExecutionException {
+    void assertReturnValues() throws InterruptedException, ExecutionException {
       for (int i = 0; i < count; i++) {
         LongWritable value = returnFutures.get(i).get();
-        if (expectedValues.get(i) != value.get()) {
-          LOG.fatal(String.format("Call-%d failed!", i));
-          failed = true;
-          break;
+        Assert.assertEquals("call" + i + " failed.",
+            expectedValues.get(i).longValue(), value.get());
+      }
+      Assert.assertFalse(failed);
+    }
+
+    void assertReturnValues(long timeout, TimeUnit unit)
+        throws InterruptedException, ExecutionException {
+      final boolean[] checked = new boolean[count];
+      for(boolean done = false; !done;) {
+        done = true;
+        for (int i = 0; i < count; i++) {
+          if (checked[i]) {
+            continue;
+          } else {
+            done = false;
+          }
+
+          final LongWritable value;
+          try {
+            value = returnFutures.get(i).get(timeout, unit);
+          } catch (TimeoutException e) {
+            LOG.info("call" + i + " caught ", e);
+            continue;
+          }
+
+          Assert.assertEquals("call" + i + " failed.",
+              expectedValues.get(i).longValue(), value.get());
+          checked[i] = true;
         }
       }
+      Assert.assertFalse(failed);
     }
   }
 
@@ -183,8 +204,7 @@ public class TestAsyncIPC {
 
     private void doCall(final int idx, final long param) throws IOException {
       TestIPC.call(client, param, server, conf);
-      Future<LongWritable> returnFuture = Client.getReturnRpcResponse();
-      returnFutures.put(idx, returnFuture);
+      returnFutures.put(idx, Client.getAsyncRpcResponse());
       expectedValues.put(idx, param);
     }
 
@@ -233,10 +253,7 @@ public class TestAsyncIPC {
     }
     for (int i = 0; i < callerCount; i++) {
       callers[i].join();
-      callers[i].waitForReturnValues();
-      String msg = String.format("Expected not failed for caller-%d: %s.", i,
-          callers[i]);
-      assertFalse(msg, callers[i].failed);
+      callers[i].assertReturnValues();
     }
     for (int i = 0; i < clientCount; i++) {
       clients[i].stop();
@@ -258,25 +275,37 @@ public class TestAsyncIPC {
     try {
       AsyncCaller caller = new AsyncCaller(client, addr, callCount);
       caller.run();
+      caller.assertReturnValues();
+      caller.assertReturnValues();
+      caller.assertReturnValues();
+      Assert.assertEquals(asyncCallCount, client.getAsyncCallCount());
+    } finally {
+      client.stop();
+      server.stop();
+    }
+  }
 
-      caller.waitForReturnValues();
-      String msg = String.format(
-          "First time, expected not failed for caller: %s.", caller);
-      assertFalse(msg, caller.failed);
+  @Test(timeout = 60000)
+  public void testFutureGetWithTimeout() throws IOException,
+      InterruptedException, ExecutionException {
+//    GenericTestUtils.setLogLevel(AsyncGetFuture.LOG, Level.ALL);
+    final Server server = new TestIPC.TestServer(10, true, conf);
+    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
+    server.start();
 
-      caller.waitForReturnValues();
-      assertTrue(asyncCallCount == client.getAsyncCallCount());
-      msg = String.format("Second time, expected not failed for caller: %s.",
-          caller);
-      assertFalse(msg, caller.failed);
+    final Client client = new Client(LongWritable.class, conf);
 
-      assertTrue(asyncCallCount == client.getAsyncCallCount());
+    try {
+      final AsyncCaller caller = new AsyncCaller(client, addr, 10);
+      caller.run();
+      caller.assertReturnValues(10, TimeUnit.MILLISECONDS);
     } finally {
       client.stop();
       server.stop();
     }
   }
 
+
   public void internalTestAsyncCallLimit(int handlerCount, boolean handlerSleep,
       int clientCount, int callerCount, int callCount) throws IOException,
       InterruptedException, ExecutionException {
@@ -367,9 +396,7 @@ public class TestAsyncIPC {
       server.start();
       final AsyncCaller caller = new AsyncCaller(client, addr, 4);
       caller.run();
-      caller.waitForReturnValues();
-      String msg = String.format("Expected not failed for caller: %s.", caller);
-      assertFalse(msg, caller.failed);
+      caller.assertReturnValues();
     } finally {
       client.stop();
       server.stop();
@@ -406,9 +433,7 @@ public class TestAsyncIPC {
       server.start();
       final AsyncCaller caller = new AsyncCaller(client, addr, 10);
       caller.run();
-      caller.waitForReturnValues();
-      String msg = String.format("Expected not failed for caller: %s.", caller);
-      assertFalse(msg, caller.failed);
+      caller.assertReturnValues();
     } finally {
       client.stop();
       server.stop();
@@ -443,9 +468,7 @@ public class TestAsyncIPC {
       server.start();
       final AsyncCaller caller = new AsyncCaller(client, addr, 10);
       caller.run();
-      caller.waitForReturnValues();
-      String msg = String.format("Expected not failed for caller: %s.", caller);
-      assertFalse(msg, caller.failed);
+      caller.assertReturnValues();
     } finally {
       client.stop();
       server.stop();
@@ -489,10 +512,7 @@ public class TestAsyncIPC {
       }
       for (int i = 0; i < callerCount; ++i) {
         callers[i].join();
-        callers[i].waitForReturnValues();
-        String msg = String.format("Expected not failed for caller-%d: %s.", i,
-            callers[i]);
-        assertFalse(msg, callers[i].failed);
+        callers[i].assertReturnValues();
       }
     } finally {
       client.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/574dcd34/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 4fe0861..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,20 +19,16 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 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.FsPermission;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.hadoop.ipc.Client;
 
-import com.google.common.util.concurrent.AbstractFuture;
-
 /****************************************************************
  * Implementation of the asynchronous distributed file system.
  * This instance of this class is the way end-user code interacts
@@ -52,22 +48,8 @@ public class AsyncDistributedFileSystem {
   }
 
   static <T> Future<T> getReturnValue() {
-    final Callable<T> returnValueCallback = ClientNamenodeProtocolTranslatorPB
-        .getReturnValueCallback();
-    Future<T> returnFuture = new AbstractFuture<T>() {
-      private final AtomicBoolean called = new AtomicBoolean(false);
-      public T get() throws InterruptedException, ExecutionException {
-        if (called.compareAndSet(false, true)) {
-          try {
-            set(returnValueCallback.call());
-          } catch (Exception e) {
-            setException(e);
-          }
-        }
-        return super.get();
-      }
-    };
-    return returnFuture;
+    return new AsyncGetFuture<>(
+        ClientNamenodeProtocolTranslatorPB.getAsyncReturnValue());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/574dcd34/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 faa925c..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
@@ -24,7 +24,8 @@ import java.util.EnumSet;
 import java.util.List;
 
 import com.google.common.collect.Lists;
-import java.util.concurrent.Callable;
+
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -198,6 +199,7 @@ import org.apache.hadoop.security.token.Token;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Message;
 import com.google.protobuf.ServiceException;
+import org.apache.hadoop.util.concurrent.AsyncGet;
 
 /**
  * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
@@ -209,8 +211,8 @@ import com.google.protobuf.ServiceException;
 public class ClientNamenodeProtocolTranslatorPB implements
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
-  private static final ThreadLocal<Callable<?>>
-      RETURN_VALUE_CALLBACK = new ThreadLocal<>();
+  private static final ThreadLocal<AsyncGet<?, Exception>>
+      ASYNC_RETURN_VALUE = new ThreadLocal<>();
 
   static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
       GetServerDefaultsRequestProto.newBuilder().build();
@@ -246,8 +248,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @SuppressWarnings("unchecked")
   @Unstable
-  public static <T> Callable<T> getReturnValueCallback() {
-    return (Callable<T>) RETURN_VALUE_CALLBACK.get();
+  public static <T> AsyncGet<T, Exception> getAsyncReturnValue() {
+    return (AsyncGet<T, Exception>) ASYNC_RETURN_VALUE.get();
   }
 
   @Override
@@ -369,7 +371,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       if (Client.isAsynchronousMode()) {
         rpcProxy.setPermission(null, req);
-        setReturnValueCallback();
+        setAsyncReturnValue();
       } else {
         rpcProxy.setPermission(null, req);
       }
@@ -378,17 +380,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
-  private void setReturnValueCallback() {
-    final Callable<Message> returnMessageCallback = ProtobufRpcEngine
-        .getReturnMessageCallback();
-    Callable<Void> callBack = new Callable<Void>() {
+  private void setAsyncReturnValue() {
+    final AsyncGet<Message, Exception> asyncReturnMessage
+        = ProtobufRpcEngine.getAsyncReturnMessage();
+    final AsyncGet<Void, Exception> asyncGet
+        = new AsyncGet<Void, Exception>() {
       @Override
-      public Void call() throws Exception {
-        returnMessageCallback.call();
+      public Void get(long timeout, TimeUnit unit) throws Exception {
+        asyncReturnMessage.get(timeout, unit);
         return null;
       }
     };
-    RETURN_VALUE_CALLBACK.set(callBack);
+    ASYNC_RETURN_VALUE.set(asyncGet);
   }
 
   @Override
@@ -403,7 +406,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       if (Client.isAsynchronousMode()) {
         rpcProxy.setOwner(null, req.build());
-        setReturnValueCallback();
+        setAsyncReturnValue();
       } else {
         rpcProxy.setOwner(null, req.build());
       }
@@ -536,7 +539,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       if (Client.isAsynchronousMode()) {
         rpcProxy.rename2(null, req);
-        setReturnValueCallback();
+        setAsyncReturnValue();
       } else {
         rpcProxy.rename2(null, req);
       }


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


[25/47] hadoop git commit: Revert "HDFS-10431 Refactor and speedup TestAsyncDFSRename. Contributed by Xiaobing Zhou"

Posted by ae...@apache.org.
Revert "HDFS-10431 Refactor and speedup TestAsyncDFSRename.  Contributed by Xiaobing Zhou"

This reverts commit f4b9bcd87c66a39f0c93983431630e9d1b6e36d3.


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

Branch: refs/heads/HDFS-1312
Commit: 5ee5912ebd541d5b4c33ecd46dfdebe1e23b56c3
Parents: 8cf47d8
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 18:09:13 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 18:09:13 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    | 233 +-------
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  | 563 +++++++++++++++----
 2 files changed, 483 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ee5912e/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
index ddcf492..67262dd 100644
--- 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
@@ -29,16 +29,13 @@ 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.assertFalse;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
@@ -46,21 +43,15 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
 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.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -72,28 +63,21 @@ import com.google.common.collect.Lists;
  * */
 public class TestAsyncDFS {
   public static final Log LOG = LogFactory.getLog(TestAsyncDFS.class);
-  private final short replFactor = 1;
-  private final long blockSize = 512;
-  private long fileLen = blockSize * 3;
-  private final long seed = Time.now();
-  private final Random r = new Random(seed);
-  private final PermissionGenerator permGenerator = new PermissionGenerator(r);
-  private static final int NUM_TESTS = 50;
+  private static final int NUM_TESTS = 1000;
   private static final int NUM_NN_HANDLER = 10;
-  private static final int ASYNC_CALL_LIMIT = 1000;
+  private static final int ASYNC_CALL_LIMIT = 100;
 
   private Configuration conf;
   private MiniDFSCluster cluster;
   private FileSystem fs;
-  private AsyncDistributedFileSystem adfs;
 
   @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 permission checking
-    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_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);
@@ -102,7 +86,6 @@ public class TestAsyncDFS {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
     fs = FileSystem.get(conf);
-    adfs = cluster.getFileSystem().getAsyncDistributedFileSystem();
   }
 
   @After
@@ -147,9 +130,13 @@ public class TestAsyncDFS {
     final String basePath = "testBatchAsyncAcl";
     final Path parent = new Path(String.format("/test/%s/", basePath));
 
+    AsyncDistributedFileSystem adfs = cluster.getFileSystem()
+        .getAsyncDistributedFileSystem();
+
     // prepare test
-    final Path[] paths = new Path[NUM_TESTS];
-    for (int i = 0; i < NUM_TESTS; i++) {
+    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));
@@ -166,7 +153,7 @@ public class TestAsyncDFS {
     int start = 0, end = 0;
     try {
       // test setAcl
-      for (int i = 0; i < NUM_TESTS; i++) {
+      for (int i = 0; i < count; i++) {
         for (;;) {
           try {
             Future<Void> retFuture = adfs.setAcl(paths[i], aclSpec);
@@ -179,12 +166,12 @@ public class TestAsyncDFS {
           }
         }
       }
-      waitForAclReturnValues(setAclRetFutures, end, NUM_TESTS);
+      waitForAclReturnValues(setAclRetFutures, end, count);
 
       // test getAclStatus
       start = 0;
       end = 0;
-      for (int i = 0; i < NUM_TESTS; i++) {
+      for (int i = 0; i < count; i++) {
         for (;;) {
           try {
             Future<AclStatus> retFuture = adfs.getAclStatus(paths[i]);
@@ -198,23 +185,13 @@ public class TestAsyncDFS {
           }
         }
       }
-      waitForAclReturnValues(getAclRetFutures, end, NUM_TESTS, paths,
+      waitForAclReturnValues(getAclRetFutures, end, count, paths,
           expectedAclSpec);
     } catch (Exception e) {
       throw e;
     }
   }
 
-  static void waitForReturnValues(final Map<Integer, Future<Void>> retFutures,
-      final int start, final int end)
-      throws InterruptedException, ExecutionException {
-    LOG.info(String.format("calling waitForReturnValues [%d, %d)", start, end));
-    for (int i = start; i < end; i++) {
-      LOG.info("calling Future#get #" + i);
-      retFutures.get(i).get();
-    }
-  }
-
   private void waitForAclReturnValues(
       final Map<Integer, Future<Void>> aclRetFutures, final int start,
       final int end) throws InterruptedException, ExecutionException {
@@ -289,12 +266,9 @@ public class TestAsyncDFS {
 
     final Path parent = new Path("/test/async_api_exception/");
     final Path aclDir = new Path(parent, "aclDir");
-    final Path src = new Path(parent, "src");
-    final Path dst = new Path(parent, "dst");
-    fs.mkdirs(aclDir, FsPermission.createImmutable((short) 0700));
-    fs.mkdirs(src);
+    fs.mkdirs(aclDir, FsPermission.createImmutable((short) 0770));
 
-    AsyncDistributedFileSystem adfs1 = ugi1
+    AsyncDistributedFileSystem adfs = ugi1
         .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
           @Override
           public AsyncDistributedFileSystem run() throws Exception {
@@ -303,36 +277,9 @@ public class TestAsyncDFS {
         });
 
     Future<Void> retFuture;
-    // test rename
-    try {
-      retFuture = adfs1.rename(src, dst, Rename.OVERWRITE);
-      retFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-      assertTrue("Permission denied messages must carry the path parent", e
-          .getMessage().contains(src.getParent().toUri().getPath()));
-    }
-
-    // test setPermission
-    FsPermission fsPerm = new FsPermission(permGenerator.next());
-    try {
-      retFuture = adfs1.setPermission(src, fsPerm);
-      retFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-    }
-
-    // test setOwner
-    try {
-      retFuture = adfs1.setOwner(src, "user1", "group2");
-      retFuture.get();
-    } catch (ExecutionException e) {
-      checkPermissionDenied(e, src, user1);
-    }
-
     // test setAcl
     try {
-      retFuture = adfs1.setAcl(aclDir,
+      retFuture = adfs.setAcl(aclDir,
           Lists.newArrayList(aclEntry(ACCESS, USER, ALL)));
       retFuture.get();
       fail("setAcl should fail with permission denied");
@@ -342,7 +289,7 @@ public class TestAsyncDFS {
 
     // test getAclStatus
     try {
-      Future<AclStatus> aclRetFuture = adfs1.getAclStatus(aclDir);
+      Future<AclStatus> aclRetFuture = adfs.getAclStatus(aclDir);
       aclRetFuture.get();
       fail("getAclStatus should fail with permission denied");
     } catch (ExecutionException e) {
@@ -360,148 +307,4 @@ public class TestAsyncDFS {
     assertTrue("Permission denied messages must carry the name of the path",
         e.getMessage().contains(dir.getName()));
   }
-
-
-  @Test(timeout = 120000)
-  public void testConcurrentAsyncAPI() throws Exception {
-    String group1 = "group1";
-    String group2 = "group2";
-    String user1 = "user1";
-
-    // 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);
-
-    // prepare for test
-    final Path parent = new Path(
-        String.format("/test/%s/", "testConcurrentAsyncAPI"));
-    final Path[] srcs = new Path[NUM_TESTS];
-    final Path[] dsts = new Path[NUM_TESTS];
-    short[] permissions = new short[NUM_TESTS];
-    for (int i = 0; i < NUM_TESTS; i++) {
-      srcs[i] = new Path(parent, "src" + i);
-      dsts[i] = new Path(parent, "dst" + i);
-      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
-      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
-      assertTrue(fs.exists(srcs[i]));
-      assertTrue(fs.getFileStatus(srcs[i]).isFile());
-      assertTrue(fs.exists(dsts[i]));
-      assertTrue(fs.getFileStatus(dsts[i]).isFile());
-      permissions[i] = permGenerator.next();
-    }
-
-    Map<Integer, Future<Void>> renameRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<Void>> permRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    Map<Integer, Future<Void>> ownerRetFutures =
-        new HashMap<Integer, Future<Void>>();
-    int start = 0, end = 0;
-    // test rename
-    for (int i = 0; i < NUM_TESTS; i++) {
-      for (;;) {
-        try {
-          Future<Void> returnFuture = adfs.rename(srcs[i], dsts[i],
-              Rename.OVERWRITE);
-          renameRetFutures.put(i, returnFuture);
-          break;
-        } catch (AsyncCallLimitExceededException e) {
-          start = end;
-          end = i;
-          waitForReturnValues(renameRetFutures, start, end);
-        }
-      }
-    }
-
-    // wait for completing the calls
-    waitForAclReturnValues(renameRetFutures, end, NUM_TESTS);
-
-    // verify the src should not exist, dst should
-    for (int i = 0; i < NUM_TESTS; i++) {
-      assertFalse(fs.exists(srcs[i]));
-      assertTrue(fs.exists(dsts[i]));
-    }
-
-    // test permissions
-    for (int i = 0; i < NUM_TESTS; i++) {
-      for (;;) {
-        try {
-          Future<Void> retFuture = adfs.setPermission(dsts[i],
-              new FsPermission(permissions[i]));
-          permRetFutures.put(i, retFuture);
-          break;
-        } catch (AsyncCallLimitExceededException e) {
-          start = end;
-          end = i;
-          waitForReturnValues(permRetFutures, start, end);
-        }
-      }
-    }
-    // wait for completing the calls
-    waitForAclReturnValues(permRetFutures, end, NUM_TESTS);
-
-    // verify the permission
-    for (int i = 0; i < NUM_TESTS; i++) {
-      assertTrue(fs.exists(dsts[i]));
-      FsPermission fsPerm = new FsPermission(permissions[i]);
-      checkAccessPermissions(fs.getFileStatus(dsts[i]), fsPerm.getUserAction());
-    }
-
-    // test setOwner
-    start = 0;
-    end = 0;
-    for (int i = 0; i < NUM_TESTS; i++) {
-      for (;;) {
-        try {
-          Future<Void> retFuture = adfs.setOwner(dsts[i], "user1", "group2");
-          ownerRetFutures.put(i, retFuture);
-          break;
-        } catch (AsyncCallLimitExceededException e) {
-          start = end;
-          end = i;
-          waitForReturnValues(ownerRetFutures, start, end);
-        }
-      }
-    }
-    // wait for completing the calls
-    waitForAclReturnValues(ownerRetFutures, end, NUM_TESTS);
-
-    // verify the owner
-    for (int i = 0; i < NUM_TESTS; i++) {
-      assertTrue(fs.exists(dsts[i]));
-      assertTrue("user1".equals(fs.getFileStatus(dsts[i]).getOwner()));
-      assertTrue("group2".equals(fs.getFileStatus(dsts[i]).getGroup()));
-    }
-  }
-
-  static void checkAccessPermissions(FileStatus stat, FsAction mode)
-      throws IOException {
-    checkAccessPermissions(UserGroupInformation.getCurrentUser(), stat, mode);
-  }
-
-  static void checkAccessPermissions(final UserGroupInformation ugi,
-      FileStatus stat, FsAction mode) throws IOException {
-    FsPermission perm = stat.getPermission();
-    String user = ugi.getShortUserName();
-    List<String> groups = Arrays.asList(ugi.getGroupNames());
-
-    if (user.equals(stat.getOwner())) {
-      if (perm.getUserAction().implies(mode)) {
-        return;
-      }
-    } else if (groups.contains(stat.getGroup())) {
-      if (perm.getGroupAction().implies(mode)) {
-        return;
-      }
-    } else {
-      if (perm.getOtherAction().implies(mode)) {
-        return;
-      }
-    }
-    throw new AccessControlException(String.format(
-        "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat
-            .getPath(), stat.getOwner(), stat.getGroup(),
-        stat.isDirectory() ? "d" : "-", perm));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ee5912e/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 8d3e509..03c8151 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
@@ -19,11 +19,14 @@ package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
@@ -31,157 +34,521 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.ipc.AsyncCallLimitExceededException;
-import org.junit.After;
-import org.junit.Before;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 public class TestAsyncDFSRename {
   public static final Log LOG = LogFactory.getLog(TestAsyncDFSRename.class);
-  private final short replFactor = 1;
+  private final long seed = Time.now();
+  private final Random r = new Random(seed);
+  private final PermissionGenerator permGenerator = new PermissionGenerator(r);
+  private final short replFactor = 2;
   private final long blockSize = 512;
   private long fileLen = blockSize * 3;
-  private static final int NUM_TESTS = 50;
-  private static final int NUM_NN_HANDLER = 10;
-  private static final int ASYNC_CALL_LIMIT = 1000;
-
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private FileSystem fs;
-  private AsyncDistributedFileSystem adfs;
-
-  @Before
-  public void setup() throws IOException {
-    conf = new HdfsConfiguration();
-    // 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(1).build();
+
+  /**
+   * Check the blocks of dst file are cleaned after rename with overwrite
+   * Restart NN to check the rename successfully
+   */
+  @Test(timeout = 60000)
+  public void testAsyncRenameWithOverwrite() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        replFactor).build();
     cluster.waitActive();
-    fs = FileSystem.get(conf);
-    adfs = cluster.getFileSystem().getAsyncDistributedFileSystem();
-  }
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
 
-  @After
-  public void tearDown() throws IOException {
-    if (fs != null) {
-      fs.close();
-      fs = null;
-    }
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
+    try {
+      String src = "/foo/src";
+      String dst = "/foo/dst";
+      String src2 = "/foo/src2";
+      String dst2 = "/foo/dst2";
+      Path srcPath = new Path(src);
+      Path dstPath = new Path(dst);
+      Path srcPath2 = new Path(src2);
+      Path dstPath2 = new Path(dst2);
+
+      DFSTestUtil.createFile(dfs, srcPath, fileLen, replFactor, 1);
+      DFSTestUtil.createFile(dfs, dstPath, fileLen, replFactor, 1);
+      DFSTestUtil.createFile(dfs, srcPath2, fileLen, replFactor, 1);
+      DFSTestUtil.createFile(dfs, dstPath2, fileLen, replFactor, 1);
+
+      LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(
+          cluster.getNameNode(), dst, 0, fileLen);
+      LocatedBlocks lbs2 = NameNodeAdapter.getBlockLocations(
+          cluster.getNameNode(), dst2, 0, fileLen);
+      BlockManager bm = NameNodeAdapter.getNamesystem(cluster.getNameNode())
+          .getBlockManager();
+      assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock()
+          .getLocalBlock()) != null);
+      assertTrue(bm.getStoredBlock(lbs2.getLocatedBlocks().get(0).getBlock()
+          .getLocalBlock()) != null);
+
+      Future<Void> retVal1 = adfs.rename(srcPath, dstPath, Rename.OVERWRITE);
+      Future<Void> retVal2 = adfs.rename(srcPath2, dstPath2, Rename.OVERWRITE);
+      retVal1.get();
+      retVal2.get();
+
+      assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock()
+          .getLocalBlock()) == null);
+      assertTrue(bm.getStoredBlock(lbs2.getLocatedBlocks().get(0).getBlock()
+          .getLocalBlock()) == null);
+
+      // Restart NN and check the rename successfully
+      cluster.restartNameNodes();
+      assertFalse(dfs.exists(srcPath));
+      assertTrue(dfs.exists(dstPath));
+      assertFalse(dfs.exists(srcPath2));
+      assertTrue(dfs.exists(dstPath2));
+    } finally {
+      if (dfs != null) {
+        dfs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 
   @Test(timeout = 60000)
   public void testCallGetReturnValueMultipleTimes() throws Exception {
-    final Path parent = new Path("/test/testCallGetReturnValueMultipleTimes/");
-    assertTrue(fs.mkdirs(parent));
+    final Path renameDir = new Path(
+        "/test/testCallGetReturnValueMultipleTimes/");
+    final Configuration conf = new HdfsConfiguration();
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY, 200);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(2).build();
+    cluster.waitActive();
+    final DistributedFileSystem dfs = cluster.getFileSystem();
+    final AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
+    final int count = 100;
+    final Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
 
-    // prepare test
-    final Path[] srcs = new Path[NUM_TESTS];
-    final Path[] dsts = new Path[NUM_TESTS];
-    for (int i = 0; i < NUM_TESTS; i++) {
-      srcs[i] = new Path(parent, "src" + i);
-      dsts[i] = new Path(parent, "dst" + i);
-      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
-      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
+    assertTrue(dfs.mkdirs(renameDir));
+
+    try {
+      // concurrently invoking many rename
+      for (int i = 0; i < count; i++) {
+        Path src = new Path(renameDir, "src" + i);
+        Path dst = new Path(renameDir, "dst" + i);
+        DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
+        DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
+        Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+        returnFutures.put(i, returnFuture);
+      }
+
+      for (int i = 0; i < 5; i++) {
+        verifyCallGetReturnValueMultipleTimes(returnFutures, count, cluster,
+            renameDir, dfs);
+      }
+    } finally {
+      if (dfs != null) {
+        dfs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
+  }
 
-    // concurrently invoking many rename
-    final Map<Integer, Future<Void>> reFutures =
-        new HashMap<Integer, Future<Void>>();
-    for (int i = 0; i < NUM_TESTS; i++) {
-      Future<Void> retFuture = adfs.rename(srcs[i], dsts[i],
-          Rename.OVERWRITE);
-      reFutures.put(i, retFuture);
+  private void verifyCallGetReturnValueMultipleTimes(
+      Map<Integer, Future<Void>> returnFutures, int count,
+      MiniDFSCluster cluster, Path renameDir, DistributedFileSystem dfs)
+      throws InterruptedException, ExecutionException, IOException {
+    // wait for completing the calls
+    for (int i = 0; i < count; i++) {
+      returnFutures.get(i).get();
     }
 
-    assertEquals(NUM_TESTS, reFutures.size());
+    // Restart NN and check the rename successfully
+    cluster.restartNameNodes();
 
-    for (int i = 0; i < 5; i++) {
-      verifyCallGetReturnValueMultipleTimes(reFutures, srcs, dsts);
+    // very the src dir should not exist, dst should
+    for (int i = 0; i < count; i++) {
+      Path src = new Path(renameDir, "src" + i);
+      Path dst = new Path(renameDir, "dst" + i);
+      assertFalse(dfs.exists(src));
+      assertTrue(dfs.exists(dst));
     }
   }
 
-  private void verifyCallGetReturnValueMultipleTimes(
-      final Map<Integer, Future<Void>> reFutures, final Path[] srcs,
-      final Path[] dsts)
-      throws InterruptedException, ExecutionException, IOException {
+  @Test
+  public void testConservativeConcurrentAsyncRenameWithOverwrite()
+      throws Exception {
+    internalTestConcurrentAsyncRenameWithOverwrite(100,
+        "testAggressiveConcurrentAsyncRenameWithOverwrite");
+  }
 
-    // wait for completing the calls
-    waitForReturnValues(reFutures, 0, NUM_TESTS);
+  @Test(timeout = 60000)
+  public void testAggressiveConcurrentAsyncRenameWithOverwrite()
+      throws Exception {
+    internalTestConcurrentAsyncRenameWithOverwrite(10000,
+        "testConservativeConcurrentAsyncRenameWithOverwrite");
+  }
+
+  private void internalTestConcurrentAsyncRenameWithOverwrite(
+      final int asyncCallLimit, final String basePath) throws Exception {
+    final Path renameDir = new Path(String.format("/test/%s/", basePath));
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
+        asyncCallLimit);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
+        .build();
+    cluster.waitActive();
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    AsyncDistributedFileSystem adfs = dfs.getAsyncDistributedFileSystem();
+    int count = 1000;
+    int start = 0, end = 0;
+    Map<Integer, Future<Void>> returnFutures = new HashMap<Integer, Future<Void>>();
+
+    assertTrue(dfs.mkdirs(renameDir));
+
+    try {
+      // concurrently invoking many rename
+      for (int i = 0; i < count; i++) {
+        Path src = new Path(renameDir, "src" + i);
+        Path dst = new Path(renameDir, "dst" + i);
+        DFSTestUtil.createFile(dfs, src, fileLen, replFactor, 1);
+        DFSTestUtil.createFile(dfs, dst, fileLen, replFactor, 1);
+        for (;;) {
+          try {
+            LOG.info("rename #" + i);
+            Future<Void> returnFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+            returnFutures.put(i, returnFuture);
+            break;
+          } catch (AsyncCallLimitExceededException e) {
+            /**
+             * reached limit of async calls, fetch results of finished async
+             * calls to let follow-on calls go
+             */
+            LOG.error(e);
+            start = end;
+            end = i;
+            LOG.info(String.format("start=%d, end=%d, i=%d", start, end, i));
+            waitForReturnValues(returnFutures, start, end);
+          }
+        }
+      }
 
-    // verify the src dir should not exist, dst should
-    verifyRenames(srcs, dsts);
+      // wait for completing the calls
+      for (int i = start; i < count; i++) {
+        returnFutures.get(i).get();
+      }
+
+      // Restart NN and check the rename successfully
+      cluster.restartNameNodes();
+
+      // very the src dir should not exist, dst should
+      for (int i = 0; i < count; i++) {
+        Path src = new Path(renameDir, "src" + i);
+        Path dst = new Path(renameDir, "dst" + i);
+        assertFalse(dfs.exists(src));
+        assertTrue(dfs.exists(dst));
+      }
+    } finally {
+      if (dfs != null) {
+        dfs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private void waitForReturnValues(
+      final Map<Integer, Future<Void>> returnFutures, final int start,
+      final int end) throws InterruptedException, ExecutionException {
+    LOG.info(String.format("calling waitForReturnValues [%d, %d)", start, end));
+    for (int i = start; i < end; i++) {
+      LOG.info("calling Future#get #" + i);
+      returnFutures.get(i).get();
+    }
+  }
+
+  @Test
+  public void testConservativeConcurrentAsyncAPI() throws Exception {
+    internalTestConcurrentAsyncAPI(100, "testConservativeConcurrentAsyncAPI");
   }
 
   @Test(timeout = 60000)
-  public void testConcurrentAsyncRename() throws Exception {
-    final Path parent = new Path(
-        String.format("/test/%s/", "testConcurrentAsyncRename"));
-    assertTrue(fs.mkdirs(parent));
-
-    // prepare test
-    final Path[] srcs = new Path[NUM_TESTS];
-    final Path[] dsts = new Path[NUM_TESTS];
-    for (int i = 0; i < NUM_TESTS; i++) {
+  public void testAggressiveConcurrentAsyncAPI() throws Exception {
+    internalTestConcurrentAsyncAPI(10000, "testAggressiveConcurrentAsyncAPI");
+  }
+
+  private void internalTestConcurrentAsyncAPI(final int asyncCallLimit,
+      final String basePath) throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    String group1 = "group1";
+    String group2 = "group2";
+    String user1 = "user1";
+    int count = 500;
+
+    // explicitly turn on permission checking
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+    // set the limit of max async calls
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_ASYNC_CALLS_MAX_KEY,
+        asyncCallLimit);
+
+    // 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);
+
+    // start mini cluster
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3).build();
+    cluster.waitActive();
+    AsyncDistributedFileSystem adfs = cluster.getFileSystem()
+        .getAsyncDistributedFileSystem();
+
+    // prepare for test
+    FileSystem rootFs = FileSystem.get(conf);
+    final Path parent = new Path(String.format("/test/%s/", basePath));
+    final Path[] srcs = new Path[count];
+    final Path[] dsts = new Path[count];
+    short[] permissions = new short[count];
+    for (int i = 0; i < count; i++) {
       srcs[i] = new Path(parent, "src" + i);
       dsts[i] = new Path(parent, "dst" + i);
-      DFSTestUtil.createFile(fs, srcs[i], fileLen, replFactor, 1);
-      DFSTestUtil.createFile(fs, dsts[i], fileLen, replFactor, 1);
+      DFSTestUtil.createFile(rootFs, srcs[i], fileLen, replFactor, 1);
+      DFSTestUtil.createFile(rootFs, dsts[i], fileLen, replFactor, 1);
+      assertTrue(rootFs.exists(srcs[i]));
+      assertTrue(rootFs.getFileStatus(srcs[i]).isFile());
+      assertTrue(rootFs.exists(dsts[i]));
+      assertTrue(rootFs.getFileStatus(dsts[i]).isFile());
+      permissions[i] = permGenerator.next();
     }
 
-    // concurrently invoking many rename
-    int start = 0, end = 0;
-    Map<Integer, Future<Void>> retFutures =
+    Map<Integer, Future<Void>> renameRetFutures =
+        new HashMap<Integer, Future<Void>>();
+    Map<Integer, Future<Void>> permRetFutures =
+        new HashMap<Integer, Future<Void>>();
+    Map<Integer, Future<Void>> ownerRetFutures =
         new HashMap<Integer, Future<Void>>();
-    for (int i = 0; i < NUM_TESTS; i++) {
+    int start = 0, end = 0;
+    // test rename
+    for (int i = 0; i < count; i++) {
       for (;;) {
         try {
-          LOG.info("rename #" + i);
-          Future<Void> retFuture = adfs.rename(srcs[i], dsts[i],
+          Future<Void> returnFuture = adfs.rename(srcs[i], dsts[i],
               Rename.OVERWRITE);
-          retFutures.put(i, retFuture);
+          renameRetFutures.put(i, returnFuture);
           break;
         } catch (AsyncCallLimitExceededException e) {
-          /**
-           * reached limit of async calls, fetch results of finished async calls
-           * to let follow-on calls go
-           */
-          LOG.error(e);
           start = end;
           end = i;
-          LOG.info(String.format("start=%d, end=%d, i=%d", start, end, i));
-          waitForReturnValues(retFutures, start, end);
+          waitForReturnValues(renameRetFutures, start, end);
         }
       }
     }
 
     // wait for completing the calls
-    waitForReturnValues(retFutures, end, NUM_TESTS);
+    for (int i = start; i < count; i++) {
+      renameRetFutures.get(i).get();
+    }
+
+    // Restart NN and check the rename successfully
+    cluster.restartNameNodes();
+
+    // very the src should not exist, dst should
+    for (int i = 0; i < count; i++) {
+      assertFalse(rootFs.exists(srcs[i]));
+      assertTrue(rootFs.exists(dsts[i]));
+    }
+
+    // test permissions
+    try {
+      for (int i = 0; i < count; i++) {
+        for (;;) {
+          try {
+            Future<Void> retFuture = adfs.setPermission(dsts[i],
+                new FsPermission(permissions[i]));
+            permRetFutures.put(i, retFuture);
+            break;
+          } catch (AsyncCallLimitExceededException e) {
+            start = end;
+            end = i;
+            waitForReturnValues(permRetFutures, start, end);
+          }
+        }
+      }
+      // wait for completing the calls
+      for (int i = start; i < count; i++) {
+        permRetFutures.get(i).get();
+      }
+
+      // Restart NN and check permission then
+      cluster.restartNameNodes();
+
+      // verify the permission
+      for (int i = 0; i < count; i++) {
+        assertTrue(rootFs.exists(dsts[i]));
+        FsPermission fsPerm = new FsPermission(permissions[i]);
+        checkAccessPermissions(rootFs.getFileStatus(dsts[i]),
+            fsPerm.getUserAction());
+      }
+
+      // test setOwner
+      start = 0;
+      end = 0;
+      for (int i = 0; i < count; i++) {
+        for (;;) {
+          try {
+            Future<Void> retFuture = adfs.setOwner(dsts[i], "user1",
+                "group2");
+            ownerRetFutures.put(i, retFuture);
+            break;
+          } catch (AsyncCallLimitExceededException e) {
+            start = end;
+            end = i;
+            waitForReturnValues(ownerRetFutures, start, end);
+          }
+        }
+      }
+      // wait for completing the calls
+      for (int i = start; i < count; i++) {
+        ownerRetFutures.get(i).get();
+      }
 
-    // verify the src dir should not exist, dst should
-    verifyRenames(srcs, dsts);
+      // Restart NN and check owner then
+      cluster.restartNameNodes();
+
+      // verify the owner
+      for (int i = 0; i < count; i++) {
+        assertTrue(rootFs.exists(dsts[i]));
+        assertTrue(
+            "user1".equals(rootFs.getFileStatus(dsts[i]).getOwner()));
+        assertTrue(
+            "group2".equals(rootFs.getFileStatus(dsts[i]).getGroup()));
+      }
+    } catch (AccessControlException ace) {
+      throw ace;
+    } finally {
+      if (rootFs != null) {
+        rootFs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
   }
 
-  private void verifyRenames(final Path[] srcs, final Path[] dsts)
+  static void checkAccessPermissions(FileStatus stat, FsAction mode)
       throws IOException {
-    for (int i = 0; i < NUM_TESTS; i++) {
-      assertFalse(fs.exists(srcs[i]));
-      assertTrue(fs.exists(dsts[i]));
+    checkAccessPermissions(UserGroupInformation.getCurrentUser(), stat, mode);
+  }
+
+  static void checkAccessPermissions(final UserGroupInformation ugi,
+      FileStatus stat, FsAction mode) throws IOException {
+    FsPermission perm = stat.getPermission();
+    String user = ugi.getShortUserName();
+    List<String> groups = Arrays.asList(ugi.getGroupNames());
+
+    if (user.equals(stat.getOwner())) {
+      if (perm.getUserAction().implies(mode)) {
+        return;
+      }
+    } else if (groups.contains(stat.getGroup())) {
+      if (perm.getGroupAction().implies(mode)) {
+        return;
+      }
+    } else {
+      if (perm.getOtherAction().implies(mode)) {
+        return;
+      }
     }
+    throw new AccessControlException(String.format(
+        "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s", user, stat
+            .getPath(), stat.getOwner(), stat.getGroup(),
+        stat.isDirectory() ? "d" : "-", perm));
   }
 
-  void waitForReturnValues(final Map<Integer, Future<Void>> retFutures,
-      final int start, final int end)
-      throws InterruptedException, ExecutionException {
-    TestAsyncDFS.waitForReturnValues(retFutures, start, end);
+  @Test(timeout = 60000)
+  public void testAsyncAPIWithException() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    String group1 = "group1";
+    String group2 = "group2";
+    String user1 = "user1";
+    UserGroupInformation ugi1;
+
+    // explicitly turn on permission checking
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+
+    // 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 MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3).build();
+    cluster.waitActive();
+
+    FileSystem rootFs = FileSystem.get(conf);
+    final Path renameDir = new Path("/test/async_api_exception/");
+    final Path src = new Path(renameDir, "src");
+    final Path dst = new Path(renameDir, "dst");
+    rootFs.mkdirs(src);
+
+    AsyncDistributedFileSystem adfs = ugi1
+        .doAs(new PrivilegedExceptionAction<AsyncDistributedFileSystem>() {
+          @Override
+          public AsyncDistributedFileSystem run() throws Exception {
+            return cluster.getFileSystem().getAsyncDistributedFileSystem();
+          }
+        });
+
+    Future<Void> retFuture;
+    try {
+      retFuture = adfs.rename(src, dst, Rename.OVERWRITE);
+      retFuture.get();
+    } catch (ExecutionException e) {
+      TestAsyncDFS.checkPermissionDenied(e, src, user1);
+      assertTrue("Permission denied messages must carry the path parent", e
+          .getMessage().contains(src.getParent().toUri().getPath()));
+    }
+
+    FsPermission fsPerm = new FsPermission(permGenerator.next());
+    try {
+      retFuture = adfs.setPermission(src, fsPerm);
+      retFuture.get();
+    } catch (ExecutionException e) {
+      TestAsyncDFS.checkPermissionDenied(e, src, user1);
+      assertTrue("Permission denied messages must carry the name of the path",
+          e.getMessage().contains(src.getName()));
+    }
+
+    try {
+      retFuture = adfs.setOwner(src, "user1", "group2");
+      retFuture.get();
+    } catch (ExecutionException e) {
+      TestAsyncDFS.checkPermissionDenied(e, src, user1);
+      assertTrue("Permission denied messages must carry the name of the path",
+          e.getMessage().contains(src.getName()));
+    } finally {
+      if (rootFs != null) {
+        rootFs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
   }
 }
\ No newline at end of file


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


[21/47] hadoop git commit: HDFS-7767. Use the noredirect flag in WebHDFS to allow web browsers to upload files via the NN UI (Ravi Prakash via aw)

Posted by ae...@apache.org.
HDFS-7767. Use the noredirect flag in WebHDFS to allow web browsers to upload files via the NN UI (Ravi Prakash via aw)


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

Branch: refs/heads/HDFS-1312
Commit: 99a771cd7a3f792a76ac89c406b82a983c059d28
Parents: 15f0184
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jun 3 17:07:39 2016 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Fri Jun 3 17:07:39 2016 -0700

----------------------------------------------------------------------
 .../src/main/webapps/hdfs/explorer.html         | 25 +++++++++-
 .../src/main/webapps/hdfs/explorer.js           | 51 ++++++++++++++++++++
 2 files changed, 74 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99a771cd/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index 5106006..51f72e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -119,6 +119,23 @@
       </div>
     </div>
 
+    <div class="modal" id="modal-upload-file" tabindex="-1" role="dialog" aria-hidden="true">
+      <div class="modal-dialog">
+         <div class="modal-content">
+           <div class="modal-header"><button type="button" class="close"
+             data-dismiss="modal" aria-hidden="true">&times;</button>
+             <h4 class="modal-title" id="file-upload-title">Upload File</h4>
+           </div>
+           <div class="modal-body" id="file-upload-body">
+             <input id="modal-upload-file-input" type="file" class="file" multiple>
+           </div>
+           <div class="modal-footer">
+             <button type="button" class="btn btn-success" data-dismiss="modal">Close</button>
+             <button type="button" class="btn btn-success" id="modal-upload-file-button" data-complete-text="Uploading...">Upload</button>
+           </div>
+         </div>
+      </div>
+    </div>
   <div class="modal" id="delete-modal" tabindex="-1" role="dialog" aria-hidden="true">
     <div class="modal-dialog">
       <div class="modal-content">
@@ -142,7 +159,7 @@
   </div>
 
       <div class="row">
-      <div class="col-xs-11">
+      <div class="col-xs-10 col-md-10">
         <form onsubmit="return false;">
           <div class="input-group">
             <input type="text" class="form-control" id="directory"/>
@@ -152,12 +169,16 @@
           </div>
         </form>
       </div>
-      <div class="col-xs-1">
+      <div class="col-xs-2 col-md-2">
         <button type="button" class="btn btn-default" data-toggle="modal"
           aria-label="New Directory" data-target="#btn-create-directory"
           title="Create Directory">
             <span class="glyphicon glyphicon-folder-open"></span>
         </button>
+        <button type="button" class="btn btn-default" data-toggle="modal"
+          data-target="#modal-upload-file" title="Upload Files">
+            <span class="glyphicon glyphicon-cloud-upload"></span>
+        </button>
       </div>
     </div>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99a771cd/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index adb83a8..6fa5f19 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -366,5 +366,56 @@
     });
   })
 
+  $('#modal-upload-file-button').click(function() {
+    $(this).prop('disabled', true);
+    $(this).button('complete');
+    var files = []
+    var numCompleted = 0
+
+    for(var i = 0; i < $('#modal-upload-file-input').prop('files').length; i++) {
+      (function() {
+        var file = $('#modal-upload-file-input').prop('files')[i];
+        var url = '/webhdfs/v1' + current_directory;
+        url = encode_path(append_path(url, file.name));
+        url += '?op=CREATE&noredirect=true';
+        files.push( { file: file } )
+        files[i].request = $.ajax({
+          type: 'PUT',
+          url: url,
+          processData: false,
+          crossDomain: true
+        });
+      })()
+     }
+    for(var f in files) {
+      (function() {
+        var file = files[f];
+        file.request.done(function(data) {
+          var url = data['Location'];
+          $.ajax({
+            type: 'PUT',
+            url: url,
+            data: file.file,
+            processData: false,
+            crossDomain: true
+          }).complete(function(data) {
+            numCompleted++;
+            if(numCompleted == files.length) {
+              $('#modal-upload-file').modal('hide');
+              $('#modal-upload-file-button').button('reset');
+              browse_directory(current_directory);
+            }
+          }).error(function(jqXHR, textStatus, errorThrown) {
+            numCompleted++;
+            show_err_msg("Couldn't upload the file " + file.file.name + ". "+ errorThrown);
+          });
+        }).error(function(jqXHR, textStatus, errorThrown) {
+          numCompleted++;
+          show_err_msg("Couldn't find datanode to write file. " + errorThrown);
+        });
+      })();
+    }
+  });
+
   init();
 })();


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


[23/47] hadoop git commit: Revert "HADOOP-13226 Support async call retry and failover."

Posted by ae...@apache.org.
Revert "HADOOP-13226 Support async call retry and failover."

This reverts commit 83f2f78c118a7e52aba5104bd97b0acedc96be7b.


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

Branch: refs/heads/HDFS-1312
Commit: 5360da8bd9f720384860f411bee081aef13b4bd4
Parents: 47e0321
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 3 18:09:09 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 3 18:09:09 2016 -0700

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   8 +-
 .../hadoop/io/retry/AsyncCallHandler.java       | 321 -------------------
 .../org/apache/hadoop/io/retry/CallReturn.java  |  75 -----
 .../hadoop/io/retry/RetryInvocationHandler.java | 134 ++------
 .../apache/hadoop/io/retry/RetryPolicies.java   |   4 +-
 .../main/java/org/apache/hadoop/ipc/Client.java |  25 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |  13 +-
 .../apache/hadoop/util/concurrent/AsyncGet.java |  17 +-
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     |  10 +-
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |   7 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  42 +--
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    |  43 ++-
 .../apache/hadoop/hdfs/TestAsyncHDFSWithHA.java | 181 -----------
 .../hdfs/server/namenode/ha/HATestUtil.java     |   9 +-
 14 files changed, 114 insertions(+), 775 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
index a644aa5..ab8673b 100644
--- a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
@@ -345,13 +345,7 @@
        <Bug pattern="SF_SWITCH_FALLTHROUGH" />
      </Match>
 
-     <!-- WA_NOT_IN_LOOP is invalid in util.concurrent.AsyncGet$Util.wait. -->
-     <Match>
-       <Class name="org.apache.hadoop.util.concurrent.AsyncGet$Util" />
-       <Method name="wait" />
-       <Bug pattern="WA_NOT_IN_LOOP" />
-     </Match>
-
+     <!-- Synchronization performed on util.concurrent instance. -->
      <Match>
        <Class name="org.apache.hadoop.service.AbstractService" />
        <Method name="stop" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java
deleted file mode 100644
index 5a03b03..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java
+++ /dev/null
@@ -1,321 +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.io.retry;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.Time;
-import org.apache.hadoop.util.concurrent.AsyncGet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.Method;
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicReference;
-
-/** Handle async calls. */
-@InterfaceAudience.Private
-public class AsyncCallHandler {
-  static final Logger LOG = LoggerFactory.getLogger(AsyncCallHandler.class);
-
-  private static final ThreadLocal<AsyncGet<?, Exception>>
-      LOWER_LAYER_ASYNC_RETURN = new ThreadLocal<>();
-  private static final ThreadLocal<AsyncGet<Object, Throwable>>
-      ASYNC_RETURN = new ThreadLocal<>();
-
-  /** @return the async return value from {@link AsyncCallHandler}. */
-  @InterfaceStability.Unstable
-  @SuppressWarnings("unchecked")
-  public static <R, T extends  Throwable> AsyncGet<R, T> getAsyncReturn() {
-    final AsyncGet<R, T> asyncGet = (AsyncGet<R, T>)ASYNC_RETURN.get();
-    if (asyncGet != null) {
-      ASYNC_RETURN.set(null);
-      return asyncGet;
-    } else {
-      return (AsyncGet<R, T>) getLowerLayerAsyncReturn();
-    }
-  }
-
-  /** For the lower rpc layers to set the async return value. */
-  @InterfaceStability.Unstable
-  public static void setLowerLayerAsyncReturn(
-      AsyncGet<?, Exception> asyncReturn) {
-    LOWER_LAYER_ASYNC_RETURN.set(asyncReturn);
-  }
-
-  private static AsyncGet<?, Exception> getLowerLayerAsyncReturn() {
-    final AsyncGet<?, Exception> asyncGet = LOWER_LAYER_ASYNC_RETURN.get();
-    Preconditions.checkNotNull(asyncGet);
-    LOWER_LAYER_ASYNC_RETURN.set(null);
-    return asyncGet;
-  }
-
-  /** A simple concurrent queue which keeping track the empty start time. */
-  static class ConcurrentQueue<T> {
-    private final Queue<T> queue = new LinkedList<>();
-    private long emptyStartTime = Time.monotonicNow();
-
-    synchronized int size() {
-      return queue.size();
-    }
-
-    /** Is the queue empty for more than the given time in millisecond? */
-    synchronized boolean isEmpty(long time) {
-      return queue.isEmpty() && Time.monotonicNow() - emptyStartTime > time;
-    }
-
-    synchronized void offer(T c) {
-      final boolean added = queue.offer(c);
-      Preconditions.checkState(added);
-    }
-
-    synchronized T poll() {
-      Preconditions.checkState(!queue.isEmpty());
-      final T t = queue.poll();
-      if (queue.isEmpty()) {
-        emptyStartTime = Time.monotonicNow();
-      }
-      return t;
-    }
-  }
-
-  /** A queue for handling async calls. */
-  static class AsyncCallQueue {
-    private final ConcurrentQueue<AsyncCall> queue = new ConcurrentQueue<>();
-    private final Processor processor = new Processor();
-
-    void addCall(AsyncCall call) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("add " + call);
-      }
-      queue.offer(call);
-      processor.tryStart();
-    }
-
-    void checkCalls() {
-      final int size = queue.size();
-      for (int i = 0; i < size; i++) {
-        final AsyncCall c = queue.poll();
-        if (!c.isDone()) {
-          queue.offer(c); // the call is not done yet, add it back.
-        }
-      }
-    }
-
-    /** Process the async calls in the queue. */
-    private class Processor {
-      static final long GRACE_PERIOD = 10*1000L;
-      static final long SLEEP_PERIOD = 100L;
-
-      private final AtomicReference<Thread> running = new AtomicReference<>();
-
-      boolean isRunning(Daemon d) {
-        return d == running.get();
-      }
-
-      void tryStart() {
-        final Thread current = Thread.currentThread();
-        if (running.compareAndSet(null, current)) {
-          final Daemon daemon = new Daemon() {
-            @Override
-            public void run() {
-              for (; isRunning(this);) {
-                try {
-                  Thread.sleep(SLEEP_PERIOD);
-                } catch (InterruptedException e) {
-                  kill(this);
-                  return;
-                }
-
-                checkCalls();
-                tryStop(this);
-              }
-            }
-          };
-
-          final boolean set = running.compareAndSet(current, daemon);
-          Preconditions.checkState(set);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Starting AsyncCallQueue.Processor " + daemon);
-          }
-          daemon.start();
-        }
-      }
-
-      void tryStop(Daemon d) {
-        if (queue.isEmpty(GRACE_PERIOD)) {
-          kill(d);
-        }
-      }
-
-      void kill(Daemon d) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Killing " + d);
-        }
-        final boolean set = running.compareAndSet(d, null);
-        Preconditions.checkState(set);
-      }
-    }
-  }
-
-  static class AsyncValue<V> {
-    private V value;
-
-    synchronized V waitAsyncValue(long timeout, TimeUnit unit)
-        throws InterruptedException, TimeoutException {
-      if (value != null) {
-        return value;
-      }
-      AsyncGet.Util.wait(this, timeout, unit);
-      if (value != null) {
-        return value;
-      }
-
-      throw new TimeoutException("waitCallReturn timed out "
-          + timeout + " " + unit);
-    }
-
-    synchronized void set(V v) {
-      Preconditions.checkNotNull(v);
-      Preconditions.checkState(value == null);
-      value = v;
-      notify();
-    }
-
-    synchronized boolean isDone() {
-      return value != null;
-    }
-  }
-
-  static class AsyncCall extends RetryInvocationHandler.Call {
-    private final AsyncCallHandler asyncCallHandler;
-
-    private final AsyncValue<CallReturn> asyncCallReturn = new AsyncValue<>();
-    private AsyncGet<?, Exception> lowerLayerAsyncGet;
-
-    AsyncCall(Method method, Object[] args, boolean isRpc, int callId,
-              RetryInvocationHandler.Counters counters,
-              RetryInvocationHandler<?> retryInvocationHandler,
-              AsyncCallHandler asyncCallHandler) {
-      super(method, args, isRpc, callId, counters, retryInvocationHandler);
-
-      this.asyncCallHandler = asyncCallHandler;
-    }
-
-    /** @return true if the call is done; otherwise, return false. */
-    boolean isDone() {
-      final CallReturn r = invokeOnce();
-      switch (r.getState()) {
-        case RETURNED:
-        case EXCEPTION:
-          asyncCallReturn.set(r); // the async call is done
-          return true;
-        case RETRY:
-          invokeOnce();
-          break;
-        case ASYNC_CALL_IN_PROGRESS:
-        case ASYNC_INVOKED:
-          // nothing to do
-          break;
-        default:
-          Preconditions.checkState(false);
-      }
-      return false;
-    }
-
-    @Override
-    CallReturn invoke() throws Throwable {
-      LOG.debug("{}.invoke {}", getClass().getSimpleName(), this);
-      if (lowerLayerAsyncGet != null) {
-        // async call was submitted early, check the lower level async call
-        final boolean isDone = lowerLayerAsyncGet.isDone();
-        LOG.trace("invoke: lowerLayerAsyncGet.isDone()? {}", isDone);
-        if (!isDone) {
-          return CallReturn.ASYNC_CALL_IN_PROGRESS;
-        }
-        try {
-          return new CallReturn(lowerLayerAsyncGet.get(0, TimeUnit.SECONDS));
-        } finally {
-          lowerLayerAsyncGet = null;
-        }
-      }
-
-      // submit a new async call
-      LOG.trace("invoke: ASYNC_INVOKED");
-      final boolean mode = Client.isAsynchronousMode();
-      try {
-        Client.setAsynchronousMode(true);
-        final Object r = invokeMethod();
-        // invokeMethod should set LOWER_LAYER_ASYNC_RETURN and return null.
-        Preconditions.checkState(r == null);
-        lowerLayerAsyncGet = getLowerLayerAsyncReturn();
-
-        if (counters.isZeros()) {
-          // first async attempt, initialize
-          LOG.trace("invoke: initAsyncCall");
-          asyncCallHandler.initAsyncCall(this, asyncCallReturn);
-        }
-        return CallReturn.ASYNC_INVOKED;
-      } finally {
-        Client.setAsynchronousMode(mode);
-      }
-    }
-  }
-
-  private final AsyncCallQueue asyncCalls = new AsyncCallQueue();
-  private volatile boolean hasSuccessfulCall = false;
-
-  AsyncCall newAsyncCall(Method method, Object[] args, boolean isRpc,
-                         int callId, RetryInvocationHandler.Counters counters,
-                         RetryInvocationHandler<?> retryInvocationHandler) {
-    return new AsyncCall(method, args, isRpc, callId, counters,
-        retryInvocationHandler, this);
-  }
-
-  boolean hasSuccessfulCall() {
-    return hasSuccessfulCall;
-  }
-
-  private void initAsyncCall(final AsyncCall asyncCall,
-                             final AsyncValue<CallReturn> asyncCallReturn) {
-    asyncCalls.addCall(asyncCall);
-
-    final AsyncGet<Object, Throwable> asyncGet
-        = new AsyncGet<Object, Throwable>() {
-      @Override
-      public Object get(long timeout, TimeUnit unit) throws Throwable {
-        final CallReturn c = asyncCallReturn.waitAsyncValue(timeout, unit);
-        final Object r = c.getReturnValue();
-        hasSuccessfulCall = true;
-        return r;
-      }
-
-      @Override
-      public boolean isDone() {
-        return asyncCallReturn.isDone();
-      }
-    };
-    ASYNC_RETURN.set(asyncGet);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/CallReturn.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/CallReturn.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/CallReturn.java
deleted file mode 100644
index 943725c..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/CallReturn.java
+++ /dev/null
@@ -1,75 +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.io.retry;
-
-import com.google.common.base.Preconditions;
-
-/** The call return from a method invocation. */
-class CallReturn {
-  /** The return state. */
-  enum State {
-    /** Call is returned successfully. */
-    RETURNED,
-    /** Call throws an exception. */
-    EXCEPTION,
-    /** Call should be retried according to the {@link RetryPolicy}. */
-    RETRY,
-    /** Call, which is async, is still in progress. */
-    ASYNC_CALL_IN_PROGRESS,
-    /** Call, which is async, just has been invoked. */
-    ASYNC_INVOKED
-  }
-
-  static final CallReturn ASYNC_CALL_IN_PROGRESS = new CallReturn(
-      State.ASYNC_CALL_IN_PROGRESS);
-  static final CallReturn ASYNC_INVOKED = new CallReturn(State.ASYNC_INVOKED);
-  static final CallReturn RETRY = new CallReturn(State.RETRY);
-
-  private final Object returnValue;
-  private final Throwable thrown;
-  private final State state;
-
-  CallReturn(Object r) {
-    this(r, null, State.RETURNED);
-  }
-  CallReturn(Throwable t) {
-    this(null, t, State.EXCEPTION);
-    Preconditions.checkNotNull(t);
-  }
-  private CallReturn(State s) {
-    this(null, null, s);
-  }
-  private CallReturn(Object r, Throwable t, State s) {
-    Preconditions.checkArgument(r == null || t == null);
-    returnValue = r;
-    thrown = t;
-    state = s;
-  }
-
-  State getState() {
-    return state;
-  }
-
-  Object getReturnValue() throws Throwable {
-    if (state == State.EXCEPTION) {
-      throw thrown;
-    }
-    Preconditions.checkState(state == State.RETURNED, "state == %s", state);
-    return returnValue;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
index f2b2c99..300d0c2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
@@ -42,83 +42,11 @@ import java.util.Map;
 public class RetryInvocationHandler<T> implements RpcInvocationHandler {
   public static final Log LOG = LogFactory.getLog(RetryInvocationHandler.class);
 
-  static class Call {
-    private final Method method;
-    private final Object[] args;
-    private final boolean isRpc;
-    private final int callId;
-    final Counters counters;
-
-    private final RetryPolicy retryPolicy;
-    private final RetryInvocationHandler<?> retryInvocationHandler;
-
-    Call(Method method, Object[] args, boolean isRpc, int callId,
-         Counters counters, RetryInvocationHandler<?> retryInvocationHandler) {
-      this.method = method;
-      this.args = args;
-      this.isRpc = isRpc;
-      this.callId = callId;
-      this.counters = counters;
-
-      this.retryPolicy = retryInvocationHandler.getRetryPolicy(method);
-      this.retryInvocationHandler = retryInvocationHandler;
-    }
-
-    /** Invoke the call once without retrying. */
-    synchronized CallReturn invokeOnce() {
-      try {
-        // The number of times this invocation handler has ever been failed over
-        // before this method invocation attempt. Used to prevent concurrent
-        // failed method invocations from triggering multiple failover attempts.
-        final long failoverCount = retryInvocationHandler.getFailoverCount();
-        try {
-          return invoke();
-        } catch (Exception e) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this, e);
-          }
-          if (Thread.currentThread().isInterrupted()) {
-            // If interrupted, do not retry.
-            throw e;
-          }
-          retryInvocationHandler.handleException(
-              method, retryPolicy, failoverCount, counters, e);
-          return CallReturn.RETRY;
-        }
-      } catch(Throwable t) {
-        return new CallReturn(t);
-      }
-    }
-
-    CallReturn invoke() throws Throwable {
-      return new CallReturn(invokeMethod());
-    }
-
-    Object invokeMethod() throws Throwable {
-      if (isRpc) {
-        Client.setCallIdAndRetryCount(callId, counters.retries);
-      }
-      return retryInvocationHandler.invokeMethod(method, args);
-    }
-
-    @Override
-    public String toString() {
-      return getClass().getSimpleName() + "#" + callId + ": "
-          + method.getDeclaringClass().getSimpleName() + "." + method.getName()
-          + "(" + (args == null || args.length == 0? "": Arrays.toString(args))
-          +  ")";
-    }
-  }
-
-  static class Counters {
+  private static class Counters {
     /** Counter for retries. */
     private int retries;
     /** Counter for method invocation has been failed over. */
     private int failovers;
-
-    boolean isZeros() {
-      return retries == 0 && failovers == 0;
-    }
   }
 
   private static class ProxyDescriptor<T> {
@@ -216,13 +144,11 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
 
   private final ProxyDescriptor<T> proxyDescriptor;
 
-  private volatile boolean hasSuccessfulCall = false;
-
+  private volatile boolean hasMadeASuccessfulCall = false;
+  
   private final RetryPolicy defaultPolicy;
   private final Map<String,RetryPolicy> methodNameToPolicyMap;
 
-  private final AsyncCallHandler asyncCallHandler = new AsyncCallHandler();
-
   protected RetryInvocationHandler(FailoverProxyProvider<T> proxyProvider,
       RetryPolicy retryPolicy) {
     this(proxyProvider, retryPolicy, Collections.<String, RetryPolicy>emptyMap());
@@ -241,35 +167,38 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
     return policy != null? policy: defaultPolicy;
   }
 
-  private long getFailoverCount() {
-    return proxyDescriptor.getFailoverCount();
-  }
-
-  private Call newCall(Method method, Object[] args, boolean isRpc, int callId,
-                       Counters counters) {
-    if (Client.isAsynchronousMode()) {
-      return asyncCallHandler.newAsyncCall(method, args, isRpc, callId,
-          counters, this);
-    } else {
-      return new Call(method, args, isRpc, callId, counters, this);
-    }
-  }
-
   @Override
   public Object invoke(Object proxy, Method method, Object[] args)
       throws Throwable {
     final boolean isRpc = isRpcInvocation(proxyDescriptor.getProxy());
     final int callId = isRpc? Client.nextCallId(): RpcConstants.INVALID_CALL_ID;
-    final Counters counters = new Counters();
+    return invoke(method, args, isRpc, callId, new Counters());
+  }
+
+  private Object invoke(final Method method, final Object[] args,
+      final boolean isRpc, final int callId, final Counters counters)
+      throws Throwable {
+    final RetryPolicy policy = getRetryPolicy(method);
 
-    final Call call = newCall(method, args, isRpc, callId, counters);
     while (true) {
-      final CallReturn c = call.invokeOnce();
-      final CallReturn.State state = c.getState();
-      if (state == CallReturn.State.ASYNC_INVOKED) {
-        return null; // return null for async calls
-      } else if (c.getState() != CallReturn.State.RETRY) {
-        return c.getReturnValue();
+      // The number of times this invocation handler has ever been failed over,
+      // before this method invocation attempt. Used to prevent concurrent
+      // failed method invocations from triggering multiple failover attempts.
+      final long failoverCount = proxyDescriptor.getFailoverCount();
+
+      if (isRpc) {
+        Client.setCallIdAndRetryCount(callId, counters.retries);
+      }
+      try {
+        final Object ret = invokeMethod(method, args);
+        hasMadeASuccessfulCall = true;
+        return ret;
+      } catch (Exception ex) {
+        if (Thread.currentThread().isInterrupted()) {
+          // If interrupted, do not retry.
+          throw ex;
+        }
+        handleException(method, policy, failoverCount, counters, ex);
       }
     }
   }
@@ -310,8 +239,7 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
       final int failovers, final long delay, final Exception ex) {
     // log info if this has made some successful calls or
     // this is not the first failover
-    final boolean info = hasSuccessfulCall || failovers != 0
-        || asyncCallHandler.hasSuccessfulCall();
+    final boolean info = hasMadeASuccessfulCall || failovers != 0;
     if (!info && !LOG.isDebugEnabled()) {
       return;
     }
@@ -337,9 +265,7 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
       if (!method.isAccessible()) {
         method.setAccessible(true);
       }
-      final Object r = method.invoke(proxyDescriptor.getProxy(), args);
-      hasSuccessfulCall = true;
-      return r;
+      return method.invoke(proxyDescriptor.getProxy(), args);
     } catch (InvocationTargetException e) {
       throw e.getCause();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
index c0a14b7..131aa8f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.io.retry;
 
-import java.io.EOFException;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.NoRouteToHostException;
@@ -648,9 +647,8 @@ public class RetryPolicies {
         return new RetryAction(RetryAction.RetryDecision.FAIL, 0, "retries ("
             + retries + ") exceeded maximum allowed (" + maxRetries + ")");
       }
-
+      
       if (e instanceof ConnectException ||
-          e instanceof EOFException ||
           e instanceof NoRouteToHostException ||
           e instanceof UnknownHostException ||
           e instanceof StandbyException ||

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index ed8d905..d1d5b17 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.concurrent.AsyncGet;
+import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.Tracer;
 
@@ -93,8 +94,8 @@ public class Client implements AutoCloseable {
 
   private static final ThreadLocal<Integer> callId = new ThreadLocal<Integer>();
   private static final ThreadLocal<Integer> retryCount = new ThreadLocal<Integer>();
-  private static final ThreadLocal<AsyncGet<? extends Writable, IOException>>
-      ASYNC_RPC_RESPONSE = new ThreadLocal<>();
+  private static final ThreadLocal<Future<?>> ASYNC_RPC_RESPONSE
+      = new ThreadLocal<>();
   private static final ThreadLocal<Boolean> asynchronousMode =
       new ThreadLocal<Boolean>() {
         @Override
@@ -105,9 +106,8 @@ public class Client implements AutoCloseable {
 
   @SuppressWarnings("unchecked")
   @Unstable
-  public static <T extends Writable> AsyncGet<T, IOException>
-      getAsyncRpcResponse() {
-    return (AsyncGet<T, IOException>) ASYNC_RPC_RESPONSE.get();
+  public static <T> Future<T> getAsyncRpcResponse() {
+    return (Future<T>) ASYNC_RPC_RESPONSE.get();
   }
 
   /** Set call id and retry count for the next call. */
@@ -1413,16 +1413,9 @@ public class Client implements AutoCloseable {
             }
           }
         }
-
-        @Override
-        public boolean isDone() {
-          synchronized (call) {
-            return call.done;
-          }
-        }
       };
 
-      ASYNC_RPC_RESPONSE.set(asyncGet);
+      ASYNC_RPC_RESPONSE.set(new AsyncGetFuture<>(asyncGet));
       return null;
     } else {
       return getRpcResponse(call, connection, -1, null);
@@ -1467,8 +1460,10 @@ public class Client implements AutoCloseable {
     synchronized (call) {
       while (!call.done) {
         try {
-          AsyncGet.Util.wait(call, timeout, unit);
-          if (timeout >= 0 && !call.done) {
+          final long waitTimeout = AsyncGet.Util.asyncGetTimeout2WaitTimeout(
+              timeout, unit);
+          call.wait(waitTimeout); // wait for the result
+          if (waitTimeout > 0 && !call.done) {
             return null;
           }
         } catch (InterruptedException ie) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 315ec67..0f43fc6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -54,6 +54,7 @@ import java.lang.reflect.Proxy;
 import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -255,18 +256,14 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
       
       if (Client.isAsynchronousMode()) {
-        final AsyncGet<RpcResponseWrapper, IOException> arr
-            = Client.getAsyncRpcResponse();
+        final Future<RpcResponseWrapper> frrw = Client.getAsyncRpcResponse();
         final AsyncGet<Message, Exception> asyncGet
             = new AsyncGet<Message, Exception>() {
           @Override
           public Message get(long timeout, TimeUnit unit) throws Exception {
-            return getReturnMessage(method, arr.get(timeout, unit));
-          }
-
-          @Override
-          public boolean isDone() {
-            return arr.isDone();
+            final RpcResponseWrapper rrw = timeout < 0?
+                frrw.get(): frrw.get(timeout, unit);
+            return getReturnMessage(method, rrw);
           }
         };
         ASYNC_RETURN_MESSAGE.set(asyncGet);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
index f124890..5eac869 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java
@@ -47,19 +47,14 @@ public interface AsyncGet<R, E extends Throwable> {
   R get(long timeout, TimeUnit unit)
       throws E, TimeoutException, InterruptedException;
 
-  /** @return true if the underlying computation is done; false, otherwise. */
-  boolean isDone();
-
   /** Utility */
   class Util {
-    /** Use {@link #get(long, TimeUnit)} timeout parameters to wait. */
-    public static void wait(Object obj, long timeout, TimeUnit unit)
-        throws InterruptedException {
-      if (timeout < 0) {
-        obj.wait();
-      } else if (timeout > 0) {
-        obj.wait(unit.toMillis(timeout));
-      }
+    /**
+     * @return {@link Object#wait(long)} timeout converted
+     *         from {@link #get(long, TimeUnit)} timeout.
+     */
+    public static long asyncGetTimeout2WaitTimeout(long timeout, TimeUnit unit){
+      return timeout < 0? 0: timeout == 0? 1:unit.toMillis(timeout);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
index 4450c0c..0ad191b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.ipc.TestIPC.TestServer;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -51,11 +50,6 @@ public class TestAsyncIPC {
   private static Configuration conf;
   private static final Log LOG = LogFactory.getLog(TestAsyncIPC.class);
 
-  static <T extends Writable> AsyncGetFuture<T, IOException>
-      getAsyncRpcResponseFuture() {
-    return new AsyncGetFuture<>(Client.getAsyncRpcResponse());
-  }
-
   @Before
   public void setupConf() {
     conf = new Configuration();
@@ -90,7 +84,7 @@ public class TestAsyncIPC {
         try {
           final long param = TestIPC.RANDOM.nextLong();
           TestIPC.call(client, param, server, conf);
-          returnFutures.put(i, getAsyncRpcResponseFuture());
+          returnFutures.put(i, Client.getAsyncRpcResponse());
           expectedValues.put(i, param);
         } catch (Exception e) {
           failed = true;
@@ -210,7 +204,7 @@ public class TestAsyncIPC {
 
     private void doCall(final int idx, final long param) throws IOException {
       TestIPC.call(client, param, server, conf);
-      returnFutures.put(idx, getAsyncRpcResponseFuture());
+      returnFutures.put(idx, Client.getAsyncRpcResponse());
       expectedValues.put(idx, param);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/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 824336a..29bac2a 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
@@ -29,7 +29,7 @@ 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.io.retry.AsyncCallHandler;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
 import org.apache.hadoop.util.concurrent.AsyncGetFuture;
 import org.apache.hadoop.ipc.Client;
 
@@ -51,8 +51,9 @@ public class AsyncDistributedFileSystem {
     this.dfs = dfs;
   }
 
-  private static <T> Future<T> getReturnValue() {
-    return new AsyncGetFuture<>(AsyncCallHandler.getAsyncReturn());
+  static <T> Future<T> getReturnValue() {
+    return new AsyncGetFuture<>(
+        ClientNamenodeProtocolTranslatorPB.getAsyncReturnValue());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/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 bcf5269..2373da7 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
@@ -29,6 +29,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
@@ -183,7 +184,6 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.AsyncCallHandler;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -212,6 +212,8 @@ import org.apache.hadoop.util.concurrent.AsyncGet;
 public class ClientNamenodeProtocolTranslatorPB implements
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
+  private static final ThreadLocal<AsyncGet<?, Exception>>
+      ASYNC_RETURN_VALUE = new ThreadLocal<>();
 
   static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
       GetServerDefaultsRequestProto.newBuilder().build();
@@ -245,6 +247,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
     rpcProxy = proxy;
   }
 
+  @SuppressWarnings("unchecked")
+  @Unstable
+  public static <T> AsyncGet<T, Exception> getAsyncReturnValue() {
+    return (AsyncGet<T, Exception>) ASYNC_RETURN_VALUE.get();
+  }
+
   @Override
   public void close() {
     RPC.stopProxy(rpcProxy);
@@ -383,13 +391,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
         asyncReturnMessage.get(timeout, unit);
         return null;
       }
-
-      @Override
-      public boolean isDone() {
-        return asyncReturnMessage.isDone();
-      }
     };
-    AsyncCallHandler.setLowerLayerAsyncReturn(asyncGet);
+    ASYNC_RETURN_VALUE.set(asyncGet);
   }
 
   @Override
@@ -1364,20 +1367,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
         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));
-          }
-
-          @Override
-          public boolean isDone() {
-            return asyncReturnMessage.isDone();
-          }
-        };
-        AsyncCallHandler.setLowerLayerAsyncReturn(asyncGet);
+        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));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/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
index 6a60290..c7615a9 100644
--- 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
@@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.TestDFSPermission.PermissionGenerator;
 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.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
@@ -71,7 +70,7 @@ public class TestAsyncDFS {
   public static final Log LOG = LogFactory.getLog(TestAsyncDFS.class);
   private final short replFactor = 1;
   private final long blockSize = 512;
-  private long fileLen = 0;
+  private long fileLen = blockSize * 3;
   private final long seed = Time.now();
   private final Random r = new Random(seed);
   private final PermissionGenerator permGenerator = new PermissionGenerator(r);
@@ -81,7 +80,7 @@ public class TestAsyncDFS {
 
   private Configuration conf;
   private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
+  private FileSystem fs;
   private AsyncDistributedFileSystem adfs;
 
   @Before
@@ -96,10 +95,10 @@ public class TestAsyncDFS {
         ASYNC_CALL_LIMIT);
     // set server handlers
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY, NUM_NN_HANDLER);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
-    fs = cluster.getFileSystem();
-    adfs = fs.getAsyncDistributedFileSystem();
+    fs = FileSystem.get(conf);
+    adfs = cluster.getFileSystem().getAsyncDistributedFileSystem();
   }
 
   @After
@@ -114,6 +113,31 @@ public class TestAsyncDFS {
     }
   }
 
+  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";
@@ -324,7 +348,7 @@ public class TestAsyncDFS {
 
   public static void checkPermissionDenied(final Exception e, final Path dir,
       final String user) {
-    assertTrue(e.getCause() instanceof RemoteException);
+    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
@@ -446,9 +470,4 @@ public class TestAsyncDFS {
       assertTrue("group2".equals(fs.getFileStatus(dsts[i]).getGroup()));
     }
   }
-
-  @Test
-  public void testAsyncWithoutRetry() throws Exception {
-    TestAsyncHDFSWithHA.runTestAsyncWithoutRetry(conf, cluster, fs);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
deleted file mode 100644
index 9ade8ec..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAsyncHDFSWithHA.java
+++ /dev/null
@@ -1,181 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-import org.apache.hadoop.io.retry.AsyncCallHandler;
-import org.apache.hadoop.io.retry.RetryInvocationHandler;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.concurrent.AsyncGetFuture;
-import org.apache.log4j.Level;
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-/** Test async methods with HA setup. */
-public class TestAsyncHDFSWithHA {
-  static final Logger LOG = LoggerFactory.getLogger(TestAsyncHDFSWithHA.class);
-  static {
-    GenericTestUtils.setLogLevel(RetryInvocationHandler.LOG, Level.ALL);
-  }
-
-  private static <T> Future<T> getReturnValue() {
-    return new AsyncGetFuture<>(AsyncCallHandler.getAsyncReturn());
-  }
-
-  static void mkdirs(DistributedFileSystem dfs, String dir, Path[] srcs,
-                     Path[] dsts) throws IOException {
-    for (int i = 0; i < srcs.length; i++) {
-      srcs[i] = new Path(dir, "src" + i);
-      dsts[i] = new Path(dir, "dst" + i);
-      dfs.mkdirs(srcs[i]);
-    }
-  }
-
-  static void runTestAsyncWithoutRetry(Configuration conf,
-      MiniDFSCluster cluster, DistributedFileSystem dfs) throws Exception {
-    final int num = 5;
-
-    final String renameDir = "/testAsyncWithoutRetry/";
-    final Path[] srcs = new Path[num + 1];
-    final Path[] dsts = new Path[num + 1];
-    mkdirs(dfs, renameDir, srcs, dsts);
-
-    // create a proxy without retry.
-    final NameNodeProxiesClient.ProxyAndInfo<ClientProtocol> proxyInfo
-        = NameNodeProxies.createNonHAProxy(conf,
-        cluster.getNameNode(0).getNameNodeAddress(),
-        ClientProtocol.class, UserGroupInformation.getCurrentUser(),
-        false);
-    final ClientProtocol cp = proxyInfo.getProxy();
-
-    // submit async calls
-    Client.setAsynchronousMode(true);
-    final List<Future<Void>> results = new ArrayList<>();
-    for (int i = 0; i < num; i++) {
-      final String src = srcs[i].toString();
-      final String dst = dsts[i].toString();
-      LOG.info(i + ") rename " + src + " -> " + dst);
-      cp.rename2(src, dst);
-      results.add(getReturnValue());
-    }
-    Client.setAsynchronousMode(false);
-
-    // wait for the async calls
-    for (Future<Void> f : results) {
-      f.get();
-    }
-
-    //check results
-    for (int i = 0; i < num; i++) {
-      Assert.assertEquals(false, dfs.exists(srcs[i]));
-      Assert.assertEquals(true, dfs.exists(dsts[i]));
-    }
-  }
-
-  /** Testing HDFS async methods with HA setup. */
-  @Test(timeout = 120000)
-  public void testAsyncWithHAFailover() throws Exception {
-    final int num = 10;
-
-    final Configuration conf = new HdfsConfiguration();
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology())
-        .numDataNodes(0).build();
-
-    try {
-      cluster.waitActive();
-      cluster.transitionToActive(0);
-
-      final DistributedFileSystem dfs = HATestUtil.configureFailoverFs(
-          cluster, conf);
-      runTestAsyncWithoutRetry(conf, cluster, dfs);
-
-      final String renameDir = "/testAsyncWithHAFailover/";
-      final Path[] srcs = new Path[num + 1];
-      final Path[] dsts = new Path[num + 1];
-      mkdirs(dfs, renameDir, srcs, dsts);
-
-      // submit async calls and trigger failover in the middle.
-      final AsyncDistributedFileSystem adfs
-          = dfs.getAsyncDistributedFileSystem();
-      final ExecutorService executor = Executors.newFixedThreadPool(num + 1);
-
-      final List<Future<Void>> results = new ArrayList<>();
-      final List<IOException> exceptions = new ArrayList<>();
-      final List<Future<?>> futures = new ArrayList<>();
-      final int half = num/2;
-      for(int i = 0; i <= num; i++) {
-        final int id = i;
-        futures.add(executor.submit(new Runnable() {
-          @Override
-          public void run() {
-            try {
-              if (id == half) {
-                // failover
-                cluster.shutdownNameNode(0);
-                cluster.transitionToActive(1);
-              } else {
-                // rename
-                results.add(adfs.rename(srcs[id], dsts[id]));
-              }
-            } catch (IOException e) {
-              exceptions.add(e);
-            }
-          }
-        }));
-      }
-
-      // wait for the tasks
-      Assert.assertEquals(num + 1, futures.size());
-      for(int i = 0; i <= num; i++) {
-        futures.get(i).get();
-      }
-      // wait for the async calls
-      Assert.assertEquals(num, results.size());
-      Assert.assertTrue(exceptions.isEmpty());
-      for(Future<Void> r : results) {
-        r.get();
-      }
-
-      // check results
-      for(int i = 0; i <= num; i++) {
-        final boolean renamed = i != half;
-        Assert.assertEquals(!renamed, dfs.exists(srcs[i]));
-        Assert.assertEquals(renamed, dfs.exists(dsts[i]));
-      }
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5360da8b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
index 169bbee..42cf3d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -136,8 +135,7 @@ public abstract class HATestUtil {
   }
   
   /** Gets the filesystem instance by setting the failover configurations */
-  public static DistributedFileSystem configureFailoverFs(
-      MiniDFSCluster cluster, Configuration conf)
+  public static FileSystem configureFailoverFs(MiniDFSCluster cluster, Configuration conf)
       throws IOException, URISyntaxException {
     return configureFailoverFs(cluster, conf, 0);
   }
@@ -149,14 +147,13 @@ public abstract class HATestUtil {
    * @param nsIndex namespace index starting with zero
    * @throws IOException if an error occurs rolling the edit log
    */
-  public static DistributedFileSystem configureFailoverFs(
-      MiniDFSCluster cluster, Configuration conf,
+  public static FileSystem configureFailoverFs(MiniDFSCluster cluster, Configuration conf,
       int nsIndex) throws IOException, URISyntaxException {
     conf = new Configuration(conf);
     String logicalName = getLogicalHostname(cluster);
     setFailoverConfigurations(cluster, conf, logicalName, nsIndex);
     FileSystem fs = FileSystem.get(new URI("hdfs://" + logicalName), conf);
-    return (DistributedFileSystem)fs;
+    return fs;
   }
   
   public static void setFailoverConfigurations(MiniDFSCluster cluster,


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


[40/47] hadoop git commit: HADOOP-12807 S3AFileSystem should read AWS credentials from environment variables. Contributed by Tobin Baker.

Posted by ae...@apache.org.
HADOOP-12807 S3AFileSystem should read AWS credentials from environment variables. Contributed by Tobin Baker.


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

Branch: refs/heads/HDFS-1312
Commit: a3f78d8fa83f07f9183f3546203a191fcf50008c
Parents: 4a1cedc
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jun 6 23:40:49 2016 +0200
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jun 6 23:42:36 2016 +0200

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java  |  2 ++
 .../src/site/markdown/tools/hadoop-aws/index.md  | 19 +++++++++++++++++++
 2 files changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3f78d8f/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index c028544..0281a3a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
+import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.ClientConfiguration;
@@ -464,6 +465,7 @@ public class S3AFileSystem extends FileSystem {
           new BasicAWSCredentialsProvider(
               creds.getAccessKey(), creds.getAccessSecret()),
           new InstanceProfileCredentialsProvider(),
+          new EnvironmentVariableCredentialsProvider(),
           new AnonymousAWSCredentialsProvider()
       );
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3f78d8f/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 7a5e455..7d63a86 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -202,6 +202,25 @@ credentials in S3AFileSystem.
 For additional reading on the credential provider API see:
 [Credential Provider API](../../../hadoop-project-dist/hadoop-common/CredentialProviderAPI.html).
 
+#### Authenticating via environment variables
+
+S3A supports configuration via [the standard AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment).
+
+The core environment variables are for the access key and associated secret:
+
+```
+export AWS_ACCESS_KEY_ID=my.aws.key
+export AWS_SECRET_ACCESS_KEY=my.secret.key
+```
+
+These environment variables can be used to set the authentication credentials
+instead of properties in the Hadoop configuration. *Important:* these
+environment variables are not propagated from client to server when
+YARN applications are launched. That is: having the AWS environment variables
+set when an application is launched will not permit the launched application
+to access S3 resources. The environment variables must (somehow) be set
+on the hosts/processes where the work is executed.
+
 ##### End to End Steps for Distcp and S3 with Credential Providers
 
 ###### provision


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