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/10 03:35:12 UTC

[01/50] [abbrv] 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)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 54f7975df -> 23923086d


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-7240
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


[14/50] [abbrv] 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-7240
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


[46/50] [abbrv] hadoop git commit: HADOOP-13175. Remove hadoop-ant from hadoop-tools. Contributed by Chris Douglas.

Posted by ae...@apache.org.
HADOOP-13175. Remove hadoop-ant from hadoop-tools. Contributed by Chris Douglas.


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

Branch: refs/heads/HDFS-7240
Commit: 58c3486850c0503aecdeae8b67bb7e6bc42b4da8
Parents: 31ffaf7
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 9 13:49:52 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 9 13:49:52 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |   5 -
 hadoop-tools/hadoop-ant/pom.xml                 |  56 -----
 .../java/org/apache/hadoop/ant/DfsTask.java     | 220 -------------------
 .../ant/condition/DfsBaseConditional.java       |  68 ------
 .../apache/hadoop/ant/condition/DfsExists.java  |  24 --
 .../apache/hadoop/ant/condition/DfsIsDir.java   |  24 --
 .../apache/hadoop/ant/condition/DfsZeroLen.java |  24 --
 .../resources/org/apache/hadoop/ant/antlib.xml  |  29 ---
 hadoop-tools/hadoop-tools-dist/pom.xml          |   6 -
 hadoop-tools/pom.xml                            |   1 -
 10 files changed, 457 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 4c618a1..35166b1 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -375,11 +375,6 @@
         <artifactId>hadoop-extras</artifactId>
         <version>${project.version}</version>
       </dependency>
-      <dependency>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-ant</artifactId>
-        <version>${project.version}</version>
-      </dependency>
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/pom.xml b/hadoop-tools/hadoop-ant/pom.xml
deleted file mode 100644
index e0b038e..0000000
--- a/hadoop-tools/hadoop-ant/pom.xml
+++ /dev/null
@@ -1,56 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
-                      http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-alpha1-SNAPSHOT</version>
-    <relativePath>../../hadoop-project</relativePath>
-  </parent>
-  <groupId>org.apache.hadoop</groupId>
-  <artifactId>hadoop-ant</artifactId>
-  <version>3.0.0-alpha1-SNAPSHOT</version>
-  <description>Apache Hadoop Ant Tasks</description>
-  <name>Apache Hadoop Ant Tasks</name>
-  <packaging>jar</packaging>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.ant</groupId>
-      <artifactId>ant</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-annotations</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs-client</artifactId>
-      <scope>provided</scope>
-    </dependency>
-  </dependencies>
-
-</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/DfsTask.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/DfsTask.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/DfsTask.java
deleted file mode 100644
index 36119f5..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/DfsTask.java
+++ /dev/null
@@ -1,220 +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.ant;
-
-import java.io.ByteArrayOutputStream;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.io.UnsupportedEncodingException;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
-import java.util.LinkedList;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.tools.ant.AntClassLoader;
-import org.apache.tools.ant.BuildException;
-import org.apache.tools.ant.Task;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * {@link org.apache.hadoop.fs.FsShell FsShell} wrapper for ant Task.
- */
-@InterfaceAudience.Private
-public class DfsTask extends Task {
-
-  static {
-    // adds the default resources
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-  }
-
-  /**
-   * Default sink for {@link java.lang.System#out}
-   * and {@link java.lang.System#err}.
-   */
-  private static final OutputStream nullOut = new OutputStream() {
-      public void write(int b)    { /* ignore */ }
-      public String toString()    { return ""; }
-  };
-  private static final FsShell shell = new FsShell();
-
-  protected AntClassLoader confloader;
-  protected OutputStream out = nullOut;
-  protected OutputStream err = nullOut;
-
-  // set by ant
-  protected String cmd;
-  protected final LinkedList<String> argv = new LinkedList<String>();
-  protected String outprop;
-  protected String errprop;
-  protected boolean failonerror = true;
-
-  // saved ant context
-  private PrintStream antOut;
-  private PrintStream antErr;
-
-  /**
-   * Sets the command to run in {@link org.apache.hadoop.fs.FsShell FsShell}.
-   * @param cmd A valid command to FsShell, sans &quot;-&quot;.
-   */
-  public void setCmd(String cmd) {
-    this.cmd = "-" + cmd.trim();
-  }
-
-  /**
-   * Sets the argument list from a String of comma-separated values.
-   * @param args A String of comma-separated arguments to FsShell.
-   */
-  public void setArgs(String args) {
-    for (String s : args.trim().split("\\s*,\\s*"))
-      argv.add(s);
-  }
-
-  /**
-   * Sets the property into which System.out will be written.
-   * @param outprop The name of the property into which System.out is written.
-   * If the property is defined before this task is executed, it will not be updated.
-   */
-  public void setOut(String outprop) {
-    this.outprop = outprop;
-    out = new ByteArrayOutputStream();
-    if (outprop.equals(errprop))
-      err = out;
-  }
-
-  /**
-   * Sets the property into which System.err will be written. If this property
-   * has the same name as the property for System.out, the two will be interlaced.
-   * @param errprop The name of the property into which System.err is written.
-   * If the property is defined before this task is executed, it will not be updated.
-   */
-  public void setErr(String errprop) {
-    this.errprop = errprop;
-    err = (errprop.equals(outprop)) ? err = out : new ByteArrayOutputStream();
-  }
-
-  /**
-   * Sets the path for the parent-last ClassLoader, intended to be used for
-   * {@link org.apache.hadoop.conf.Configuration Configuration}.
-   * @param confpath The path to search for resources, classes, etc. before
-   * parent ClassLoaders.
-   */
-  public void setConf(String confpath) {
-    confloader = AccessController.doPrivileged(
-        new PrivilegedAction<AntClassLoader>() {
-          @Override
-          public AntClassLoader run() {
-            return new AntClassLoader(getClass().getClassLoader(), false);
-          }
-        });
-    confloader.setProject(getProject());
-    if (null != confpath)
-      confloader.addPathElement(confpath);
-  }
-
-  /**
-   * Sets a property controlling whether or not a
-   * {@link org.apache.tools.ant.BuildException BuildException} will be thrown
-   * if the command returns a value less than zero or throws an exception.
-   * @param failonerror If true, throw a BuildException on error.
-   */
-  public void setFailonerror(boolean failonerror) {
-    this.failonerror = failonerror;
-  }
-
-  /**
-   * Save the current values of System.out, System.err and configure output
-   * streams for FsShell.
-   */
-  protected void pushContext() {
-    antOut = System.out;
-    antErr = System.err;
-    try {
-      System.setOut(new PrintStream(out, false, "UTF-8"));
-      System.setErr(out == err ?
-          System.out : new PrintStream(err, false, "UTF-8"));
-    } catch (UnsupportedEncodingException ignored) {
-    }
-  }
-
-  /**
-   * Create the appropriate output properties with their respective output,
-   * restore System.out, System.err and release any resources from created
-   * ClassLoaders to aid garbage collection.
-   */
-  protected void popContext() {
-    // write output to property, if applicable
-    if (outprop != null && !System.out.checkError())
-      getProject().setNewProperty(outprop, out.toString());
-    if (out != err && errprop != null && !System.err.checkError())
-      getProject().setNewProperty(errprop, err.toString());
-
-    System.setErr(antErr);
-    System.setOut(antOut);
-    confloader.cleanup();
-    confloader.setParent(null);
-  }
-
-  // in case DfsTask is overridden
-  protected int postCmd(int exit_code) {
-    if ("-test".equals(cmd) && exit_code != 0)
-      outprop = null;
-    return exit_code;
-  }
-
-  /**
-   * Invoke {@link org.apache.hadoop.fs.FsShell#main} after a
-   * few cursory checks of the configuration.
-   */
-  public void execute() throws BuildException {
-    if (null == cmd)
-      throw new BuildException("Missing command (cmd) argument");
-    argv.add(0, cmd);
-
-    if (null == confloader) {
-      setConf(getProject().getProperty("hadoop.conf.dir"));
-    }
-
-    int exit_code = 0;
-    try {
-      pushContext();
-
-      Configuration conf = new Configuration();
-      conf.setClassLoader(confloader);
-      exit_code = ToolRunner.run(conf, shell,
-          argv.toArray(new String[argv.size()]));
-      exit_code = postCmd(exit_code);
-
-      if (0 > exit_code) {
-        StringBuilder msg = new StringBuilder();
-        for (String s : argv)
-          msg.append(s + " ");
-        msg.append("failed: " + exit_code);
-        throw new Exception(msg.toString());
-      }
-    } catch (Exception e) {
-      if (failonerror)
-          throw new BuildException(e);
-    } finally {
-      popContext();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsBaseConditional.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsBaseConditional.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsBaseConditional.java
deleted file mode 100644
index 33cf52b..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsBaseConditional.java
+++ /dev/null
@@ -1,68 +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.ant.condition;
-
-import org.apache.tools.ant.taskdefs.condition.Condition;
-
-/**
- * This wrapper around {@link org.apache.hadoop.ant.DfsTask} implements the
- * Ant &gt;1.5
- * {@link org.apache.tools.ant.taskdefs.condition.Condition Condition}
- * interface for HDFS tests. So one can test conditions like this:
- * {@code
- *   <condition property="precond">
- *     <and>
- *       <hadoop:exists file="fileA" />
- *       <hadoop:exists file="fileB" />
- *       <hadoop:sizezero file="fileB" />
- *     </and>
- *   </condition>
- * }
- * This will define the property precond if fileA exists and fileB has zero
- * length.
- */
-public abstract class DfsBaseConditional extends org.apache.hadoop.ant.DfsTask
-                       implements Condition {
-
-  protected boolean result;
-  String file;
-
-  private void initArgs() {
-    setCmd("test");
-    setArgs("-"  +  getFlag() + "," + file);
-  }
-
-  public void setFile(String file) {
-    this.file = file;
-  }
-
-  protected abstract char getFlag();
-
-  protected int postCmd(int exit_code) {
-    exit_code = super.postCmd(exit_code);
-    result = exit_code == 0;
-    return exit_code;
-  }
-
-  public boolean eval() {
-    initArgs();
-    execute();
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsExists.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsExists.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsExists.java
deleted file mode 100644
index 95e3868..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsExists.java
+++ /dev/null
@@ -1,24 +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.ant.condition;
-
-public class DfsExists extends DfsBaseConditional {
-  protected static final char flag = 'e';
-  protected char getFlag() { return flag; }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsIsDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsIsDir.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsIsDir.java
deleted file mode 100644
index b889615..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsIsDir.java
+++ /dev/null
@@ -1,24 +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.ant.condition;
-
-public class DfsIsDir extends DfsBaseConditional {
-  protected static final char flag = 'd';
-  protected char getFlag() { return flag; }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsZeroLen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsZeroLen.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsZeroLen.java
deleted file mode 100644
index 70b4fbf..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsZeroLen.java
+++ /dev/null
@@ -1,24 +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.ant.condition;
-
-public class DfsZeroLen extends DfsBaseConditional {
-  protected static final char flag = 'z';
-  protected char getFlag() { return flag; }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/resources/org/apache/hadoop/ant/antlib.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/resources/org/apache/hadoop/ant/antlib.xml b/hadoop-tools/hadoop-ant/src/main/resources/org/apache/hadoop/ant/antlib.xml
deleted file mode 100644
index 968b899..0000000
--- a/hadoop-tools/hadoop-ant/src/main/resources/org/apache/hadoop/ant/antlib.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.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.
--->
-
-<antlib>
-  <taskdef name="hdfs"
-           classname="org.apache.hadoop.ant.DfsTask" />
-  <taskdef name="exists"
-           classname="org.apache.hadoop.ant.condition.DfsExists" />
-  <taskdef name="isdir"
-           classname="org.apache.hadoop.ant.condition.DfsIsDir" />
-  <taskdef name="sizezero"
-           classname="org.apache.hadoop.ant.condition.DfsZeroLen" />
-</antlib>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-tools-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml
index ef469fa..e1fbef1 100644
--- a/hadoop-tools/hadoop-tools-dist/pom.xml
+++ b/hadoop-tools/hadoop-tools-dist/pom.xml
@@ -105,12 +105,6 @@
       <artifactId>hadoop-sls</artifactId>
       <scope>compile</scope>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-ant</artifactId>
-      <version>${project.version}</version>
-      <scope>compile</scope>
-    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index bb6d893..bd5f784 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -38,7 +38,6 @@
     <module>hadoop-rumen</module>
     <module>hadoop-gridmix</module>
     <module>hadoop-datajoin</module>
-    <module>hadoop-ant</module>
     <module>hadoop-tools-dist</module>
     <module>hadoop-extras</module>
     <module>hadoop-pipes</module>


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


[07/50] [abbrv] 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-7240
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


[05/50] [abbrv] 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-7240
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


[43/50] [abbrv] hadoop git commit: HADOOP-13237: s3a initialization against public bucket fails if caller lacks any credentials. Contributed by Chris Nauroth

Posted by ae...@apache.org.
HADOOP-13237: s3a initialization against public bucket fails if caller lacks any credentials. Contributed by Chris Nauroth


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

Branch: refs/heads/HDFS-7240
Commit: 656c460c0e79ee144d6ef48d85cec04a1af3b2cc
Parents: 8ea9bbc
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jun 9 16:36:27 2016 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jun 9 17:28:49 2016 +0100

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         | 13 ++++-
 .../fs/s3a/AnonymousAWSCredentialsProvider.java | 11 ++++
 .../fs/s3a/BasicAWSCredentialsProvider.java     |  8 +++
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 22 +++++---
 .../src/site/markdown/tools/hadoop-aws/index.md | 14 ++++-
 .../fs/s3a/TestS3AAWSCredentialsProvider.java   | 55 ++++++++++++++++++++
 6 files changed, 113 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/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 a65246b..8bb27ea 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
@@ -791,7 +791,18 @@
 
 <property>
   <name>fs.s3a.aws.credentials.provider</name>
-  <description>Class name of a credentials provider that implements com.amazonaws.auth.AWSCredentialsProvider. Omit if using access/secret keys or another authentication mechanism.</description>
+  <description>
+    Class name of a credentials provider that implements
+    com.amazonaws.auth.AWSCredentialsProvider.  Omit if using access/secret keys
+    or another authentication mechanism.  The specified class must provide an
+    accessible constructor accepting java.net.URI and
+    org.apache.hadoop.conf.Configuration, or an accessible default constructor.
+    Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
+    anonymous access to a publicly accessible S3 bucket without any credentials.
+    Please note that allowing anonymous access to an S3 bucket compromises
+    security and therefore is unsuitable for most use cases.  It can be useful
+    for accessing public data sets without requiring AWS credentials.
+  </description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java
index e62ec77..2c863fc 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java
@@ -24,6 +24,17 @@ import com.amazonaws.auth.AWSCredentials;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+/**
+ * AnonymousAWSCredentialsProvider supports anonymous access to AWS services
+ * through the AWS SDK.  AWS requests will not be signed.  This is not suitable
+ * for most cases, because allowing anonymous access to an S3 bucket compromises
+ * security.  This can be useful for accessing public data sets without
+ * requiring AWS credentials.
+ *
+ * Please note that users may reference this class name from configuration
+ * property fs.s3a.aws.credentials.provider.  Therefore, changing the class name
+ * would be a backward-incompatible change.
+ */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class AnonymousAWSCredentialsProvider implements AWSCredentialsProvider {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
index 2f721e4..3a5ee8c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
@@ -26,6 +26,14 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+/**
+ * BasicAWSCredentialsProvider supports static configuration of access key ID
+ * and secret access key for use with the AWS SDK.
+ *
+ * Please note that users may reference this class name from configuration
+ * property fs.s3a.aws.credentials.provider.  Therefore, changing the class name
+ * would be a backward-incompatible change.
+ */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class BasicAWSCredentialsProvider implements AWSCredentialsProvider {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/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 0281a3a..9af0a99 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
@@ -465,20 +465,28 @@ public class S3AFileSystem extends FileSystem {
           new BasicAWSCredentialsProvider(
               creds.getAccessKey(), creds.getAccessSecret()),
           new InstanceProfileCredentialsProvider(),
-          new EnvironmentVariableCredentialsProvider(),
-          new AnonymousAWSCredentialsProvider()
-      );
+          new EnvironmentVariableCredentialsProvider());
 
     } else {
       try {
         LOG.debug("Credential provider class is {}", className);
-        credentials = (AWSCredentialsProvider) Class.forName(className)
-            .getDeclaredConstructor(URI.class, Configuration.class)
-            .newInstance(this.uri, conf);
+        Class<?> credClass = Class.forName(className);
+        try {
+          credentials =
+              (AWSCredentialsProvider)credClass.getDeclaredConstructor(
+                  URI.class, Configuration.class).newInstance(this.uri, conf);
+        } catch (NoSuchMethodException | SecurityException e) {
+          credentials =
+              (AWSCredentialsProvider)credClass.getDeclaredConstructor()
+                  .newInstance();
+        }
       } catch (ClassNotFoundException e) {
         throw new IOException(className + " not found.", e);
       } catch (NoSuchMethodException | SecurityException e) {
-        throw new IOException(className + " constructor exception.", e);
+        throw new IOException(String.format("%s constructor exception.  A "
+            + "class specified in %s must provide an accessible constructor "
+            + "accepting URI and Configuration, or an accessible default "
+            + "constructor.", className, AWS_CREDENTIALS_PROVIDER), e);
       } catch (ReflectiveOperationException | IllegalArgumentException e) {
         throw new IOException(className + " instantiation exception.", e);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/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 7d63a86..4086bc0 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
@@ -187,8 +187,18 @@ If you do any of these: change your credentials immediately!
 
     <property>
       <name>fs.s3a.aws.credentials.provider</name>
-      <description>Class name of a credentials provider that implements com.amazonaws.auth.AWSCredentialsProvider.
-      Omit if using access/secret keys or another authentication mechanism.</description>
+      <description>
+        Class name of a credentials provider that implements
+        com.amazonaws.auth.AWSCredentialsProvider.  Omit if using access/secret keys
+        or another authentication mechanism.  The specified class must provide an
+        accessible constructor accepting java.net.URI and
+        org.apache.hadoop.conf.Configuration, or an accessible default constructor.
+        Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
+        anonymous access to a publicly accessible S3 bucket without any credentials.
+        Please note that allowing anonymous access to an S3 bucket compromises
+        security and therefore is unsuitable for most use cases.  It can be useful
+        for accessing public data sets without requiring AWS credentials.
+      </description>
     </property>
 
 #### Protecting the AWS Credentials in S3A

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
index 1a11a45..a25ca9c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
@@ -26,8 +27,13 @@ import java.net.URI;
 import java.nio.file.AccessDeniedException;
 
 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.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.Timeout;
 
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
@@ -45,6 +51,12 @@ public class TestS3AAWSCredentialsProvider {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestS3AAWSCredentialsProvider.class);
 
+  @Rule
+  public Timeout testTimeout = new Timeout(1 * 60 * 1000);
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
   @Test
   public void testBadConfiguration() throws IOException {
     Configuration conf = new Configuration();
@@ -113,4 +125,47 @@ public class TestS3AAWSCredentialsProvider {
     conf.set(AWS_CREDENTIALS_PROVIDER, GoodCredentialsProvider.class.getName());
     S3ATestUtils.createTestFileSystem(conf);
   }
+
+  @Test
+  public void testAnonymousProvider() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(AWS_CREDENTIALS_PROVIDER,
+        AnonymousAWSCredentialsProvider.class.getName());
+    Path testFile = new Path(
+        conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+    FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf);
+    assertNotNull(fs);
+    assertTrue(fs instanceof S3AFileSystem);
+    FileStatus stat = fs.getFileStatus(testFile);
+    assertNotNull(stat);
+    assertEquals(testFile, stat.getPath());
+  }
+
+  static class ConstructorErrorProvider implements AWSCredentialsProvider {
+
+    @SuppressWarnings("unused")
+    public ConstructorErrorProvider(String str) {
+    }
+
+    @Override
+    public AWSCredentials getCredentials() {
+      return null;
+    }
+
+    @Override
+    public void refresh() {
+    }
+  }
+
+  @Test
+  public void testProviderConstructorError() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(AWS_CREDENTIALS_PROVIDER,
+        ConstructorErrorProvider.class.getName());
+    Path testFile = new Path(
+        conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+    exception.expect(IOException.class);
+    exception.expectMessage("constructor exception");
+    FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf);
+  }
 }


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


[09/50] [abbrv] 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-7240
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


[25/50] [abbrv] 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-7240
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


[35/50] [abbrv] hadoop git commit: YARN-5204. Properly report status of killed/stopped queued containers. (Konstantinos Karanasos via asuresh)

Posted by ae...@apache.org.
YARN-5204. Properly report status of killed/stopped queued containers. (Konstantinos Karanasos via asuresh)


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

Branch: refs/heads/HDFS-7240
Commit: 3344ba70e027c929e07bad5e6877c796d41181e9
Parents: 8c8a377
Author: Arun Suresh <as...@apache.org>
Authored: Wed Jun 8 08:31:32 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Wed Jun 8 08:31:32 2016 -0700

----------------------------------------------------------------------
 .../queuing/QueuingContainerManagerImpl.java    |  15 ++-
 .../queuing/TestQueuingContainerManager.java    | 129 +++++++++++++++----
 2 files changed, 115 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3344ba70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.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/queuing/QueuingContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
index a1e3bdb..38b1b07 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
@@ -175,8 +175,9 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
       }
 
       nodeStatusUpdater.sendOutofBandHeartBeat();
+    } else {
+      super.stopContainerInternal(containerID);
     }
-    super.stopContainerInternal(containerID);
   }
 
   /**
@@ -456,6 +457,18 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
             ContainerExitStatus.INVALID, this.context.getQueuingContext()
                 .getQueuedContainers().get(containerID).getResource(),
             executionType);
+      } else {
+        // Check if part of the stopped/killed queued containers.
+        for (ContainerTokenIdentifier cTokenId : this.context
+            .getQueuingContext().getKilledQueuedContainers().keySet()) {
+          if (cTokenId.getContainerID().equals(containerID)) {
+            return BuilderUtils.newContainerStatus(containerID,
+                org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE,
+                this.context.getQueuingContext().getKilledQueuedContainers()
+                    .get(cTokenId), ContainerExitStatus.ABORTED, cTokenId
+                        .getResource(), cTokenId.getExecutionType());
+          }
+        }
       }
     }
     return super.getContainerStatusInternal(containerID, nmTokenIdentifier);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3344ba70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.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/queuing/TestQueuingContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
index 4d44d8d..caebef7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
@@ -24,13 +24,13 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -41,15 +41,12 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
-import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorPlugin;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -58,11 +55,6 @@ import org.junit.Test;
  * Class for testing the {@link QueuingContainerManagerImpl}.
  */
 public class TestQueuingContainerManager extends BaseContainerManagerTest {
-
-  interface HasResources {
-    boolean decide(Context context, ContainerId cId);
-  }
-
   public TestQueuingContainerManager() throws UnsupportedFileSystemException {
     super();
   }
@@ -78,18 +70,6 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
       DeletionService delSrvc) {
     return new QueuingContainerManagerImpl(context, exec, delSrvc,
         nodeStatusUpdater, metrics, dirsHandler) {
-
-      @Override
-      public void serviceInit(Configuration conf) throws Exception {
-        conf.set(
-            YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
-            MockResourceCalculatorPlugin.class.getCanonicalName());
-        conf.set(
-            YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
-            MockResourceCalculatorProcessTree.class.getCanonicalName());
-        super.serviceInit(conf);
-      }
-
       @Override
       public void
       setBlockNewContainerRequests(boolean blockNewContainerRequests) {
@@ -398,7 +378,7 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
     containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
-        createContainerId(0), ContainerState.DONE, 30);
+        createContainerId(0), ContainerState.DONE, 40);
     Thread.sleep(5000);
 
     // Get container statuses. Container 0 should be killed, container 1
@@ -429,7 +409,7 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
 
     // Make sure the remaining OPPORTUNISTIC container starts its execution.
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
-        createContainerId(2), ContainerState.DONE, 30);
+        createContainerId(2), ContainerState.DONE, 40);
     Thread.sleep(5000);
     statRequest = GetContainerStatusesRequest.newInstance(Arrays.asList(
         createContainerId(1)));
@@ -488,13 +468,12 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
     containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
-        createContainerId(0), ContainerState.DONE, 30);
+        createContainerId(0), ContainerState.DONE, 40);
     Thread.sleep(5000);
 
     // Get container statuses. Container 0 should be killed, container 1
     // should be queued and container 2 should be running.
     int killedContainers = 0;
-    int runningContainers = 0;
     List<ContainerId> statList = new ArrayList<ContainerId>();
     for (int i = 0; i < 4; i++) {
       statList.add(createContainerId(i));
@@ -508,14 +487,108 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
           "Container killed by the ApplicationMaster")) {
         killedContainers++;
       }
+      System.out.println("\nStatus : [" + status + "]\n");
+    }
+
+    Assert.assertEquals(2, killedContainers);
+  }
+
+  /**
+   * Start running one GUARANTEED container and queue two OPPORTUNISTIC ones.
+   * Try killing one of the two queued containers.
+   * @throws Exception
+   */
+  @Test
+  public void testStopQueuedContainer() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(2000);
+
+    // Assert there is initially one container running and two queued.
+    int runningContainersNo = 0;
+    int queuedContainersNo = 0;
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest = GetContainerStatusesRequest
+        .newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
       if (status.getState() ==
           org.apache.hadoop.yarn.api.records.ContainerState.RUNNING) {
-        runningContainers++;
+        runningContainersNo++;
+      } else if (status.getState() ==
+          org.apache.hadoop.yarn.api.records.ContainerState.QUEUED) {
+        queuedContainersNo++;
       }
       System.out.println("\nStatus : [" + status + "]\n");
     }
 
-    Assert.assertEquals(2, killedContainers);
-    Assert.assertEquals(2, runningContainers);
+    Assert.assertEquals(1, runningContainersNo);
+    Assert.assertEquals(2, queuedContainersNo);
+
+    // Stop one of the two queued containers.
+    StopContainersRequest stopRequest = StopContainersRequest.
+        newInstance(Arrays.asList(createContainerId(1)));
+    containerManager.stopContainers(stopRequest);
+
+    Thread.sleep(2000);
+
+    // Assert queued container got properly stopped.
+    statList.clear();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    statRequest = GetContainerStatusesRequest.newInstance(statList);
+    containerStatuses = containerManager.getContainerStatuses(statRequest)
+        .getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getContainerId().equals(createContainerId(0))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      } else if (status.getContainerId().equals(createContainerId(1))) {
+        Assert.assertTrue(status.getDiagnostics().contains(
+            "Queued container request removed"));
+      } else if (status.getContainerId().equals(createContainerId(2))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
+            status.getState());
+      }
+      System.out.println("\nStatus : [" + status + "]\n");
+    }
   }
 }


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


[33/50] [abbrv] hadoop git commit: YARN-5176. More test cases for queuing of containers at the NM. (Konstantinos Karanasos via asuresh)

Posted by ae...@apache.org.
YARN-5176. More test cases for queuing of containers at the NM. (Konstantinos Karanasos via asuresh)


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

Branch: refs/heads/HDFS-7240
Commit: 76f0800c21f49fba01694cbdc870103053da802c
Parents: 58be55b
Author: Arun Suresh <as...@apache.org>
Authored: Tue Jun 7 17:16:18 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Tue Jun 7 17:16:18 2016 -0700

----------------------------------------------------------------------
 .../queuing/QueuingContainerManagerImpl.java    |  11 +
 .../BaseContainerManagerTest.java               |  64 +++
 .../containermanager/TestContainerManager.java  |  76 +---
 .../queuing/TestQueuingContainerManager.java    | 388 +++++++++++++++----
 4 files changed, 391 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76f0800c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.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/queuing/QueuingContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
index 1ce3356..a1e3bdb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
@@ -160,6 +160,7 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
           containerTokenId.getExecutionType());
 
       if (foundInQueue) {
+        LOG.info("Removing queued container with ID " + containerID);
         this.context.getQueuingContext().getKilledQueuedContainers().put(
             containerTokenId,
             "Queued container request removed by ApplicationMaster.");
@@ -502,6 +503,16 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
     return allocatedOpportunisticContainers.size();
   }
 
+  @VisibleForTesting
+  public int getNumQueuedGuaranteedContainers() {
+    return queuedGuaranteedContainers.size();
+  }
+
+  @VisibleForTesting
+  public int getNumQueuedOpportunisticContainers() {
+    return queuedOpportunisticContainers.size();
+  }
+
   class QueuingApplicationEventDispatcher implements
       EventHandler<ApplicationEvent> {
     private EventHandler<ApplicationEvent> applicationEventDispatcher;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76f0800c/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 ab60288..4f0e5c3 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
@@ -40,10 +40,17 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -51,6 +58,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
@@ -71,6 +79,7 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreServic
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.After;
 import org.junit.Before;
 
@@ -354,4 +363,59 @@ public abstract class BaseContainerManagerTest {
     Assert.assertEquals("ContainerState is not correct (timedout)",
         finalState, currentState);
   }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user,
+      NMContainerTokenSecretManager containerTokenSecretManager)
+      throws IOException {
+    return createContainerToken(cId, rmIdentifier, nodeId, user,
+      containerTokenSecretManager, null);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext)
+      throws IOException {
+    Resource r = BuilderUtils.newResource(1024, 1);
+    return createContainerToken(cId, rmIdentifier, nodeId, user, r,
+        containerTokenSecretManager, logAggregationContext);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext)
+      throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
+          System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+          Priority.newInstance(0), 0, logAggregationContext, null);
+    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
+        .retrievePassword(containerTokenIdentifier),
+            containerTokenIdentifier);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext, ExecutionType executionType)
+      throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
+            System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+            Priority.newInstance(0), 0, logAggregationContext, null,
+            ContainerType.TASK, executionType);
+    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
+            .retrievePassword(containerTokenIdentifier),
+        containerTokenIdentifier);
+  }
+
+  public static ContainerId createContainerId(int id) {
+    ApplicationId appId = ApplicationId.newInstance(0, 0);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
+    return containerId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76f0800c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.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/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 702198e..10b9155 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
@@ -38,10 +43,10 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
-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.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
@@ -58,13 +63,9 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.LogAggregationContext;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
@@ -75,7 +76,6 @@ import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
-import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
@@ -90,8 +90,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
-import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -99,11 +97,6 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.timeout;
-import static org.mockito.Mockito.verify;
-
 public class TestContainerManager extends BaseContainerManagerTest {
 
   public TestContainerManager() throws UnsupportedFileSystemException {
@@ -119,14 +112,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
   public void setup() throws IOException {
     super.setup();
   }
-
-  public static ContainerId createContainerId(int id) {
-    ApplicationId appId = ApplicationId.newInstance(0, 0);
-    ApplicationAttemptId appAttemptId =
-        ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
-    return containerId;
-  }
   
   @Override
   protected ContainerManagerImpl
@@ -1153,53 +1138,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
     assertEquals(targetResource, containerStatus.getCapability());
   }
 
-  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
-      NodeId nodeId, String user,
-      NMContainerTokenSecretManager containerTokenSecretManager)
-      throws IOException {
-    return createContainerToken(cId, rmIdentifier, nodeId, user,
-      containerTokenSecretManager, null);
-  }
-
-  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
-      NodeId nodeId, String user,
-      NMContainerTokenSecretManager containerTokenSecretManager,
-      LogAggregationContext logAggregationContext)
-      throws IOException {
-    Resource r = BuilderUtils.newResource(1024, 1);
-    return createContainerToken(cId, rmIdentifier, nodeId, user, r,
-        containerTokenSecretManager, logAggregationContext);
-  }
-
-  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
-      NodeId nodeId, String user, Resource resource,
-      NMContainerTokenSecretManager containerTokenSecretManager,
-      LogAggregationContext logAggregationContext)
-      throws IOException {
-    ContainerTokenIdentifier containerTokenIdentifier =
-        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
-          System.currentTimeMillis() + 100000L, 123, rmIdentifier,
-          Priority.newInstance(0), 0, logAggregationContext, null);
-    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
-        .retrievePassword(containerTokenIdentifier),
-            containerTokenIdentifier);
-  }
-
-  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
-      NodeId nodeId, String user, Resource resource,
-      NMContainerTokenSecretManager containerTokenSecretManager,
-      LogAggregationContext logAggregationContext, ExecutionType executionType)
-      throws IOException {
-    ContainerTokenIdentifier containerTokenIdentifier =
-        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
-            System.currentTimeMillis() + 100000L, 123, rmIdentifier,
-            Priority.newInstance(0), 0, logAggregationContext, null,
-            ContainerType.TASK, executionType);
-    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
-            .retrievePassword(containerTokenIdentifier),
-        containerTokenIdentifier);
-  }
-
   @Test
   public void testOutputThreadDumpSignal() throws IOException,
       InterruptedException, YarnException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76f0800c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.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/queuing/TestQueuingContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
index 8a0d155..4d44d8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -32,42 +37,27 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
-
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
-
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
-    .ContainersMonitorImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class TestQueuingContainerManager extends TestContainerManager {
+/**
+ * Class for testing the {@link QueuingContainerManagerImpl}.
+ */
+public class TestQueuingContainerManager extends BaseContainerManagerTest {
 
   interface HasResources {
     boolean decide(Context context, ContainerId cId);
@@ -120,15 +110,6 @@ public class TestQueuingContainerManager extends TestContainerManager {
       }
 
       @Override
-      protected void authorizeGetAndStopContainerRequest(
-          ContainerId containerId, Container container, boolean stopRequest,
-          NMTokenIdentifier identifier) throws YarnException {
-        if (container == null || container.getUser().equals("Fail")) {
-          throw new YarnException("Reject this container");
-        }
-      }
-
-      @Override
       protected ContainersMonitor createContainersMonitor(
           ContainerExecutor exec) {
         return new ContainersMonitorImpl(exec, dispatcher, this.context) {
@@ -148,7 +129,7 @@ public class TestQueuingContainerManager extends TestContainerManager {
 
           @Override
           public long getVCoresAllocatedForContainers() {
-            return 2;
+            return 4;
           }
         };
       }
@@ -186,54 +167,17 @@ public class TestQueuingContainerManager extends TestContainerManager {
   }
 
   /**
-   * Test to verify that an OPPORTUNISTIC container is killed when
-   * a GUARANTEED container arrives and all the Node Resources are used up
-   *
-   * For this specific test case, 4 containers are requested (last one being
-   * guaranteed). Assumptions :
-   * 1) The first OPPORTUNISTIC Container will start running
-   * 2) The second and third OPP containers will be queued
-   * 3) When the GUARANTEED container comes in, the running OPP container
-   *    will be killed to make room
-   * 4) After the GUARANTEED container finishes, the remaining 2 OPP
-   *    containers will be dequeued and run.
-   * 5) Only the first OPP container will be killed.
-   *
+   * Starting one GUARANTEED and one OPPORTUNISTIC container.
    * @throws Exception
    */
   @Test
-  public void testSimpleOpportunisticContainer() throws Exception {
+  public void testStartMultipleContainers() throws Exception {
     shouldDeleteWait = true;
     containerManager.start();
 
-    // ////// Create the resources for the container
-    File dir = new File(tmpDir, "dir");
-    dir.mkdirs();
-    File file = new File(dir, "file");
-    PrintWriter fileWriter = new PrintWriter(file);
-    fileWriter.write("Hello World!");
-    fileWriter.close();
-
-    // ////// Construct the container-spec.
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
-            .makeQualified(new Path(file.getAbsolutePath())));
-    LocalResource rsrc_alpha =
-        recordFactory.newRecordInstance(LocalResource.class);
-    rsrc_alpha.setResource(resource_alpha);
-    rsrc_alpha.setSize(-1);
-    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
-    rsrc_alpha.setType(LocalResourceType.FILE);
-    rsrc_alpha.setTimestamp(file.lastModified());
-    String destinationFile = "dest_file";
-    Map<String, LocalResource> localResources =
-        new HashMap<String, LocalResource>();
-    localResources.put(destinationFile, rsrc_alpha);
-    containerLaunchContext.setLocalResources(localResources);
-
-    // Start 3 OPPORTUNISTIC containers and 1 GUARANTEED container
+
     List<StartContainerRequest> list = new ArrayList<>();
     list.add(StartContainerRequest.newInstance(
         containerLaunchContext,
@@ -241,6 +185,122 @@ public class TestQueuingContainerManager extends TestContainerManager {
             context.getNodeId(),
             user, BuilderUtils.newResource(1024, 1),
             context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(1024, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    BaseContainerManagerTest.waitForContainerState(containerManager,
+        createContainerId(0),
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
+    BaseContainerManagerTest.waitForContainerState(containerManager,
+        createContainerId(1),
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
+
+    // Ensure all containers are running.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 2; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+    }
+  }
+
+  /**
+   * Submit both a GUARANTEED and an OPPORTUNISTIC container, each of which
+   * requires more resources than available at the node, and make sure they
+   * are both queued.
+   * @throws Exception
+   */
+  @Test
+  public void testQueueMultipleContainers() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(3072, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(3072, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(5000);
+
+    // Ensure both containers are queued.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 2; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
+          status.getState());
+    }
+
+    // Ensure both containers are properly queued.
+    Assert.assertEquals(2, containerManager.getContext().getQueuingContext()
+        .getQueuedContainers().size());
+    Assert.assertEquals(1, ((QueuingContainerManagerImpl) containerManager)
+        .getNumQueuedGuaranteedContainers());
+    Assert.assertEquals(1, ((QueuingContainerManagerImpl) containerManager)
+        .getNumQueuedOpportunisticContainers());
+  }
+
+  /**
+   * Starts one OPPORTUNISTIC container that takes up the whole node's
+   * resources, and submit two more that will be queued.
+   * @throws Exception
+   */
+  @Test
+  public void testStartAndQueueMultipleContainers() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
             ExecutionType.OPPORTUNISTIC)));
     list.add(StartContainerRequest.newInstance(
         containerLaunchContext,
@@ -256,23 +316,95 @@ public class TestQueuingContainerManager extends TestContainerManager {
             user, BuilderUtils.newResource(1024, 1),
             context.getContainerTokenSecretManager(), null,
             ExecutionType.OPPORTUNISTIC)));
-    // GUARANTEED
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(5000);
+
+    // Ensure first container is running and others are queued.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest = GetContainerStatusesRequest
+        .newInstance(Arrays.asList(createContainerId(0)));
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getContainerId().equals(createContainerId(0))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      } else {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
+            status.getState());
+      }
+    }
+
+    // Ensure two containers are properly queued.
+    Assert.assertEquals(2, containerManager.getContext().getQueuingContext()
+        .getQueuedContainers().size());
+    Assert.assertEquals(0, ((QueuingContainerManagerImpl) containerManager)
+        .getNumQueuedGuaranteedContainers());
+    Assert.assertEquals(2, ((QueuingContainerManagerImpl) containerManager)
+        .getNumQueuedOpportunisticContainers());
+  }
+
+  /**
+   * Submit two OPPORTUNISTIC and one GUARANTEED containers. The resources
+   * requests by each container as such that only one can run in parallel.
+   * Thus, the OPPORTUNISTIC container that started running, will be
+   * killed for the GUARANTEED container to start.
+   * Once the GUARANTEED container finishes its execution, the remaining
+   * OPPORTUNISTIC container will be executed.
+   * @throws Exception
+   */
+  @Test
+  public void testKillOpportunisticForGuaranteedContainer() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
     list.add(StartContainerRequest.newInstance(
         containerLaunchContext,
-        createContainerToken(createContainerId(3), DUMMY_RM_IDENTIFIER,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
             context.getNodeId(),
-            user, BuilderUtils.newResource(1024, 1),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
             context.getContainerTokenSecretManager(), null,
             ExecutionType.GUARANTEED)));
+
     StartContainersRequest allRequests =
         StartContainersRequest.newInstance(list);
-
     containerManager.startContainers(allRequests);
 
-    Thread.sleep(10000);
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(0), ContainerState.DONE, 30);
+    Thread.sleep(5000);
 
+    // Get container statuses. Container 0 should be killed, container 1
+    // should be queued and container 2 should be running.
     List<ContainerId> statList = new ArrayList<ContainerId>();
-    for (int i = 0; i < 4; i++) {
+    for (int i = 0; i < 3; i++) {
       statList.add(createContainerId(i));
     }
     GetContainerStatusesRequest statRequest =
@@ -280,12 +412,110 @@ public class TestQueuingContainerManager extends TestContainerManager {
     List<ContainerStatus> containerStatuses = containerManager
         .getContainerStatuses(statRequest).getContainerStatuses();
     for (ContainerStatus status : containerStatuses) {
-      // Ensure that the first opportunistic container is killed
       if (status.getContainerId().equals(createContainerId(0))) {
         Assert.assertTrue(status.getDiagnostics()
             .contains("Container killed by the ApplicationMaster"));
+      } else if (status.getContainerId().equals(createContainerId(1))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
+            status.getState());
+      } else if (status.getContainerId().equals(createContainerId(2))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      }
+      System.out.println("\nStatus : [" + status + "]\n");
+    }
+
+    // Make sure the remaining OPPORTUNISTIC container starts its execution.
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(2), ContainerState.DONE, 30);
+    Thread.sleep(5000);
+    statRequest = GetContainerStatusesRequest.newInstance(Arrays.asList(
+        createContainerId(1)));
+    ContainerStatus contStatus1 = containerManager.getContainerStatuses(
+        statRequest).getContainerStatuses().get(0);
+    Assert.assertEquals(
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+        contStatus1.getState());
+  }
+
+  /**
+   * Submit three OPPORTUNISTIC containers that can run concurrently, and one
+   * GUARANTEED that needs to kill two of the OPPORTUNISTIC for it to run.
+   * @throws Exception
+   */
+  @Test
+  public void testKillMultipleOpportunisticContainers() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(3), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(1500, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(0), ContainerState.DONE, 30);
+    Thread.sleep(5000);
+
+    // Get container statuses. Container 0 should be killed, container 1
+    // should be queued and container 2 should be running.
+    int killedContainers = 0;
+    int runningContainers = 0;
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 4; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getDiagnostics().contains(
+          "Container killed by the ApplicationMaster")) {
+        killedContainers++;
+      }
+      if (status.getState() ==
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING) {
+        runningContainers++;
       }
       System.out.println("\nStatus : [" + status + "]\n");
     }
+
+    Assert.assertEquals(2, killedContainers);
+    Assert.assertEquals(2, runningContainers);
   }
 }


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


[38/50] [abbrv] hadoop git commit: MAPREDUCE-6240. Hadoop client displays confusing error message. (gera)

Posted by ae...@apache.org.
MAPREDUCE-6240. Hadoop client displays confusing error message. (gera)


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

Branch: refs/heads/HDFS-7240
Commit: 0af96a1c08594c809ecb254cee4f60dd22399772
Parents: 1ee9ea0
Author: Gera Shegalov <ge...@apache.org>
Authored: Sat May 28 22:01:07 2016 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Wed Jun 8 12:59:37 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/mapreduce/Cluster.java    | 15 ++++++-----
 .../TestClientProtocolProviderImpls.java        | 26 +++++++++++++++++---
 2 files changed, 32 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0af96a1c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
index 9563c0b..6ca918d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
@@ -102,6 +102,10 @@ public class Cluster {
       throws IOException {
 
     initProviderList();
+    final IOException initEx = new IOException(
+        "Cannot initialize Cluster. Please check your configuration for "
+            + MRConfig.FRAMEWORK_NAME
+            + " and the correspond server addresses.");
     for (ClientProtocolProvider provider : providerList) {
       LOG.debug("Trying ClientProtocolProvider : "
           + provider.getClass().getName());
@@ -124,16 +128,15 @@ public class Cluster {
               + " as the ClientProtocolProvider - returned null protocol");
         }
       } catch (Exception e) {
-        LOG.info("Failed to use " + provider.getClass().getName()
-            + " due to error: ", e);
+        final String errMsg = "Failed to use " + provider.getClass().getName()
+            + " due to error: ";
+        initEx.addSuppressed(new IOException(errMsg, e));
+        LOG.info(errMsg, e);
       }
     }
 
     if (null == clientProtocolProvider || null == client) {
-      throw new IOException(
-          "Cannot initialize Cluster. Please check your configuration for "
-              + MRConfig.FRAMEWORK_NAME
-              + " and the correspond server addresses.");
+      throw initEx;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0af96a1c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
index 6ad76e9..500e133 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
@@ -18,17 +18,20 @@
 
 package org.apache.hadoop.mapreduce;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.mapred.LocalJobRunner;
 import org.apache.hadoop.mapred.YARNRunner;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestClientProtocolProviderImpls {
 
   @Test
@@ -76,4 +79,21 @@ public class TestClientProtocolProviderImpls {
           "Cannot initialize Cluster. Please check"));
     }
   }
+
+  @Test
+  public void testClusterExceptionRootCause() throws Exception {
+    final Configuration conf = new Configuration();
+    conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "nosuchfs:///");
+    conf.set(JTConfig.JT_IPC_ADDRESS, "local");
+    try {
+      new Cluster(conf);
+      fail("Cluster init should fail because of non-existing FileSystem");
+    } catch (IOException ioEx) {
+      final String stackTrace = StringUtils.stringifyException(ioEx);
+      assertTrue("No root cause detected",
+          stackTrace.contains(UnsupportedFileSystemException.class.getName())
+              && stackTrace.contains("nosuchfs"));
+    }
+  }
 }


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


[19/50] [abbrv] 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-7240
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


[41/50] [abbrv] hadoop git commit: HDFS-10508. DFSInputStream should set thread's interrupt status after catching InterruptException from sleep. Contributed by Jing Zhao.

Posted by ae...@apache.org.
HDFS-10508. DFSInputStream should set thread's interrupt status after catching InterruptException from sleep. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-7240
Commit: 8ea9bbce2614e8eb499af73589f021ed1789e78f
Parents: 1500a0a
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Thu Jun 9 14:52:29 2016 +0900
Committer: Masatake Iwasaki <iw...@apache.org>
Committed: Thu Jun 9 14:52:29 2016 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java      | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ea9bbce/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 7f32a56..6132f83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -306,6 +306,7 @@ public class DFSInputStream extends FSInputStream
     try {
       Thread.sleep(waitTime);
     } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
       throw new InterruptedIOException(
           "Interrupted while getting the last block length.");
     }
@@ -417,6 +418,7 @@ public class DFSInputStream extends FSInputStream
         try {
           Thread.sleep(500); // delay between retries.
         } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
           throw new InterruptedIOException(
               "Interrupted while getting the length.");
         }
@@ -1063,6 +1065,7 @@ public class DFSInputStream extends FSInputStream
               " IOException, will wait for " + waitTime + " msec.");
           Thread.sleep((long)waitTime);
         } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
           throw new InterruptedIOException(
               "Interrupted while choosing DataNode for read.");
         }


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


[45/50] [abbrv] hadoop git commit: HADOOP-12537 S3A to support Amazon STS temporary credentials. Contributed by Sean Mackrory.

Posted by ae...@apache.org.
HADOOP-12537 S3A to support Amazon STS temporary credentials. Contributed by Sean Mackrory.


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

Branch: refs/heads/HDFS-7240
Commit: 31ffaf76f2b6e1fd2a141daa4daaebdfecefe727
Parents: 9378d94
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jun 9 20:58:30 2016 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jun 9 21:00:47 2016 +0100

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |   5 +
 hadoop-project/pom.xml                          |   8 +-
 hadoop-tools/hadoop-aws/pom.xml                 |   5 +
 .../fs/s3a/BasicAWSCredentialsProvider.java     |   3 +-
 .../org/apache/hadoop/fs/s3a/Constants.java     |   3 +
 .../s3a/CredentialInitializationException.java  |  46 ++++++
 .../fs/s3a/TemporaryAWSCredentialsProvider.java |  70 +++++++++
 .../src/site/markdown/tools/hadoop-aws/index.md |  71 ++++++++-
 .../fs/s3a/TestS3ATemporaryCredentials.java     | 150 +++++++++++++++++++
 9 files changed, 357 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/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 8bb27ea..39b7132 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
@@ -806,6 +806,11 @@
 </property>
 
 <property>
+  <name>fs.s3a.session.token</name>
+  <description>The session token used with temporary credentials. Used only with provider org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider.</description>
+</property>
+
+<property>
   <name>fs.s3a.connection.maximum</name>
   <value>15</value>
   <description>Controls the maximum number of simultaneous connections to S3.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 2b6b162..4c618a1 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -116,6 +116,7 @@
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <surefire.fork.timeout>900</surefire.fork.timeout>
+    <aws-java-sdk.version>1.10.6</aws-java-sdk.version>
   </properties>
 
   <dependencyManagement>
@@ -690,7 +691,12 @@
       <dependency>
         <groupId>com.amazonaws</groupId>
         <artifactId>aws-java-sdk-s3</artifactId>
-        <version>1.10.6</version>
+        <version>${aws-java-sdk.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.amazonaws</groupId>
+        <artifactId>aws-java-sdk-sts</artifactId>
+        <version>${aws-java-sdk.version}</version>
       </dependency>
       <dependency>
         <groupId>org.apache.mina</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index c95f1e6..7c25e60 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -231,6 +231,11 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>aws-java-sdk-sts</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
index 3a5ee8c..61be43f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import com.amazonaws.AmazonClientException;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.BasicAWSCredentials;
 import com.amazonaws.auth.AWSCredentials;
@@ -49,7 +48,7 @@ public class BasicAWSCredentialsProvider implements AWSCredentialsProvider {
     if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
       return new BasicAWSCredentials(accessKey, secretKey);
     }
-    throw new AmazonClientException(
+    throw new CredentialInitializationException(
         "Access key or secret key is null");
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/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 eb859ac..4abb550 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
@@ -41,6 +41,9 @@ public final class Constants {
   public static final String AWS_CREDENTIALS_PROVIDER =
       "fs.s3a.aws.credentials.provider";
 
+  // session token for when using TemporaryAWSCredentialsProvider
+  public static final String SESSION_TOKEN = "fs.s3a.session.token";
+
   // number of simultaneous connections to s3
   public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum";
   public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
new file mode 100644
index 0000000..46655bc
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
@@ -0,0 +1,46 @@
+/*
+ * 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.AmazonClientException;
+
+/**
+ * Exception which Hadoop's AWSCredentialsProvider implementations should
+ * throw when there is a problem with the credential setup. This
+ * is a subclass of {@link AmazonClientException} which sets
+ * {@link #isRetryable()} to false, so as to fail fast.
+ */
+public class CredentialInitializationException extends AmazonClientException {
+  public CredentialInitializationException(String message, Throwable t) {
+    super(message, t);
+  }
+
+  public CredentialInitializationException(String message) {
+    super(message);
+  }
+
+  /**
+   * This exception is not going to go away if you try calling it again.
+   * @return false, always.
+   */
+  @Override
+  public boolean isRetryable() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
new file mode 100644
index 0000000..190f7bc
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
@@ -0,0 +1,70 @@
+/**
+ * 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.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.BasicSessionCredentials;
+import com.amazonaws.auth.AWSCredentials;
+import org.apache.commons.lang.StringUtils;
+
+import java.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Support session credentials for authenticating with AWS.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class TemporaryAWSCredentialsProvider implements AWSCredentialsProvider {
+
+  public static final String NAME
+      = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+  private final String accessKey;
+  private final String secretKey;
+  private final String sessionToken;
+
+  public TemporaryAWSCredentialsProvider(URI uri, Configuration conf) {
+    this.accessKey = conf.get(ACCESS_KEY, null);
+    this.secretKey = conf.get(SECRET_KEY, null);
+    this.sessionToken = conf.get(SESSION_TOKEN, null);
+  }
+
+  public AWSCredentials getCredentials() {
+    if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)
+        && !StringUtils.isEmpty(sessionToken)) {
+      return new BasicSessionCredentials(accessKey, secretKey, sessionToken);
+    }
+    throw new CredentialInitializationException(
+        "Access key, secret key or session token is unset");
+  }
+
+  @Override
+  public void refresh() {}
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/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 4086bc0..606275c 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
@@ -201,6 +201,46 @@ If you do any of these: change your credentials immediately!
       </description>
     </property>
 
+    <property>
+      <name>fs.s3a.session.token</name>
+      <description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider as the providers.</description>
+    </property>
+
+#### Authentication methods
+
+The standard way to authenticate is with an access key and secret key using the
+properties above. You can also avoid configuring credentials if the EC2
+instances in your cluster are configured with IAM instance profiles that grant
+the appropriate S3 access.
+
+A temporary set of credentials can also be obtained from Amazon STS; these
+consist of an access key, a secret key, and a session token. To use these
+temporary credentials you must include the `aws-java-sdk-sts` JAR in your
+classpath (consult the POM for the current version) and set the
+`TemporaryAWSCredentialsProvider` class as the provider. The session key
+must be set in the property `fs.s3a.session.token` \u2014and the access and secret
+key properties to those of this temporary session.
+
+    <property>
+      <name>fs.s3a.aws.credentials.provider</name>
+      <value>org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider</value>
+    </property>
+
+    <property>
+      <name>fs.s3a.access.key</name>
+      <value>SESSION-ACCESS-KEY</value>
+    </property>
+
+    <property>
+      <name>fs.s3a.secret.key</name>
+      <value>SESSION-SECRET-KEY</value>
+    </property>
+
+    <property>
+      <name>fs.s3a.session.token</name>
+      <value>SECRET-SESSION-TOKEN</value>
+    </property>
+
 #### Protecting the AWS Credentials in S3A
 
 To protect the access/secret keys from prying eyes, it is recommended that you
@@ -605,6 +645,13 @@ Example:
         <description>AWS secret key. Omit for IAM role-based authentication.</description>
         <value>DONOTEVERSHARETHISSECRETKEY!</value>
       </property>
+
+      <property>
+        <name>test.sts.endpoint</name>
+        <description>Specific endpoint to use for STS requests.</description>
+        <value>sts.amazonaws.com</value>
+      </property>
+
     </configuration>
 
 ### File `contract-test-options.xml`
@@ -714,8 +761,30 @@ that the file `contract-test-options.xml` does not contain any
 secret credentials itself. As the auth keys XML file is kept out of the
 source code tree, it is not going to get accidentally committed.
 
-### Running Performance Tests against non-AWS storage infrastructures
+### Running Tests against non-AWS storage infrastructures
+
+### S3A session tests
+
+The test `TestS3ATemporaryCredentials` requests a set of temporary
+credentials from the STS service, then uses them to authenticate with S3.
+
+If an S3 implementation does not support STS, then the functional test
+cases must be disabled:
+
+        <property>
+          <name>test.fs.s3a.sts.enabled</name>
+          <value>false</value>
+        </property>
+
+These tests reqest a temporary set of credentials from the STS service endpoint.
+An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`.
+
+        <property>
+          <name>test.fs.s3a.sts.endpoint</name>
+          <value>https://sts.example.org/</value>
+        </property>
 
+The default is ""; meaning "use the amazon default value".
 
 #### CSV Data source
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
new file mode 100644
index 0000000..be5c599
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
@@ -0,0 +1,150 @@
+/**
+ * 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 java.io.IOException;
+
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProviderChain;
+import com.amazonaws.auth.InstanceProfileCredentialsProvider;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient;
+import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
+import com.amazonaws.services.securitytoken.model.GetSessionTokenResult;
+import com.amazonaws.services.securitytoken.model.Credentials;
+
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.conf.Configuration;
+
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Tests use of temporary credentials (for example, AWS STS & S3).
+ * This test extends a class that "does things to the root directory", and
+ * should only be used against transient filesystems where you don't care about
+ * the data.
+ */
+public class TestS3ATemporaryCredentials extends AbstractFSContractTestBase {
+  public static final String TEST_STS_ENABLED = "test.fs.s3a.sts.enabled";
+  public static final String TEST_STS_ENDPOINT = "test.fs.s3a.sts.endpoint";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestS3ATemporaryCredentials.class);
+
+  private S3AFileSystem fs;
+
+
+  private static final String PROVIDER_CLASS =
+      "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+
+  private static final long TEST_FILE_SIZE = 1024;
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new S3AContract(conf);
+  }
+
+  /**
+   * Test use of STS for requesting temporary credentials.
+   *
+   * The property test.sts.endpoint can be set to point this at different
+   * STS endpoints. This test will use the AWS credentials (if provided) for
+   * S3A tests to request temporary credentials, then attempt to use those
+   * credentials instead.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testSTS() throws IOException {
+    Configuration conf = getContract().getConf();
+    if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
+      skip("STS functional tests disabled");
+    }
+
+    String parentAccessKey = conf.getTrimmed(ACCESS_KEY, null);
+    String parentSecretKey = conf.getTrimmed(SECRET_KEY, null);
+    String stsEndpoint = conf.getTrimmed(TEST_STS_ENDPOINT, "");
+    AWSCredentialsProviderChain parentCredentials;
+    parentCredentials = new AWSCredentialsProviderChain(
+      new BasicAWSCredentialsProvider(parentAccessKey, parentSecretKey),
+      new InstanceProfileCredentialsProvider()
+    );
+
+    AWSSecurityTokenServiceClient stsClient;
+    stsClient = new AWSSecurityTokenServiceClient(parentCredentials);
+    if (!stsEndpoint.isEmpty()) {
+      LOG.debug("STS Endpoint ={}", stsEndpoint);
+      stsClient.setEndpoint(stsEndpoint);
+    }
+    GetSessionTokenRequest sessionTokenRequest = new GetSessionTokenRequest();
+    sessionTokenRequest.setDurationSeconds(900);
+    GetSessionTokenResult sessionTokenResult;
+    sessionTokenResult = stsClient.getSessionToken(sessionTokenRequest);
+    Credentials sessionCreds = sessionTokenResult.getCredentials();
+
+    String childAccessKey = sessionCreds.getAccessKeyId();
+    conf.set(ACCESS_KEY, childAccessKey);
+    String childSecretKey = sessionCreds.getSecretAccessKey();
+    conf.set(SECRET_KEY, childSecretKey);
+    String sessionToken = sessionCreds.getSessionToken();
+    conf.set(SESSION_TOKEN, sessionToken);
+
+    conf.set(AWS_CREDENTIALS_PROVIDER, PROVIDER_CLASS);
+
+    try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      createAndVerifyFile(fs, path("testSTS"), TEST_FILE_SIZE);
+    }
+
+    // now create an invalid set of credentials by changing the session
+    // token
+    conf.set(SESSION_TOKEN, "invalid-" + sessionToken);
+    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      createAndVerifyFile(fs, path("testSTSInvalidToken"), TEST_FILE_SIZE);
+      fail("Expected an access exception, but file access to "
+          + fs.getUri() + " was allowed: " + fs);
+    } catch (AWSS3IOException ex) {
+      LOG.info("Expected Exception: {}", ex.toString());
+      LOG.debug("Expected Exception: {}", ex, ex);
+    }
+  }
+
+  @Test
+  public void testTemporaryCredentialValidation() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ACCESS_KEY, "accesskey");
+    conf.set(SECRET_KEY, "secretkey");
+    conf.set(SESSION_TOKEN, "");
+    TemporaryAWSCredentialsProvider provider
+        = new TemporaryAWSCredentialsProvider(getFileSystem().getUri(), conf);
+    try {
+      AWSCredentials credentials = provider.getCredentials();
+      fail("Expected a CredentialInitializationException,"
+          + " got " + credentials);
+    } catch (CredentialInitializationException expected) {
+      // expected
+    }
+  }
+}


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


[37/50] [abbrv] hadoop git commit: Remove redundant TestMiniDFSCluster.testDualClusters. Contributed by Jiayi Zhou.

Posted by ae...@apache.org.
Remove redundant TestMiniDFSCluster.testDualClusters. 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/1ee9ea00
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1ee9ea00
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1ee9ea00

Branch: refs/heads/HDFS-7240
Commit: 1ee9ea002609971ad58082bf525d57fca8a37035
Parents: 5a43583
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Jun 8 12:58:56 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Jun 8 12:58:56 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/TestMiniDFSCluster.java  | 31 --------------------
 1 file changed, 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee9ea00/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
index 78ae8b1..ec72d87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
@@ -84,36 +83,6 @@ public class TestMiniDFSCluster {
     }
   }
 
-  /**
-   * Bring up two clusters and assert that they are in different directories.
-   * @throws Throwable on a failure
-   */
-  @Test(timeout=100000)
-  public void testDualClusters() throws Throwable {
-    File testDataCluster2 = new File(testDataPath, CLUSTER_2);
-    File testDataCluster3 = new File(testDataPath, CLUSTER_3);
-    Configuration conf = new HdfsConfiguration();
-    String c2Path = testDataCluster2.getAbsolutePath();
-    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, c2Path);
-    MiniDFSCluster cluster2 = new MiniDFSCluster.Builder(conf).build();
-    MiniDFSCluster cluster3 = null;
-    try {
-      String dataDir2 = cluster2.getDataDirectory();
-      assertEquals(new File(c2Path + "/data"), new File(dataDir2));
-      //change the data dir
-      conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR,
-               testDataCluster3.getAbsolutePath());
-      MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-      cluster3 = builder.build();
-      String dataDir3 = cluster3.getDataDirectory();
-      assertTrue("Clusters are bound to the same directory: " + dataDir2,
-                        !dataDir2.equals(dataDir3));
-    } finally {
-      MiniDFSCluster.shutdownCluster(cluster3);
-      MiniDFSCluster.shutdownCluster(cluster2);
-    }
-  }
-
   @Test(timeout=100000)
   public void testIsClusterUpAfterShutdown() throws Throwable {
     Configuration conf = new HdfsConfiguration();


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


[34/50] [abbrv] hadoop git commit: YARN-5206. RegistrySecurity includes id:pass in exception text if considered invalid. Contributed by Steve Loughran

Posted by ae...@apache.org.
YARN-5206. RegistrySecurity includes id:pass in exception text if considered invalid. 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/8c8a377c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8c8a377c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8c8a377c

Branch: refs/heads/HDFS-7240
Commit: 8c8a377cac10b086a7ff37ee366b79e6b04d2738
Parents: 723432b
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jun 8 14:11:25 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jun 8 14:11:25 2016 +0000

----------------------------------------------------------------------
 .../apache/hadoop/registry/client/impl/zk/RegistrySecurity.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c8a377c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
index fc61460..49673fc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
@@ -443,7 +443,7 @@ public class RegistrySecurity extends AbstractService {
    */
   public String digest(String idPasswordPair) throws IOException {
     if (StringUtils.isEmpty(idPasswordPair) || !isValid(idPasswordPair)) {
-      throw new IOException("Invalid id:password: " + idPasswordPair);
+      throw new IOException("Invalid id:password");
     }
     try {
       return DigestAuthenticationProvider.generateDigest(idPasswordPair);


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


[49/50] [abbrv] hadoop git commit: HADOOP-12666. Support Microsoft Azure Data Lake - as a file system in Hadoop. Contributed by Vishwajeet Dusane.

Posted by ae...@apache.org.
HADOOP-12666. Support Microsoft Azure Data Lake - as a file system in Hadoop. Contributed by Vishwajeet Dusane.


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

Branch: refs/heads/HDFS-7240
Commit: 9581fb715cbc8a6ad28566e83c6d0242a7306688
Parents: e383b73
Author: Chris Nauroth <cn...@apache.org>
Authored: Thu Jun 9 14:33:31 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Thu Jun 9 14:33:31 2016 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |   60 +
 .../conf/TestCommonConfigurationFields.java     |    6 +
 hadoop-project/src/site/site.xml                |    2 +
 .../dev-support/findbugs-exclude.xml            |   24 +
 hadoop-tools/hadoop-azure-datalake/pom.xml      |  180 +++
 .../main/java/org/apache/hadoop/fs/adl/Adl.java |   52 +
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |   41 +
 ...hedRefreshTokenBasedAccessTokenProvider.java |  135 +++
 .../hadoop/fs/adl/oauth2/package-info.java      |   23 +
 .../org/apache/hadoop/fs/adl/package-info.java  |   23 +
 .../org/apache/hadoop/hdfs/web/ADLConfKeys.java |   61 +
 .../apache/hadoop/hdfs/web/BufferManager.java   |  180 +++
 .../web/PrivateAzureDataLakeFileSystem.java     | 1108 ++++++++++++++++++
 ...hedRefreshTokenBasedAccessTokenProvider.java |   37 +
 .../hadoop/hdfs/web/oauth2/package-info.java    |   24 +
 .../apache/hadoop/hdfs/web/package-info.java    |   25 +
 .../hadoop/hdfs/web/resources/ADLFlush.java     |   49 +
 .../hdfs/web/resources/ADLGetOpParam.java       |   96 ++
 .../hdfs/web/resources/ADLPostOpParam.java      |   97 ++
 .../hdfs/web/resources/ADLPutOpParam.java       |   94 ++
 .../hdfs/web/resources/ADLVersionInfo.java      |   51 +
 .../web/resources/AppendADLNoRedirectParam.java |   45 +
 .../web/resources/CreateADLNoRedirectParam.java |   44 +
 .../hadoop/hdfs/web/resources/LeaseParam.java   |   53 +
 .../web/resources/ReadADLNoRedirectParam.java   |   44 +
 .../hadoop/hdfs/web/resources/package-info.java |   27 +
 .../src/site/markdown/index.md                  |  219 ++++
 ...hedRefreshTokenBasedAccessTokenProvider.java |  147 +++
 hadoop-tools/hadoop-tools-dist/pom.xml          |    6 +
 hadoop-tools/pom.xml                            |    1 +
 30 files changed, 2954 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/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 39b7132..f1d77dd 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
@@ -2213,4 +2213,64 @@
       needs to be specified in net.topology.script.file.name.
     </description>
   </property>
+
+
+  <!-- Azure Data Lake File System Configurations -->
+
+  <property>
+    <name>adl.feature.override.readahead</name>
+    <value>true</value>
+    <description>
+      Enables read aheads in the ADL client, the feature is used to
+      improve read throughput.
+      This works in conjunction with the value set in
+      adl.feature.override.readahead.max.buffersize.
+      When set to false the read ahead feature is turned off.
+      Default : True if not configured.
+    </description>
+  </property>
+
+  <property>
+    <name>adl.feature.override.readahead.max.buffersize</name>
+    <value>8388608</value>
+    <description>
+      Define maximum buffer size to cache read ahead data, this is
+      allocated per process to
+      cache read ahead data. Applicable only when
+      adl.feature.override.readahead is set to true.
+      Default : 8388608 Byte i.e. 8MB if not configured.
+    </description>
+  </property>
+
+  <property>
+    <name>adl.feature.override.readahead.max.concurrent.connection</name>
+    <value>2</value>
+    <description>
+      Define maximum concurrent connection can be established to
+      read ahead. If the data size is less than 4MB then only 1 read n/w
+      connection
+      is set. If the data size is less than 4MB but less than 8MB then 2 read
+      n/w connection
+      is set. Data greater than 8MB then value set under the property would
+      take
+      effect. Applicable only when adl.feature.override.readahead is set
+      to true and buffer size is greater than 8MB.
+      It is recommended to reset this property if the
+      adl.feature.override.readahead.max.buffersize
+      is less than 8MB to gain performance. Application has to consider
+      throttling limit for the account as well before configuring large
+      buffer size.
+    </description>
+  </property>
+
+  <property>
+    <name>fs.adl.impl</name>
+    <value>org.apache.hadoop.fs.adl.AdlFileSystem</value>
+  </property>
+
+  <property>
+    <name>fs.AbstractFileSystem.adl.impl</name>
+    <value>org.apache.hadoop.fs.adl.Adl</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
index 90f7514..020474f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
@@ -102,6 +102,12 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPrefixToSkipCompare.add("s3.");
     xmlPrefixToSkipCompare.add("s3native.");
 
+    // ADL properties are in a different subtree
+    // - org.apache.hadoop.hdfs.web.ADLConfKeys
+    xmlPrefixToSkipCompare.add("adl.");
+    xmlPropsToSkipCompare.add("fs.adl.impl");
+    xmlPropsToSkipCompare.add("fs.AbstractFileSystem.adl.impl");
+
     // Deprecated properties.  These should eventually be removed from the
     // class.
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index f9f4726..a89a220 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -146,6 +146,8 @@
     <menu name="Hadoop Compatible File Systems" inherit="top">
       <item name="Amazon S3" href="hadoop-aws/tools/hadoop-aws/index.html"/>
       <item name="Azure Blob Storage" href="hadoop-azure/index.html"/>
+      <item name="Azure Data Lake Storage"
+            href="hadoop-azure-datalake/index.html"/>
       <item name="OpenStack Swift" href="hadoop-openstack/index.html"/>
     </menu>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-azure-datalake/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000..4fd36ef
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/dev-support/findbugs-exclude.xml
@@ -0,0 +1,24 @@
+<!--
+   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.
+-->
+<FindBugsFilter>
+    <!-- Buffer object is accessed withing trusted code and intentionally assigned instead of array copy -->
+    <Match>
+        <Class name="org.apache.hadoop.hdfs.web.PrivateAzureDataLakeFileSystem$BatchAppendOutputStream$CommitTask"/>
+        <Bug pattern="EI_EXPOSE_REP2"/>
+        <Priority value="2"/>
+    </Match>
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
new file mode 100644
index 0000000..a4b1fe1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -0,0 +1,180 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha1-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-azure-datalake</artifactId>
+  <name>Apache Hadoop Azure Data Lake support</name>
+  <description>
+    This module contains code to support integration with Azure Data Lake.
+  </description>
+  <packaging>jar</packaging>
+  <properties>
+    <okHttpVersion>2.4.0</okHttpVersion>
+    <minimalJsonVersion>0.9.1</minimalJsonVersion>
+    <file.encoding>UTF-8</file.encoding>
+    <downloadSources>true</downloadSources>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <findbugsXmlOutput>true</findbugsXmlOutput>
+          <xmlOutput>true</xmlOutput>
+          <excludeFilterFile>
+            ${basedir}/dev-support/findbugs-exclude.xml
+          </excludeFilterFile>
+          <effort>Max</effort>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+
+        <configuration>
+          <dependencyDetailsEnabled>false</dependencyDetailsEnabled>
+          <dependencyLocationsEnabled>false
+          </dependencyLocationsEnabled>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>deplist</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>list</goal>
+            </goals>
+            <configuration>
+              <!-- build a shellprofile -->
+              <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-optional.txt</outputFile>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+
+
+    <!--
+       The following is to suppress a m2e warning in eclipse
+       (m2e doesn't know how to handle maven-enforcer:enforce, so we have to tell m2e to ignore it)
+       see: http://stackoverflow.com/questions/13040788/how-to-elimate-the-maven-enforcer-plugin-goal-enforce-is-ignored-by-m2e-wa
+    -->
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <version>1.0.0</version>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins
+                    </groupId>
+                    <artifactId>maven-enforcer-plugin
+                    </artifactId>
+                    <versionRange>[1.0.0,)</versionRange>
+                    <goals>
+                      <goal>enforce</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore/>
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <exclusions>
+        <exclusion>
+          <artifactId>servlet-api</artifactId>
+          <groupId>javax.servlet</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+    </dependency>
+    <dependency>
+    <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+  </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.eclipsesource.minimal-json</groupId>
+      <artifactId>minimal-json</artifactId>
+      <version>0.9.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>com.squareup.okhttp</groupId>
+      <artifactId>mockwebserver</artifactId>
+      <version>2.4.0</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/Adl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/Adl.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/Adl.java
new file mode 100644
index 0000000..4642d6b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/Adl.java
@@ -0,0 +1,52 @@
+/*
+ * 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.adl;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Expose adl:// scheme to access ADL file system.
+ */
+public class Adl extends DelegateToFileSystem {
+
+  Adl(URI theUri, Configuration conf) throws IOException, URISyntaxException {
+    super(theUri, createDataLakeFileSystem(conf), conf, AdlFileSystem.SCHEME,
+        false);
+  }
+
+  private static AdlFileSystem createDataLakeFileSystem(Configuration conf) {
+    AdlFileSystem fs = new AdlFileSystem();
+    fs.setConf(conf);
+    return fs;
+  }
+
+  /**
+   * @return Default port for ADL File system to communicate
+   */
+  @Override
+  public final int getUriDefaultPort() {
+    return AdlFileSystem.DEFAULT_PORT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
new file mode 100644
index 0000000..11e1e0b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -0,0 +1,41 @@
+/*
+ * 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.adl;
+
+import org.apache.hadoop.hdfs.web.PrivateAzureDataLakeFileSystem;
+
+/**
+ * Expose adl:// scheme to access ADL file system.
+ */
+public class AdlFileSystem extends PrivateAzureDataLakeFileSystem {
+
+  public static final String SCHEME = "adl";
+  public static final int DEFAULT_PORT = 443;
+
+  @Override
+  public String getScheme() {
+    return SCHEME;
+  }
+
+  @Override
+  public int getDefaultPort() {
+    return DEFAULT_PORT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/CachedRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/CachedRefreshTokenBasedAccessTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/CachedRefreshTokenBasedAccessTokenProvider.java
new file mode 100644
index 0000000..b7f3b00
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/CachedRefreshTokenBasedAccessTokenProvider.java
@@ -0,0 +1,135 @@
+/*
+ * 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.adl.oauth2;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.LinkedHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.oauth2.AccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.PrivateCachedRefreshTokenBasedAccessTokenProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_KEY;
+
+/**
+ * Share refresh tokens across all ADLS instances with a common client ID. The
+ * {@link AccessTokenProvider} can be shared across multiple instances,
+ * amortizing the cost of refreshing tokens.
+ */
+public class CachedRefreshTokenBasedAccessTokenProvider
+    extends PrivateCachedRefreshTokenBasedAccessTokenProvider {
+
+  public static final String FORCE_REFRESH = "adl.force.token.refresh";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CachedRefreshTokenBasedAccessTokenProvider.class);
+
+  /** Limit size of provider cache. */
+  static final int MAX_PROVIDERS = 10;
+  @SuppressWarnings("serial")
+  private static final Map<String, AccessTokenProvider> CACHE =
+      new LinkedHashMap<String, AccessTokenProvider>() {
+        @Override
+        public boolean removeEldestEntry(
+            Map.Entry<String, AccessTokenProvider> e) {
+          return size() > MAX_PROVIDERS;
+        }
+      };
+
+  private AccessTokenProvider instance = null;
+
+  /**
+   * Create handle for cached instance.
+   */
+  public CachedRefreshTokenBasedAccessTokenProvider() {
+  }
+
+  /**
+   * Gets the access token from internally cached
+   * ConfRefreshTokenBasedAccessTokenProvider instance.
+   *
+   * @return Valid OAuth2 access token for the user.
+   * @throws IOException when system error, internal server error or user error
+   */
+  @Override
+  public synchronized String getAccessToken() throws IOException {
+    return instance.getAccessToken();
+  }
+
+  /**
+   * @return A cached Configuration consistent with the parameters of this
+   * instance.
+   */
+  @Override
+  public synchronized Configuration getConf() {
+    return instance.getConf();
+  }
+
+  /**
+   * Configure cached instance. Note that the Configuration instance returned
+   * from subsequent calls to {@link #getConf() getConf} may be from a
+   * previous, cached entry.
+   * @param conf Configuration instance
+   */
+  @Override
+  public synchronized void setConf(Configuration conf) {
+    String id = conf.get(OAUTH_CLIENT_ID_KEY);
+    if (null == id) {
+      throw new IllegalArgumentException("Missing client ID");
+    }
+    synchronized (CACHE) {
+      instance = CACHE.get(id);
+      if (null == instance
+          || conf.getBoolean(FORCE_REFRESH, false)
+          || replace(instance, conf)) {
+        instance = newInstance();
+        // clone configuration
+        instance.setConf(new Configuration(conf));
+        CACHE.put(id, instance);
+        LOG.debug("Created new client {}", id);
+      }
+    }
+  }
+
+  AccessTokenProvider newInstance() {
+    return new ConfRefreshTokenBasedAccessTokenProvider();
+  }
+
+  private static boolean replace(AccessTokenProvider cached, Configuration c2) {
+    // ConfRefreshTokenBasedAccessTokenProvider::setConf asserts !null
+    final Configuration c1 = cached.getConf();
+    for (String key : new String[] {
+        OAUTH_REFRESH_TOKEN_KEY, OAUTH_REFRESH_URL_KEY }) {
+      if (!c1.get(key).equals(c2.get(key))) {
+        // replace cached instance for this clientID
+        return true;
+      }
+    }
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/package-info.java
new file mode 100644
index 0000000..b444984
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * public interface to expose OAuth2 authentication related features.
+ */
+package org.apache.hadoop.fs.adl.oauth2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/package-info.java
new file mode 100644
index 0000000..98e6a77
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * Supporting classes for metrics instrumentation.
+ */
+package org.apache.hadoop.fs.adl;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/ADLConfKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/ADLConfKeys.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/ADLConfKeys.java
new file mode 100644
index 0000000..a7f932f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/ADLConfKeys.java
@@ -0,0 +1,61 @@
+/*
+ * 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.web;
+
+/**
+ * Constants.
+ */
+public final class ADLConfKeys {
+  public static final String
+      ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN =
+      "adl.feature.override.readahead.max.concurrent.connection";
+  public static final int
+      ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN_DEFAULT = 2;
+  public static final String ADL_WEBSDK_VERSION_KEY = "ADLFeatureSet";
+  static final String ADL_DEBUG_OVERRIDE_LOCAL_USER_AS_OWNER =
+      "adl.debug.override.localuserasfileowner";
+  static final boolean ADL_DEBUG_SET_LOCAL_USER_AS_OWNER_DEFAULT = false;
+  static final String ADL_FEATURE_REDIRECT_OFF =
+      "adl.feature.override.redirection.off";
+  static final boolean ADL_FEATURE_REDIRECT_OFF_DEFAULT = true;
+  static final String ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED =
+      "adl.feature.override.getblocklocation.locally.bundled";
+  static final boolean ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED_DEFAULT
+      = true;
+  static final String ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD =
+      "adl.feature.override.readahead";
+  static final boolean ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_DEFAULT =
+      true;
+  static final String ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE =
+      "adl.feature.override.readahead.max.buffersize";
+
+  static final int KB = 1024;
+  static final int MB = KB * KB;
+  static final int DEFAULT_BLOCK_SIZE = 4 * MB;
+  static final int DEFAULT_EXTENT_SIZE = 256 * MB;
+  static final int DEFAULT_TIMEOUT_IN_SECONDS = 120;
+  static final int
+      ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE_DEFAULT =
+      8 * MB;
+
+  private ADLConfKeys() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/BufferManager.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/BufferManager.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/BufferManager.java
new file mode 100644
index 0000000..350c6e7
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/BufferManager.java
@@ -0,0 +1,180 @@
+/*
+ * 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.web;
+
+/**
+ * Responsible for holding buffered data in the process. Hold only 1 and only
+ * 1 buffer block in the memory. Buffer block
+ * information is for the given file and the offset from the which the block
+ * is fetched. Across the webhdfs instances if
+ * same buffer block has been used then backend trip is avoided. Buffer block
+ * is certainly important since ADL fetches
+ * large amount of data (Default is 4MB however can be configured through
+ * core-site.xml) from the backend.
+ * Observation is in case of ORC/Avro kind of compressed file, buffer block
+ * does not avoid few backend calls across
+ * webhdfs
+ * instances.
+ */
+final class BufferManager {
+  private static final BufferManager BUFFER_MANAGER_INSTANCE = new
+      BufferManager();
+  private static Object lock = new Object();
+  private Buffer buffer = null;
+  private String fileName;
+
+  /**
+   * Constructor.
+   */
+  private BufferManager() {
+  }
+
+  public static Object getLock() {
+    return lock;
+  }
+
+  public static BufferManager getInstance() {
+    return BUFFER_MANAGER_INSTANCE;
+  }
+
+  /**
+   * Validate if the current buffer block is of given stream.
+   *
+   * @param path   ADL stream path
+   * @param offset Stream offset that caller is interested in
+   * @return True if the buffer block is available otherwise false
+   */
+  boolean hasValidDataForOffset(String path, long offset) {
+    if (this.fileName == null) {
+      return false;
+    }
+
+    if (!this.fileName.equals(path)) {
+      return false;
+    }
+
+    if (buffer == null) {
+      return false;
+    }
+
+    if ((offset < buffer.offset) || (offset >= (buffer.offset
+        + buffer.data.length))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Clean buffer block.
+   */
+  void clear() {
+    buffer = null;
+  }
+
+  /**
+   * Validate if the current buffer block is of given stream. For now partial
+   * data available is not supported.
+   * Data must be available exactly or within the range of offset and size
+   * passed as parameter.
+   *
+   * @param path   Stream path
+   * @param offset Offset of the stream
+   * @param size   Size of the data from the offset of the stream caller
+   *               interested in
+   * @return True if the data is available from the given offset and of the
+   * size caller is interested in.
+   */
+  boolean hasData(String path, long offset, int size) {
+
+    if (!hasValidDataForOffset(path, offset)) {
+      return false;
+    }
+
+    if ((size + offset) > (buffer.data.length + buffer.offset)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Return the buffer block from the requested offset. It is caller
+   * responsibility to check if the buffer block is
+   * of there interest and offset is valid.
+   *
+   * @param data   Byte array to be filed from the buffer block
+   * @param offset Data to be fetched from the offset.
+   */
+  void get(byte[] data, long offset) {
+    System.arraycopy(buffer.data, (int) (offset - buffer.offset), data, 0,
+        data.length);
+  }
+
+  /**
+   * Create new empty buffer block of the given size.
+   *
+   * @param len Size of the buffer block.
+   * @return Empty byte array.
+   */
+  byte[] getEmpty(int len) {
+    return new byte[len];
+  }
+
+  /**
+   * This function allows caller to specify new buffer block for the stream
+   * which is pulled from the backend.
+   *
+   * @param data   Buffer
+   * @param path   Stream path to which buffer belongs to
+   * @param offset Stream offset where buffer start with
+   */
+  void add(byte[] data, String path, long offset) {
+    if (data == null) {
+      return;
+    }
+
+    buffer = new Buffer();
+    buffer.data = data;
+    buffer.offset = offset;
+    this.fileName = path;
+  }
+
+  /**
+   * @return Size of the buffer.
+   */
+  int getBufferSize() {
+    return buffer.data.length;
+  }
+
+  /**
+   * @return Stream offset where buffer start with
+   */
+  long getBufferOffset() {
+    return buffer.offset;
+  }
+
+  /**
+   * Buffer container.
+   */
+  static class Buffer {
+    private byte[] data;
+    private long offset;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
new file mode 100644
index 0000000..89011d2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
@@ -0,0 +1,1108 @@
+/*
+ * 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.web;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.web.resources.ADLFlush;
+import org.apache.hadoop.hdfs.web.resources.ADLGetOpParam;
+import org.apache.hadoop.hdfs.web.resources.ADLPostOpParam;
+import org.apache.hadoop.hdfs.web.resources.ADLPutOpParam;
+import org.apache.hadoop.hdfs.web.resources.ADLVersionInfo;
+import org.apache.hadoop.hdfs.web.resources.AppendADLNoRedirectParam;
+import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
+import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
+import org.apache.hadoop.hdfs.web.resources.CreateADLNoRedirectParam;
+import org.apache.hadoop.hdfs.web.resources.CreateFlagParam;
+import org.apache.hadoop.hdfs.web.resources.CreateParentParam;
+import org.apache.hadoop.hdfs.web.resources.GetOpParam;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.hdfs.web.resources.LeaseParam;
+import org.apache.hadoop.hdfs.web.resources.LengthParam;
+import org.apache.hadoop.hdfs.web.resources.OffsetParam;
+import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
+import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.hdfs.web.resources.PermissionParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.hdfs.web.resources.ReadADLNoRedirectParam;
+import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.VersionInfo;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.SocketException;
+import java.net.URI;
+import java.net.URL;
+import java.util.EnumSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Extended @see SWebHdfsFileSystem API. This class contains Azure data lake
+ * specific stability, Reliability and performance improvement.
+ * <p>
+ * Motivation behind PrivateAzureDataLakeFileSystem to encapsulate dependent
+ * implementation on org.apache.hadoop.hdfs.web package to configure query
+ * parameters, configuration over HTTP request send to backend .. etc. This
+ * class should be refactored and moved under package org.apache.hadoop.fs
+ * .adl once the required dependent changes are made into ASF code.
+ */
+public class PrivateAzureDataLakeFileSystem extends SWebHdfsFileSystem {
+
+  public static final String SCHEME = "adl";
+
+  // Feature configuration
+  private boolean featureGetBlockLocationLocallyBundled = true;
+  private boolean featureConcurrentReadWithReadAhead = true;
+  private boolean featureRedirectOff = true;
+  private boolean featureFlushWhenEOF = true;
+  private boolean overrideOwner = false;
+  private int maxConcurrentConnection;
+  private int maxBufferSize;
+  private String userName;
+
+  /**
+   * Constructor.
+   */
+  public PrivateAzureDataLakeFileSystem() {
+    try {
+      userName = UserGroupInformation.getCurrentUser().getShortUserName();
+    } catch (IOException e) {
+      userName = "hadoop";
+    }
+  }
+
+  @Override
+  public synchronized void initialize(URI uri, Configuration conf)
+      throws IOException {
+    super.initialize(uri, conf);
+    overrideOwner = getConf()
+        .getBoolean(ADLConfKeys.ADL_DEBUG_OVERRIDE_LOCAL_USER_AS_OWNER,
+            ADLConfKeys.ADL_DEBUG_SET_LOCAL_USER_AS_OWNER_DEFAULT);
+
+    featureRedirectOff = getConf()
+        .getBoolean(ADLConfKeys.ADL_FEATURE_REDIRECT_OFF,
+            ADLConfKeys.ADL_FEATURE_REDIRECT_OFF_DEFAULT);
+
+    featureGetBlockLocationLocallyBundled = getConf()
+        .getBoolean(ADLConfKeys.ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED,
+            ADLConfKeys.ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED_DEFAULT);
+
+    featureConcurrentReadWithReadAhead = getConf().
+        getBoolean(ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD,
+            ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_DEFAULT);
+
+    maxBufferSize = getConf().getInt(
+        ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE,
+        ADLConfKeys
+            .ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE_DEFAULT);
+
+    maxConcurrentConnection = getConf().getInt(
+        ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN,
+        ADLConfKeys
+            .ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN_DEFAULT);
+  }
+
+  @VisibleForTesting
+  protected boolean isFeatureGetBlockLocationLocallyBundled() {
+    return featureGetBlockLocationLocallyBundled;
+  }
+
+  @VisibleForTesting
+  protected boolean isFeatureConcurrentReadWithReadAhead() {
+    return featureConcurrentReadWithReadAhead;
+  }
+
+  @VisibleForTesting
+  protected boolean isFeatureRedirectOff() {
+    return featureRedirectOff;
+  }
+
+  @VisibleForTesting
+  protected boolean isOverrideOwnerFeatureOn() {
+    return overrideOwner;
+  }
+
+  @VisibleForTesting
+  protected int getMaxBufferSize() {
+    return maxBufferSize;
+  }
+
+  @VisibleForTesting
+  protected int getMaxConcurrentConnection() {
+    return maxConcurrentConnection;
+  }
+
+  @Override
+  public String getScheme() {
+    return SCHEME;
+  }
+
+  /**
+   * Constructing home directory locally is fine as long as Hadoop
+   * local user name and ADL user name relationship story is not fully baked
+   * yet.
+   *
+   * @return Hadoop local user home directory.
+   */
+  @Override
+  public final Path getHomeDirectory() {
+    try {
+      return makeQualified(new Path(
+          "/user/" + UserGroupInformation.getCurrentUser().getShortUserName()));
+    } catch (IOException e) {
+    }
+
+    return new Path("/user/" + userName);
+  }
+
+  /**
+   * Azure data lake does not support user configuration for data replication
+   * hence not leaving system to query on
+   * azure data lake.
+   *
+   * Stub implementation
+   *
+   * @param p           Not honoured
+   * @param replication Not honoured
+   * @return True hard coded since ADL file system does not support
+   * replication configuration
+   * @throws IOException No exception would not thrown in this case however
+   *                     aligning with parent api definition.
+   */
+  @Override
+  public final boolean setReplication(final Path p, final short replication)
+      throws IOException {
+    return true;
+  }
+
+  /**
+   * @param f File/Folder path
+   * @return FileStatus instance containing metadata information of f
+   * @throws IOException For any system error
+   */
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    statistics.incrementReadOps(1);
+    FileStatus status = super.getFileStatus(f);
+
+    if (overrideOwner) {
+      FileStatus proxiedStatus = new FileStatus(status.getLen(),
+          status.isDirectory(), status.getReplication(), status.getBlockSize(),
+          status.getModificationTime(), status.getAccessTime(),
+          status.getPermission(), userName, "hdfs", status.getPath());
+      return proxiedStatus;
+    } else {
+      return status;
+    }
+  }
+
+  /**
+   * Create call semantic is handled differently in case of ADL. Create
+   * semantics is translated to Create/Append
+   * semantics.
+   * 1. No dedicated connection to server.
+   * 2. Buffering is locally done, Once buffer is full or flush is invoked on
+   * the by the caller. All the pending
+   * data is pushed to ADL as APPEND operation code.
+   * 3. On close - Additional call is send to server to close the stream, and
+   * release lock from the stream.
+   *
+   * Necessity of Create/Append semantics is
+   * 1. ADL backend server does not allow idle connection for longer duration
+   * . In case of slow writer scenario,
+   * observed connection timeout/Connection reset causing occasional job
+   * failures.
+   * 2. Performance boost to jobs which are slow writer, avoided network latency
+   * 3. ADL equally better performing with multiple of 4MB chunk as append
+   * calls.
+   *
+   * @param f           File path
+   * @param permission  Access permission for the newly created file
+   * @param overwrite   Remove existing file and recreate new one if true
+   *                    otherwise throw error if file exist
+   * @param bufferSize  Buffer size, ADL backend does not honour
+   * @param replication Replication count, ADL backend does not honour
+   * @param blockSize   Block size, ADL backend does not honour
+   * @param progress    Progress indicator
+   * @return FSDataOutputStream OutputStream on which application can push
+   * stream of bytes
+   * @throws IOException when system error, internal server error or user error
+   */
+  @Override
+  public FSDataOutputStream create(final Path f, final FsPermission permission,
+      final boolean overwrite, final int bufferSize, final short replication,
+      final long blockSize, final Progressable progress) throws IOException {
+    statistics.incrementWriteOps(1);
+
+    return new FSDataOutputStream(new BatchAppendOutputStream(f, bufferSize,
+        new PermissionParam(applyUMask(permission)),
+        new OverwriteParam(overwrite), new BufferSizeParam(bufferSize),
+        new ReplicationParam(replication), new BlockSizeParam(blockSize),
+        new ADLVersionInfo(VersionInfo.getVersion())), statistics) {
+    };
+  }
+
+  @Override
+  public FSDataOutputStream createNonRecursive(final Path f,
+      final FsPermission permission, final EnumSet<CreateFlag> flag,
+      final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+    statistics.incrementWriteOps(1);
+
+    String leaseId = java.util.UUID.randomUUID().toString();
+    return new FSDataOutputStream(new BatchAppendOutputStream(f, bufferSize,
+        new PermissionParam(applyUMask(permission)), new CreateFlagParam(flag),
+        new CreateParentParam(false), new BufferSizeParam(bufferSize),
+        new ReplicationParam(replication), new LeaseParam(leaseId),
+        new BlockSizeParam(blockSize),
+        new ADLVersionInfo(VersionInfo.getVersion())), statistics) {
+    };
+  }
+
+  /**
+   * Since defined as private in parent class, redefined to pass through
+   * Create api implementation.
+   *
+   * @param permission
+   * @return FsPermission list
+   */
+  private FsPermission applyUMask(FsPermission permission) {
+    FsPermission fsPermission = permission;
+    if (fsPermission == null) {
+      fsPermission = FsPermission.getDefault();
+    }
+    return fsPermission.applyUMask(FsPermission.getUMask(getConf()));
+  }
+
+  /**
+   * Open call semantic is handled differently in case of ADL. Instead of
+   * network stream is returned to the user,
+   * Overridden FsInputStream is returned.
+   *
+   * 1. No dedicated connection to server.
+   * 2. Process level concurrent read ahead Buffering is done, This allows
+   * data to be available for caller quickly.
+   * 3. Number of byte to read ahead is configurable.
+   *
+   * Advantage of Process level concurrent read ahead Buffering semantics is
+   * 1. ADL backend server does not allow idle connection for longer duration
+   * . In case of slow reader scenario,
+   * observed connection timeout/Connection reset causing occasional job
+   * failures.
+   * 2. Performance boost to jobs which are slow reader, avoided network latency
+   * 3. Compressed format support like ORC, and large data files gains the
+   * most out of this implementation.
+   *
+   * Read ahead feature is configurable.
+   *
+   * @param f          File path
+   * @param buffersize Buffer size
+   * @return FSDataInputStream InputStream on which application can read
+   * stream of bytes
+   * @throws IOException when system error, internal server error or user error
+   */
+  @Override
+  public FSDataInputStream open(final Path f, final int buffersize)
+      throws IOException {
+    statistics.incrementReadOps(1);
+
+    final HttpOpParam.Op op = GetOpParam.Op.OPEN;
+    // use a runner so the open can recover from an invalid token
+    FsPathConnectionRunner runner = null;
+
+    if (featureConcurrentReadWithReadAhead) {
+      URL url = this.toUrl(op, f, new BufferSizeParam(buffersize),
+          new ReadADLNoRedirectParam(true),
+          new ADLVersionInfo(VersionInfo.getVersion()));
+
+      BatchByteArrayInputStream bb = new BatchByteArrayInputStream(url, f,
+          maxBufferSize, maxConcurrentConnection);
+
+      FSDataInputStream fin = new FSDataInputStream(bb);
+      return fin;
+    } else {
+      if (featureRedirectOff) {
+        runner = new FsPathConnectionRunner(ADLGetOpParam.Op.OPEN, f,
+            new BufferSizeParam(buffersize), new ReadADLNoRedirectParam(true),
+            new ADLVersionInfo(VersionInfo.getVersion()));
+      } else {
+        runner = new FsPathConnectionRunner(op, f,
+            new BufferSizeParam(buffersize));
+      }
+
+      return new FSDataInputStream(
+          new OffsetUrlInputStream(new UnresolvedUrlOpener(runner),
+              new OffsetUrlOpener(null)));
+    }
+  }
+
+  /**
+   * @param f File/Folder path
+   * @return FileStatus array list
+   * @throws IOException For system error
+   */
+  @Override
+  public FileStatus[] listStatus(final Path f) throws IOException {
+    FileStatus[] fileStatuses = super.listStatus(f);
+    for (int i = 0; i < fileStatuses.length; i++) {
+      if (overrideOwner) {
+        fileStatuses[i] = new FileStatus(fileStatuses[i].getLen(),
+            fileStatuses[i].isDirectory(), fileStatuses[i].getReplication(),
+            fileStatuses[i].getBlockSize(),
+            fileStatuses[i].getModificationTime(),
+            fileStatuses[i].getAccessTime(), fileStatuses[i].getPermission(),
+            userName, "hdfs", fileStatuses[i].getPath());
+      }
+    }
+    return fileStatuses;
+  }
+
+  @Override
+  public BlockLocation[] getFileBlockLocations(final FileStatus status,
+      final long offset, final long length) throws IOException {
+    if (status == null) {
+      return null;
+    }
+
+    if (featureGetBlockLocationLocallyBundled) {
+      if ((offset < 0) || (length < 0)) {
+        throw new IllegalArgumentException("Invalid start or len parameter");
+      }
+
+      if (status.getLen() < offset) {
+        return new BlockLocation[0];
+      }
+
+      final String[] name = {"localhost"};
+      final String[] host = {"localhost"};
+      long blockSize = ADLConfKeys.DEFAULT_EXTENT_SIZE; // Block size must be
+      // non zero
+      int numberOfLocations =
+          (int) (length / blockSize) + ((length % blockSize == 0) ? 0 : 1);
+      BlockLocation[] locations = new BlockLocation[numberOfLocations];
+      for (int i = 0; i < locations.length; i++) {
+        long currentOffset = offset + (i * blockSize);
+        long currentLength = Math
+            .min(blockSize, offset + length - currentOffset);
+        locations[i] = new BlockLocation(name, host, currentOffset,
+            currentLength);
+      }
+
+      return locations;
+    } else {
+      return getFileBlockLocations(status.getPath(), offset, length);
+    }
+  }
+
+  @Override
+  public BlockLocation[] getFileBlockLocations(final Path p, final long offset,
+      final long length) throws IOException {
+    statistics.incrementReadOps(1);
+
+    if (featureGetBlockLocationLocallyBundled) {
+      FileStatus fileStatus = getFileStatus(p);
+      return getFileBlockLocations(fileStatus, offset, length);
+    } else {
+      return super.getFileBlockLocations(p, offset, length);
+    }
+  }
+
+  enum StreamState {
+    Initial,
+    DataCachedInLocalBuffer,
+    StreamEnd
+  }
+
+  class BatchAppendOutputStream extends OutputStream {
+    private Path fsPath;
+    private Param<?, ?>[] parameters;
+    private byte[] data = null;
+    private int offset = 0;
+    private long length = 0;
+    private boolean eof = false;
+    private boolean hadError = false;
+    private byte[] dataBuffers = null;
+    private int bufSize = 0;
+    private boolean streamClosed = false;
+
+    public BatchAppendOutputStream(Path path, int bufferSize,
+        Param<?, ?>... param) throws IOException {
+      if (bufferSize < (ADLConfKeys.DEFAULT_BLOCK_SIZE)) {
+        bufSize = ADLConfKeys.DEFAULT_BLOCK_SIZE;
+      } else {
+        bufSize = bufferSize;
+      }
+
+      this.fsPath = path;
+      this.parameters = param;
+      this.data = getBuffer();
+      FSDataOutputStream createStream = null;
+      try {
+        if (featureRedirectOff) {
+          CreateADLNoRedirectParam skipRedirect = new CreateADLNoRedirectParam(
+              true);
+          Param<?, ?>[] tmpParam = featureFlushWhenEOF ?
+              new Param<?, ?>[param.length + 2] :
+              new Param<?, ?>[param.length + 1];
+          System.arraycopy(param, 0, tmpParam, 0, param.length);
+          tmpParam[param.length] = skipRedirect;
+          if (featureFlushWhenEOF) {
+            tmpParam[param.length + 1] = new ADLFlush(false);
+          }
+          createStream = new FsPathOutputStreamRunner(ADLPutOpParam.Op.CREATE,
+              fsPath, 1, tmpParam).run();
+        } else {
+          createStream = new FsPathOutputStreamRunner(PutOpParam.Op.CREATE,
+              fsPath, 1, param).run();
+        }
+      } finally {
+        if (createStream != null) {
+          createStream.close();
+        }
+      }
+    }
+
+    @Override
+    public final synchronized void write(int b) throws IOException {
+      if (streamClosed) {
+        throw new IOException(fsPath + " stream object is closed.");
+      }
+
+      if (offset == (data.length)) {
+        flush();
+      }
+
+      data[offset] = (byte) b;
+      offset++;
+
+      // Statistics will get incremented again as part of the batch updates,
+      // decrement here to avoid double value
+      if (statistics != null) {
+        statistics.incrementBytesWritten(-1);
+      }
+    }
+
+    @Override
+    public final synchronized void write(byte[] buf, int off, int len)
+        throws IOException {
+      if (streamClosed) {
+        throw new IOException(fsPath + " stream object is closed.");
+      }
+
+      int bytesToWrite = len;
+      int localOff = off;
+      int localLen = len;
+      if (localLen >= data.length) {
+        // Flush data that is already in our internal buffer
+        flush();
+
+        // Keep committing data until we have less than our internal buffers
+        // length left
+        do {
+          try {
+            commit(buf, localOff, data.length, eof);
+          } catch (IOException e) {
+            hadError = true;
+            throw e;
+          }
+          localOff += data.length;
+          localLen -= data.length;
+        } while (localLen >= data.length);
+      }
+
+      // At this point, we have less than data.length left to copy from users
+      // buffer
+      if (offset + localLen >= data.length) {
+        // Users buffer has enough data left to fill our internal buffer
+        int bytesToCopy = data.length - offset;
+        System.arraycopy(buf, localOff, data, offset, bytesToCopy);
+        offset += bytesToCopy;
+
+        // Flush our internal buffer
+        flush();
+        localOff += bytesToCopy;
+        localLen -= bytesToCopy;
+      }
+
+      if (localLen > 0) {
+        // Simply copy the remainder from the users buffer into our internal
+        // buffer
+        System.arraycopy(buf, localOff, data, offset, localLen);
+        offset += localLen;
+      }
+
+      // Statistics will get incremented again as part of the batch updates,
+      // decrement here to avoid double value
+      if (statistics != null) {
+        statistics.incrementBytesWritten(-bytesToWrite);
+      }
+    }
+
+    @Override
+    public final synchronized void flush() throws IOException {
+      if (streamClosed) {
+        throw new IOException(fsPath + " stream object is closed.");
+      }
+
+      if (offset > 0) {
+        try {
+          commit(data, 0, offset, eof);
+        } catch (IOException e) {
+          hadError = true;
+          throw e;
+        }
+      }
+
+      offset = 0;
+    }
+
+    @Override
+    public final synchronized void close() throws IOException {
+      // Stream is closed earlier, return quietly.
+      if(streamClosed) {
+        return;
+      }
+
+      if (featureRedirectOff) {
+        eof = true;
+      }
+
+      boolean flushedSomething = false;
+      if (hadError) {
+        // No point proceeding further since the error has occurred and
+        // stream would be required to upload again.
+        streamClosed = true;
+        return;
+      } else {
+        flushedSomething = offset > 0;
+        try {
+          flush();
+        } finally {
+          streamClosed = true;
+        }
+      }
+
+      if (featureRedirectOff) {
+        // If we didn't flush anything from our internal buffer, we have to
+        // call the service again
+        // with an empty payload and flush=true in the url
+        if (!flushedSomething) {
+          try {
+            commit(null, 0, ADLConfKeys.KB, true);
+          } finally {
+            streamClosed = true;
+          }
+        }
+      }
+    }
+
+    private void commit(byte[] buffer, int off, int len, boolean endOfFile)
+        throws IOException {
+      OutputStream out = null;
+      try {
+        if (featureRedirectOff) {
+          AppendADLNoRedirectParam skipRedirect = new AppendADLNoRedirectParam(
+              true);
+          Param<?, ?>[] tmpParam = featureFlushWhenEOF ?
+              new Param<?, ?>[parameters.length + 3] :
+              new Param<?, ?>[parameters.length + 1];
+          System.arraycopy(parameters, 0, tmpParam, 0, parameters.length);
+          tmpParam[parameters.length] = skipRedirect;
+          if (featureFlushWhenEOF) {
+            tmpParam[parameters.length + 1] = new ADLFlush(endOfFile);
+            tmpParam[parameters.length + 2] = new OffsetParam(length);
+          }
+
+          out = new FsPathOutputStreamRunner(ADLPostOpParam.Op.APPEND, fsPath,
+              len, tmpParam).run();
+        } else {
+          out = new FsPathOutputStreamRunner(ADLPostOpParam.Op.APPEND, fsPath,
+              len, parameters).run();
+        }
+
+        if (buffer != null) {
+          out.write(buffer, off, len);
+          length += len;
+        }
+      } finally {
+        if (out != null) {
+          out.close();
+        }
+      }
+    }
+
+    private byte[] getBuffer() {
+      // Switch between the first and second buffer
+      dataBuffers = new byte[bufSize];
+      return dataBuffers;
+    }
+  }
+
+  /**
+   * Read data from backend in chunks instead of persistent connection. This
+   * is to avoid slow reader causing socket
+   * timeout.
+   */
+  protected class BatchByteArrayInputStream extends FSInputStream {
+
+    private static final int SIZE4MB = 4 * 1024 * 1024;
+    private final URL runner;
+    private byte[] data = null;
+    private long validDataHoldingSize = 0;
+    private int bufferOffset = 0;
+    private long currentFileOffset = 0;
+    private long nextFileOffset = 0;
+    private long fileSize = 0;
+    private StreamState state = StreamState.Initial;
+    private int maxBufferSize;
+    private int maxConcurrentConnection;
+    private Path fsPath;
+    private boolean streamIsClosed;
+    private Future[] subtasks = null;
+
+    BatchByteArrayInputStream(URL url, Path p, int bufferSize,
+        int concurrentConnection) throws IOException {
+      this.runner = url;
+      fsPath = p;
+      FileStatus fStatus = getFileStatus(fsPath);
+      if (!fStatus.isFile()) {
+        throw new IOException("Cannot open the directory " + p + " for " +
+            "reading");
+      }
+      fileSize = fStatus.getLen();
+      this.maxBufferSize = bufferSize;
+      this.maxConcurrentConnection = concurrentConnection;
+      this.streamIsClosed = false;
+    }
+
+    @Override
+    public synchronized final int read(long position, byte[] buffer, int offset,
+        int length) throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      long oldPos = this.getPos();
+
+      int nread1;
+      try {
+        this.seek(position);
+        nread1 = this.read(buffer, offset, length);
+      } finally {
+        this.seek(oldPos);
+      }
+
+      return nread1;
+    }
+
+    @Override
+    public synchronized final int read() throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      int status = doBufferAvailabilityCheck();
+      if (status == -1) {
+        return status;
+      }
+      int ch = data[bufferOffset++] & (0xff);
+      if (statistics != null) {
+        statistics.incrementBytesRead(1);
+      }
+      return ch;
+    }
+
+    @Override
+    public synchronized final void readFully(long position, byte[] buffer,
+        int offset, int length) throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+
+      super.readFully(position, buffer, offset, length);
+      if (statistics != null) {
+        statistics.incrementBytesRead(length);
+      }
+    }
+
+    @Override
+    public synchronized final int read(byte[] b, int off, int len)
+        throws IOException {
+      if (b == null) {
+        throw new IllegalArgumentException();
+      } else if (off < 0 || len < 0 || len > b.length - off) {
+        throw new IndexOutOfBoundsException();
+      } else if (len == 0) {
+        return 0;
+      }
+
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      int status = doBufferAvailabilityCheck();
+      if (status == -1) {
+        return status;
+      }
+
+      int byteRead = 0;
+      long availableBytes = validDataHoldingSize - off;
+      long requestedBytes = bufferOffset + len - off;
+      if (requestedBytes <= availableBytes) {
+        System.arraycopy(data, bufferOffset, b, off, len);
+        bufferOffset += len;
+        byteRead = len;
+      } else {
+        byteRead = super.read(b, off, len);
+      }
+
+      if (statistics != null) {
+        statistics.incrementBytesRead(byteRead);
+      }
+
+      return byteRead;
+    }
+
+    private int doBufferAvailabilityCheck() throws IOException {
+      if (state == StreamState.Initial) {
+        validDataHoldingSize = fill(nextFileOffset);
+      }
+
+      long dataReloadSize = 0;
+      switch ((int) validDataHoldingSize) {
+      case -1:
+        state = StreamState.StreamEnd;
+        return -1;
+      case 0:
+        dataReloadSize = fill(nextFileOffset);
+        if (dataReloadSize <= 0) {
+          state = StreamState.StreamEnd;
+          return (int) dataReloadSize;
+        } else {
+          validDataHoldingSize = dataReloadSize;
+        }
+        break;
+      default:
+        break;
+      }
+
+      if (bufferOffset >= validDataHoldingSize) {
+        dataReloadSize = fill(nextFileOffset);
+      }
+
+      if (bufferOffset >= ((dataReloadSize == 0) ?
+          validDataHoldingSize :
+          dataReloadSize)) {
+        state = StreamState.StreamEnd;
+        return -1;
+      }
+
+      validDataHoldingSize = ((dataReloadSize == 0) ?
+          validDataHoldingSize :
+          dataReloadSize);
+      state = StreamState.DataCachedInLocalBuffer;
+      return 0;
+    }
+
+    private long fill(final long off) throws IOException {
+      if (state == StreamState.StreamEnd) {
+        return -1;
+      }
+
+      if (fileSize <= off) {
+        state = StreamState.StreamEnd;
+        return -1;
+      }
+      int len = maxBufferSize;
+      long fileOffset = 0;
+      boolean isEntireFileCached = true;
+      if ((fileSize <= maxBufferSize)) {
+        len = (int) fileSize;
+        currentFileOffset = 0;
+        nextFileOffset = 0;
+      } else {
+        if (len > (fileSize - off)) {
+          len = (int) (fileSize - off);
+        }
+
+        synchronized (BufferManager.getLock()) {
+          if (BufferManager.getInstance()
+              .hasValidDataForOffset(fsPath.toString(), off)) {
+            len = (int) (
+                BufferManager.getInstance().getBufferOffset() + BufferManager
+                    .getInstance().getBufferSize() - (int) off);
+          }
+        }
+
+        if (len <= 0) {
+          len = maxBufferSize;
+        }
+        fileOffset = off;
+        isEntireFileCached = false;
+      }
+
+      data = null;
+      BufferManager bm = BufferManager.getInstance();
+      data = bm.getEmpty(len);
+      boolean fetchDataOverNetwork = false;
+      synchronized (BufferManager.getLock()) {
+        if (bm.hasData(fsPath.toString(), fileOffset, len)) {
+          try {
+            bm.get(data, fileOffset);
+            validDataHoldingSize = data.length;
+            currentFileOffset = fileOffset;
+          } catch (ArrayIndexOutOfBoundsException e) {
+            fetchDataOverNetwork = true;
+          }
+        } else {
+          fetchDataOverNetwork = true;
+        }
+      }
+
+      if (fetchDataOverNetwork) {
+        int splitSize = getSplitSize(len);
+        try {
+          validDataHoldingSize = fillDataConcurrently(data, len, fileOffset,
+              splitSize);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted filling buffer", e);
+        }
+
+        synchronized (BufferManager.getLock()) {
+          bm.add(data, fsPath.toString(), fileOffset);
+        }
+        currentFileOffset = nextFileOffset;
+      }
+
+      nextFileOffset += validDataHoldingSize;
+      state = StreamState.DataCachedInLocalBuffer;
+      bufferOffset = isEntireFileCached ? (int) off : 0;
+      return validDataHoldingSize;
+    }
+
+    int getSplitSize(int size) {
+      if (size <= SIZE4MB) {
+        return 1;
+      }
+
+      // Not practical
+      if (size > maxBufferSize) {
+        size = maxBufferSize;
+      }
+
+      int equalBufferSplit = Math.max(Math.round(size / SIZE4MB), 1);
+      int splitSize = Math.min(equalBufferSplit, maxConcurrentConnection);
+      return splitSize;
+    }
+
+    @Override
+    public synchronized final void seek(long pos) throws IOException {
+      if (pos == -1) {
+        throw new IOException("Bad offset, cannot seek to " + pos);
+      }
+
+      BufferManager bm = BufferManager.getInstance();
+      synchronized (BufferManager.getLock()) {
+        if (bm.hasValidDataForOffset(fsPath.toString(), pos)) {
+          state = StreamState.DataCachedInLocalBuffer;
+        } else if (pos >= 0) {
+          state = StreamState.Initial;
+        }
+      }
+
+      long availableBytes = (currentFileOffset + validDataHoldingSize);
+
+      // Check if this position falls under buffered data
+      if (pos < currentFileOffset || availableBytes <= 0) {
+        validDataHoldingSize = 0;
+        currentFileOffset = pos;
+        nextFileOffset = pos;
+        bufferOffset = 0;
+        return;
+      }
+
+      if (pos < availableBytes && pos >= currentFileOffset) {
+        state = StreamState.DataCachedInLocalBuffer;
+        bufferOffset = (int) (pos - currentFileOffset);
+      } else {
+        validDataHoldingSize = 0;
+        currentFileOffset = pos;
+        nextFileOffset = pos;
+        bufferOffset = 0;
+      }
+    }
+
+    @Override
+    public synchronized final long getPos() throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      return bufferOffset + currentFileOffset;
+    }
+
+    @Override
+    public synchronized final int available() throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      return Integer.MAX_VALUE;
+    }
+
+    @Override
+    public final boolean seekToNewSource(long targetPos) throws IOException {
+      return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    private int fillDataConcurrently(byte[] byteArray, int length,
+        long globalOffset, int splitSize)
+        throws IOException, InterruptedException {
+      ExecutorService executor = Executors.newFixedThreadPool(splitSize);
+      subtasks = new Future[splitSize];
+      for (int i = 0; i < splitSize; i++) {
+        int offset = i * (length / splitSize);
+        int splitLength = (splitSize == (i + 1)) ?
+            (length / splitSize) + (length % splitSize) :
+            (length / splitSize);
+        subtasks[i] = executor.submit(
+            new BackgroundReadThread(byteArray, offset, splitLength,
+                globalOffset + offset));
+      }
+
+      executor.shutdown();
+      // wait until all tasks are finished
+      executor.awaitTermination(ADLConfKeys.DEFAULT_TIMEOUT_IN_SECONDS,
+          TimeUnit.SECONDS);
+
+      int totalBytePainted = 0;
+      for (int i = 0; i < splitSize; ++i) {
+        try {
+          totalBytePainted += (Integer) subtasks[i].get();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException(e.getCause());
+        } catch (ExecutionException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException(e.getCause());
+        }
+      }
+
+      if (totalBytePainted != length) {
+        throw new IOException("Expected " + length + " bytes, Got " +
+            totalBytePainted + " bytes");
+      }
+
+      return totalBytePainted;
+    }
+
+    @Override
+    public synchronized final void close() throws IOException {
+      synchronized (BufferManager.getLock()) {
+        BufferManager.getInstance().clear();
+      }
+      //need to cleanup the above code the stream and connection close doesn't
+      // happen here
+      //flag set to mark close happened, cannot use the stream once closed
+      streamIsClosed = true;
+    }
+
+    /**
+     * Reads data from the ADL backend from the specified global offset and
+     * given
+     * length. Read data from ADL backend is copied to buffer array from the
+     * offset value specified.
+     *
+     * @param buffer       Store read data from ADL backend in the buffer.
+     * @param offset       Store read data from ADL backend in the buffer
+     *                     from the
+     *                     offset.
+     * @param length       Size of the data read from the ADL backend.
+     * @param globalOffset Read data from file offset.
+     * @return Number of bytes read from the ADL backend
+     * @throws IOException For any intermittent server issues or internal
+     *                     failures.
+     */
+    private int fillUpData(byte[] buffer, int offset, int length,
+        long globalOffset) throws IOException {
+      int totalBytesRead = 0;
+      final URL offsetUrl = new URL(
+          runner + "&" + new OffsetParam(String.valueOf(globalOffset)) + "&"
+              + new LengthParam(String.valueOf(length)));
+      HttpURLConnection conn = new URLRunner(GetOpParam.Op.OPEN, offsetUrl,
+          true).run();
+      InputStream in = conn.getInputStream();
+      try {
+        int bytesRead = 0;
+        while ((bytesRead = in.read(buffer, (int) offset + totalBytesRead,
+            (int) (length - totalBytesRead))) > 0) {
+          totalBytesRead += bytesRead;
+        }
+
+        // InputStream must be fully consumed to enable http keep-alive
+        if (bytesRead == 0) {
+          // Looking for EOF marker byte needs to be read.
+          if (in.read() != -1) {
+            throw new SocketException(
+                "Server returned more than requested data.");
+          }
+        }
+      } finally {
+        in.close();
+        conn.disconnect();
+      }
+
+      return totalBytesRead;
+    }
+
+    private class BackgroundReadThread implements Callable {
+
+      private final byte[] data;
+      private int offset;
+      private int length;
+      private long globalOffset;
+
+      BackgroundReadThread(byte[] buffer, int off, int size, long position) {
+        this.data = buffer;
+        this.offset = off;
+        this.length = size;
+        this.globalOffset = position;
+      }
+
+      public Object call() throws IOException {
+        return fillUpData(data, offset, length, globalOffset);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
new file mode 100644
index 0000000..d7dce25
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
@@ -0,0 +1,37 @@
+/*
+ * 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.web.oauth2;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Exposing AccessTokenProvider publicly to extend in com.microsoft.azure
+ * .datalake package. Extended version to cache
+ * token for the process to gain performance gain.
+ */
+@Private
+@Unstable
+public abstract class PrivateCachedRefreshTokenBasedAccessTokenProvider
+    extends AccessTokenProvider {
+
+  // visibility workaround
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
new file mode 100644
index 0000000..7a9dffa
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * A distributed implementation of {@link
+ * org.apache.hadoop.hdfs.web.oauth2} for oauth2 token management support.
+ */
+package org.apache.hadoop.hdfs.web.oauth2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
new file mode 100644
index 0000000..1cc8273
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * A distributed implementation of {@link org.apache.hadoop.hdfs.web} for
+ * reading and writing files on Azure data lake file system. This
+ * implementation is derivation from the webhdfs specification.
+ */
+package org.apache.hadoop.hdfs.web;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
new file mode 100644
index 0000000..b76aaaa
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
@@ -0,0 +1,49 @@
+/*
+ * 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.web.resources;
+
+/**
+ * Query parameter to notify backend server that the all the data has been
+ * pushed to over the stream.
+ *
+ * Used in operation code Create and Append.
+ */
+public class ADLFlush extends BooleanParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = "flush";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   *
+   * @param value the parameter value.
+   */
+  public ADLFlush(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
new file mode 100644
index 0000000..6b3708f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
@@ -0,0 +1,96 @@
+/*
+ * 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.web.resources;
+
+import java.net.HttpURLConnection;
+
+/**
+ * Extended Webhdfs GetOpParam to avoid redirect operation for azure data
+ * lake storage.
+ */
+public class ADLGetOpParam extends HttpOpParam<ADLGetOpParam.Op> {
+  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   *
+   * @param str a string representation of the parameter value.
+   */
+  public ADLGetOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+
+  /**
+   * Get operations.
+   */
+  public static enum Op implements HttpOpParam.Op {
+    OPEN(false, HttpURLConnection.HTTP_OK);
+
+    private final boolean redirect;
+    private final int expectedHttpResponseCode;
+    private final boolean requireAuth;
+
+    Op(final boolean doRedirect, final int expectHttpResponseCode) {
+      this(doRedirect, expectHttpResponseCode, false);
+    }
+
+    Op(final boolean doRedirect, final int expectHttpResponseCode,
+        final boolean doRequireAuth) {
+      this.redirect = doRedirect;
+      this.expectedHttpResponseCode = expectHttpResponseCode;
+      this.requireAuth = doRequireAuth;
+    }
+
+    @Override
+    public HttpOpParam.Type getType() {
+      return HttpOpParam.Type.GET;
+    }
+
+    @Override
+    public boolean getRequireAuth() {
+      return requireAuth;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return false;
+    }
+
+    @Override
+    public boolean getRedirect() {
+      return redirect;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+}


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


[02/50] [abbrv] 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-7240
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


[24/50] [abbrv] 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-7240
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


[26/50] [abbrv] 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-7240
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


[20/50] [abbrv] 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-7240
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


[23/50] [abbrv] 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-7240
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


[13/50] [abbrv] 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-7240
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


[03/50] [abbrv] 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-7240
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


[22/50] [abbrv] 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-7240
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/50] [abbrv] hadoop git commit: YARN-5191. Renamed the newly added “download=true” option for getting logs via NMWebServices and AHSWebServices to be a better "format" option. (Xuan Gong via vinodkv)

Posted by ae...@apache.org.
YARN-5191. Renamed the newly added \u201cdownload=true\u201d option for getting logs via NMWebServices and AHSWebServices to be a better "format" option. (Xuan Gong via vinodkv)


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

Branch: refs/heads/HDFS-7240
Commit: 9378d9428f127eff7acd6c13544016cdbf2d65fb
Parents: 656c460
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Thu Jun 9 12:30:58 2016 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Thu Jun 9 12:30:58 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/webapp/util/WebAppUtils.java    | 18 ++++++++++
 .../webapp/AHSWebServices.java                  | 36 ++++++++++++--------
 .../nodemanager/webapp/NMWebServices.java       | 33 +++++++++++-------
 .../nodemanager/webapp/TestNMWebServices.java   | 15 +++++++-
 4 files changed, 74 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9378d942/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
index faf4a77..3aa773a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
@@ -24,6 +24,7 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -400,4 +401,21 @@ public class WebAppUtils {
     }
     return aid;
   }
+
+  public static String getSupportedLogContentType(String format) {
+    if (format.equalsIgnoreCase("text")) {
+      return "text/plain";
+    } else if (format.equalsIgnoreCase("octet-stream")) {
+      return "application/octet-stream";
+    }
+    return null;
+  }
+
+  public static String getDefaultLogContentType() {
+    return "text/plain";
+  }
+
+  public static List<String> listSupportedLogContentType() {
+    return Arrays.asList("text", "octet-stream");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9378d942/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 59dbd44..692b172 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
@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import com.google.common.base.Joiner;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
@@ -212,7 +213,7 @@ public class AHSWebServices extends WebServices {
       @Context HttpServletResponse res,
       @PathParam("containerid") String containerIdStr,
       @PathParam("filename") String filename,
-      @QueryParam("download") String download,
+      @QueryParam("format") String format,
       @QueryParam("size") String size) {
     init(res);
     ContainerId containerId;
@@ -223,9 +224,6 @@ public class AHSWebServices extends WebServices {
           "Invalid ContainerId: " + containerIdStr);
     }
 
-    boolean downloadFile = parseBooleanParam(download);
-
-
     final long length = parseLongParam(size);
 
     ApplicationId appId = containerId.getApplicationAttemptId()
@@ -236,7 +234,7 @@ public class AHSWebServices extends WebServices {
     } catch (Exception ex) {
       // directly find logs from HDFS.
       return sendStreamOutputResponse(appId, null, null, containerIdStr,
-          filename, downloadFile, length);
+          filename, format, length);
     }
     String appOwner = appInfo.getUser();
 
@@ -250,7 +248,7 @@ public class AHSWebServices extends WebServices {
       if (isFinishedState(appInfo.getAppState())) {
         // directly find logs from HDFS.
         return sendStreamOutputResponse(appId, appOwner, null, containerIdStr,
-            filename, downloadFile, length);
+            filename, format, length);
       }
       return createBadResponse(Status.INTERNAL_SERVER_ERROR,
           "Can not get ContainerInfo for the container: " + containerId);
@@ -270,7 +268,7 @@ public class AHSWebServices extends WebServices {
       return response.build();
     } else if (isFinishedState(appInfo.getAppState())) {
       return sendStreamOutputResponse(appId, appOwner, nodeId,
-          containerIdStr, filename, downloadFile, length);
+          containerIdStr, filename, format, length);
     } else {
       return createBadResponse(Status.NOT_FOUND,
           "The application is not at Running or Finished State.");
@@ -293,13 +291,19 @@ public class AHSWebServices extends WebServices {
     return response;
   }
 
-  private boolean parseBooleanParam(String param) {
-    return ("true").equalsIgnoreCase(param);
-  }
-
   private Response sendStreamOutputResponse(ApplicationId appId,
       String appOwner, String nodeId, String containerIdStr,
-      String fileName, boolean downloadFile, long bytes) {
+      String fileName, String format, long bytes) {
+    String contentType = WebAppUtils.getDefaultLogContentType();
+    if (format != null && !format.isEmpty()) {
+      contentType = WebAppUtils.getSupportedLogContentType(format);
+      if (contentType == null) {
+        String errorMessage = "The valid values for the parameter : format "
+            + "are " + WebAppUtils.listSupportedLogContentType();
+        return Response.status(Status.BAD_REQUEST).entity(errorMessage)
+            .build();
+      }
+    }
     StreamingOutput stream = null;
     try {
       stream = getStreamingOutput(appId, appOwner, nodeId,
@@ -313,9 +317,11 @@ public class AHSWebServices extends WebServices {
           "Can not get log for container: " + containerIdStr);
     }
     ResponseBuilder response = Response.ok(stream);
-    if (downloadFile) {
-      response.header("Content-Type", "application/octet-stream");
-    }
+    response.header("Content-Type", contentType);
+    // Sending the X-Content-Type-Options response header with the value
+    // nosniff will prevent Internet Explorer from MIME-sniffing a response
+    // away from the declared content-type.
+    response.header("X-Content-Type-Options", "nosniff");
     return response.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9378d942/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 943f3cc..efc0e7e 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
@@ -206,6 +206,10 @@ public class NMWebServices {
    *    The container ID
    * @param filename
    *    The name of the log file
+   * @param format
+   *    The content type
+   * @param size
+   *    the size of the log file
    * @return
    *    The contents of the container's log file
    */
@@ -216,7 +220,7 @@ public class NMWebServices {
   @Unstable
   public Response getLogs(@PathParam("containerid") String containerIdStr,
       @PathParam("filename") String filename,
-      @QueryParam("download") String download,
+      @QueryParam("format") String format,
       @QueryParam("size") String size) {
     ContainerId containerId;
     try {
@@ -234,8 +238,18 @@ public class NMWebServices {
     } catch (YarnException ex) {
       return Response.serverError().entity(ex.getMessage()).build();
     }
-    boolean downloadFile = parseBooleanParam(download);
     final long bytes = parseLongParam(size);
+    String contentType = WebAppUtils.getDefaultLogContentType();
+    if (format != null && !format.isEmpty()) {
+      contentType = WebAppUtils.getSupportedLogContentType(format);
+      if (contentType == null) {
+        String errorMessage = "The valid values for the parameter : format "
+            + "are " + WebAppUtils.listSupportedLogContentType();
+        return Response.status(Status.BAD_REQUEST).entity(errorMessage)
+            .build();
+      }
+    }
+
     try {
       final FileInputStream fis = ContainerLogsUtils.openLogFileForRead(
           containerIdStr, logFile, nmContext);
@@ -288,22 +302,17 @@ public class NMWebServices {
         }
       };
       ResponseBuilder resp = Response.ok(stream);
-      if (downloadFile) {
-        resp.header("Content-Type", "application/octet-stream");
-      }
+      resp.header("Content-Type", contentType);
+      // Sending the X-Content-Type-Options response header with the value
+      // nosniff will prevent Internet Explorer from MIME-sniffing a response
+      // away from the declared content-type.
+      resp.header("X-Content-Type-Options", "nosniff");
       return resp.build();
     } catch (IOException ex) {
       return Response.serverError().entity(ex.getMessage()).build();
     }
   }
 
-  private boolean parseBooleanParam(String param) {
-    if (param != null) {
-      return ("true").equalsIgnoreCase(param);
-    }
-    return false;
-  }
-
   private long parseLongParam(String bytes) {
     if (bytes == null || bytes.isEmpty()) {
       return Long.MAX_VALUE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9378d942/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 4e2feee..a4305da 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
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.AfterClass;
@@ -389,18 +390,30 @@ public class TestNMWebServices extends JerseyTestBase {
         .queryParam("size", "-10000")
         .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
     responseText = response.getEntity(String.class);
+    assertEquals("text/plain", response.getType().toString());
     assertEquals(fullTextSize, responseText.getBytes().length);
     assertEquals(logMessage, responseText);
 
     // ask and download it
     response = r.path("ws").path("v1").path("node").path("containerlogs")
-        .path(containerIdStr).path(filename).queryParam("download", "true")
+        .path(containerIdStr).path(filename)
+        .queryParam("format", "octet-stream")
         .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
     responseText = response.getEntity(String.class);
     assertEquals(logMessage, responseText);
     assertEquals(200, response.getStatus());
     assertEquals("application/octet-stream", response.getType().toString());
 
+    // specify a invalid format value
+    response = r.path("ws").path("v1").path("node").path("containerlogs")
+        .path(containerIdStr).path(filename)
+        .queryParam("format", "123")
+        .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals("The valid values for the parameter : format are "
+        + WebAppUtils.listSupportedLogContentType(), responseText);
+    assertEquals(400, response.getStatus());
+
     // ask for file that doesn't exist
     response = r.path("ws").path("v1").path("node")
         .path("containerlogs").path(containerIdStr).path("uhhh")


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


[27/50] [abbrv] hadoop git commit: HDFS-10468. HDFS read ends up ignoring an interrupt. Contributed by Jing Zhao

Posted by ae...@apache.org.
HDFS-10468. HDFS read ends up ignoring an interrupt. Contributed by Jing Zhao


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

Branch: refs/heads/HDFS-7240
Commit: be34e85e682880f46eee0310bf00ecc7d39cd5bd
Parents: c14c1b2
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Jun 7 10:48:21 2016 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Tue Jun 7 10:48:21 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 36 ++++++--
 .../java/org/apache/hadoop/hdfs/TestRead.java   | 87 ++++++++++++++++++++
 .../server/datanode/SimulatedFSDataset.java     |  4 +-
 3 files changed, 119 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/be34e85e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 2ed0abd..7f32a56 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -19,8 +19,10 @@ package org.apache.hadoop.hdfs;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
 import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -304,7 +306,7 @@ public class DFSInputStream extends FSInputStream
     try {
       Thread.sleep(waitTime);
     } catch (InterruptedException e) {
-      throw new IOException(
+      throw new InterruptedIOException(
           "Interrupted while getting the last block length.");
     }
   }
@@ -379,6 +381,7 @@ public class DFSInputStream extends FSInputStream
           return n;
         }
       } catch (IOException ioe) {
+        checkInterrupted(ioe);
         if (ioe instanceof RemoteException) {
           if (((RemoteException) ioe).unwrapRemoteException() instanceof
               ReplicaNotFoundException) {
@@ -414,7 +417,8 @@ public class DFSInputStream extends FSInputStream
         try {
           Thread.sleep(500); // delay between retries.
         } catch (InterruptedException e) {
-          throw new IOException("Interrupted while getting the length.");
+          throw new InterruptedIOException(
+              "Interrupted while getting the length.");
         }
       }
 
@@ -660,6 +664,7 @@ public class DFSInputStream extends FSInputStream
         }
         return chosenNode;
       } catch (IOException ex) {
+        checkInterrupted(ex);
         if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
           DFSClient.LOG.info("Will fetch a new encryption key and retry, "
               + "encryption key was invalid when connecting to " + targetAddr
@@ -681,6 +686,15 @@ public class DFSInputStream extends FSInputStream
     }
   }
 
+  private void checkInterrupted(IOException e) throws IOException {
+    if (Thread.currentThread().isInterrupted() &&
+        (e instanceof ClosedByInterruptException ||
+            e instanceof InterruptedIOException)) {
+      DFSClient.LOG.debug("The reading thread has been interrupted.", e);
+      throw e;
+    }
+  }
+
   protected BlockReader getBlockReader(LocatedBlock targetBlock,
       long offsetInBlock, long length, InetSocketAddress targetAddr,
       StorageType storageType, DatanodeInfo datanode) throws IOException {
@@ -948,6 +962,7 @@ public class DFSInputStream extends FSInputStream
         } catch (ChecksumException ce) {
           throw ce;
         } catch (IOException e) {
+          checkInterrupted(e);
           if (retries == 1) {
             DFSClient.LOG.warn("DFS Read", e);
           }
@@ -1044,9 +1059,12 @@ public class DFSInputStream extends FSInputStream
               // expanding time window for each failure
               timeWindow * (failures + 1) *
               ThreadLocalRandom.current().nextDouble();
-          DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
+          DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) +
+              " IOException, will wait for " + waitTime + " msec.");
           Thread.sleep((long)waitTime);
-        } catch (InterruptedException ignored) {
+        } catch (InterruptedException e) {
+          throw new InterruptedIOException(
+              "Interrupted while choosing DataNode for read.");
         }
         deadNodes.clear(); //2nd option is to remove only nodes[blockId]
         openInfo(true);
@@ -1140,7 +1158,8 @@ public class DFSInputStream extends FSInputStream
             buf, offset, corruptedBlocks);
         return;
       } catch (IOException e) {
-        // Ignore. Already processed inside the function.
+        checkInterrupted(e); // check if the read has been interrupted
+        // Ignore other IOException. Already processed inside the function.
         // Loop through to try the next node.
       }
     }
@@ -1218,6 +1237,7 @@ public class DFSInputStream extends FSInputStream
         addToDeadNodes(datanode.info);
         throw new IOException(msg);
       } catch (IOException e) {
+        checkInterrupted(e);
         if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
           DFSClient.LOG.info("Will fetch a new encryption key and retry, "
               + "encryption key was invalid when connecting to " + datanode.addr
@@ -1306,8 +1326,11 @@ public class DFSInputStream extends FSInputStream
           ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
           // continue; no need to refresh block locations
-        } catch (InterruptedException | ExecutionException e) {
+        } catch (ExecutionException e) {
           // Ignore
+        } catch (InterruptedException e) {
+          throw new InterruptedIOException(
+              "Interrupted while waiting for reading task");
         }
       } else {
         // We are starting up a 'hedged' read. We have a read already
@@ -1594,6 +1617,7 @@ public class DFSInputStream extends FSInputStream
         } catch (IOException e) {//make following read to retry
           DFSClient.LOG.debug("Exception while seek to {} from {} of {} from "
               + "{}", targetPos, getCurrentBlock(), src, currentNode, e);
+          checkInterrupted(e);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be34e85e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRead.java
index 9d38fd7..974fdf8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRead.java
@@ -19,9 +19,19 @@ package org.apache.hadoop.hdfs;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
 import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -103,4 +113,81 @@ public class TestRead {
       cluster.shutdown();
     }
   }
+
+  @Test(timeout=60000)
+  public void testInterruptReader() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
+        DelayedSimulatedFSDataset.Factory.class.getName());
+
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(conf).numDataNodes(1).build();
+    final DistributedFileSystem fs = cluster.getFileSystem();
+    try {
+      cluster.waitActive();
+      final Path file = new Path("/foo");
+      DFSTestUtil.createFile(fs, file, 1024, (short) 1, 0L);
+
+      final FSDataInputStream in = fs.open(file);
+      AtomicBoolean readInterrupted = new AtomicBoolean(false);
+      final Thread reader = new Thread(new Runnable() {
+        @Override
+        public void run() {
+          try {
+            in.read(new byte[1024], 0, 1024);
+          } catch (IOException e) {
+            if (e instanceof ClosedByInterruptException ||
+                e instanceof InterruptedIOException) {
+              readInterrupted.set(true);
+            }
+          }
+        }
+      });
+
+      reader.start();
+      Thread.sleep(1000);
+      reader.interrupt();
+      reader.join();
+
+      Assert.assertTrue(readInterrupted.get());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private static class DelayedSimulatedFSDataset extends SimulatedFSDataset {
+    private volatile boolean isDelayed = true;
+
+    DelayedSimulatedFSDataset(DataNode datanode, DataStorage storage,
+        Configuration conf) {
+      super(datanode, storage, conf);
+    }
+
+    @Override
+    public synchronized InputStream getBlockInputStream(ExtendedBlock b,
+        long seekOffset) throws IOException {
+      while (isDelayed) {
+        try {
+          this.wait();
+        } catch (InterruptedException ignored) {
+        }
+      }
+      InputStream result = super.getBlockInputStream(b);
+      IOUtils.skipFully(result, seekOffset);
+      return result;
+    }
+
+    static class Factory extends FsDatasetSpi.Factory<DelayedSimulatedFSDataset> {
+      @Override
+      public DelayedSimulatedFSDataset newInstance(DataNode datanode,
+          DataStorage storage, Configuration conf) throws IOException {
+        return new DelayedSimulatedFSDataset(datanode, storage, conf);
+      }
+
+      @Override
+      public boolean isSimulated() {
+        return true;
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be34e85e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 1fdedca..25034c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -960,8 +960,8 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     return new ReplicaHandler(binfo, null);
   }
 
-  synchronized InputStream getBlockInputStream(ExtendedBlock b
-      ) throws IOException {
+  protected synchronized InputStream getBlockInputStream(ExtendedBlock b)
+      throws IOException {
     final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
     BInfo binfo = map.get(b.getLocalBlock());
     if (binfo == null) {


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


[06/50] [abbrv] 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-7240
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


[16/50] [abbrv] 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-7240
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


[21/50] [abbrv] 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-7240
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


[32/50] [abbrv] hadoop git commit: YARN-5199. Close LogReader in in AHSWebServices#getStreamingOutput and FileInputStream in NMWebServices#getLogs. Contributed by Xuan Gong

Posted by ae...@apache.org.
YARN-5199. Close LogReader in in AHSWebServices#getStreamingOutput and
FileInputStream in NMWebServices#getLogs. 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/58be55b6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/58be55b6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/58be55b6

Branch: refs/heads/HDFS-7240
Commit: 58be55b6e07b94aa55ed87c461f3e5c04cc61630
Parents: 8554aee1b
Author: Xuan <xg...@apache.org>
Authored: Tue Jun 7 16:07:02 2016 -0700
Committer: Xuan <xg...@apache.org>
Committed: Tue Jun 7 16:07:02 2016 -0700

----------------------------------------------------------------------
 .../webapp/AHSWebServices.java                  | 155 ++++++++++---------
 .../nodemanager/webapp/NMWebServices.java       |  71 +++++----
 2 files changed, 118 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/58be55b6/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 d91ae55..59dbd44 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
@@ -40,7 +40,6 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
-
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -363,86 +362,94 @@ public class AHSWebServices extends WebServices {
           if ((nodeId == null || nodeName.contains(LogAggregationUtils
               .getNodeString(nodeId))) && !nodeName.endsWith(
               LogAggregationUtils.TMP_FILE_SUFFIX)) {
-            AggregatedLogFormat.LogReader reader =
-                new AggregatedLogFormat.LogReader(conf,
-                    thisNodeFile.getPath());
-            DataInputStream valueStream;
-            LogKey key = new LogKey();
-            valueStream = reader.next(key);
-            while (valueStream != null && !key.toString()
-                .equals(containerIdStr)) {
-              // Next container
-              key = new LogKey();
+            AggregatedLogFormat.LogReader reader = null;
+            try {
+              reader = new AggregatedLogFormat.LogReader(conf,
+                  thisNodeFile.getPath());
+              DataInputStream valueStream;
+              LogKey key = new LogKey();
               valueStream = reader.next(key);
-            }
-            if (valueStream == null) {
-              continue;
-            }
-            while (true) {
-              try {
-                String fileType = valueStream.readUTF();
-                String fileLengthStr = valueStream.readUTF();
-                long fileLength = Long.parseLong(fileLengthStr);
-                if (fileType.equalsIgnoreCase(logFile)) {
-                  StringBuilder sb = new StringBuilder();
-                  sb.append("LogType:");
-                  sb.append(fileType + "\n");
-                  sb.append("Log Upload Time:");
-                  sb.append(Times.format(System.currentTimeMillis()) + "\n");
-                  sb.append("LogLength:");
-                  sb.append(fileLengthStr + "\n");
-                  sb.append("Log Contents:\n");
-                  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;
+              while (valueStream != null && !key.toString()
+                  .equals(containerIdStr)) {
+                // Next container
+                key = new LogKey();
+                valueStream = reader.next(key);
+              }
+              if (valueStream == null) {
+                continue;
+              }
+              while (true) {
+                try {
+                  String fileType = valueStream.readUTF();
+                  String fileLengthStr = valueStream.readUTF();
+                  long fileLength = Long.parseLong(fileLengthStr);
+                  if (fileType.equalsIgnoreCase(logFile)) {
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("LogType:");
+                    sb.append(fileType + "\n");
+                    sb.append("Log Upload Time:");
+                    sb.append(Times.format(System.currentTimeMillis()) + "\n");
+                    sb.append("LogLength:");
+                    sb.append(fileLengthStr + "\n");
+                    sb.append("Log Contents:\n");
+                    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 skippedBytes = valueStream.skip(toSkip);
-                    if (skippedBytes != toSkip) {
-                      throw new IOException("The bytes were skipped are "
-                          + "different from the caller requested");
+
+                    long curRead = 0;
+                    long pendingRead = totalBytesToRead - curRead;
+                    int toRead = pendingRead > buf.length ? buf.length
+                        : (int) pendingRead;
+                    int len = valueStream.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 = valueStream.read(buf, 0, toRead);
                     }
+                    sb = new StringBuilder();
+                    sb.append("\nEnd of LogType:" + fileType + "\n");
+                    b = sb.toString().getBytes(Charset.forName("UTF-8"));
+                    os.write(b, 0, b.length);
+                    findLogs = true;
                   } else {
-                    if (bytes < fileLength) {
-                      totalBytesToRead = bytes;
+                    long totalSkipped = 0;
+                    long currSkipped = 0;
+                    while (currSkipped != -1 && totalSkipped < fileLength) {
+                      currSkipped = valueStream.skip(
+                          fileLength - totalSkipped);
+                      totalSkipped += currSkipped;
                     }
                   }
-
-                  long curRead = 0;
-                  long pendingRead = totalBytesToRead - curRead;
-                  int toRead = pendingRead > buf.length ? buf.length
-                      : (int) pendingRead;
-                  int len = valueStream.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 = valueStream.read(buf, 0, toRead);
-                  }
-                  sb = new StringBuilder();
-                  sb.append("\nEnd of LogType:" + fileType + "\n");
-                  b = sb.toString().getBytes(Charset.forName("UTF-8"));
-                  os.write(b, 0, b.length);
-                  findLogs = true;
-                } else {
-                  long totalSkipped = 0;
-                  long currSkipped = 0;
-                  while (currSkipped != -1 && totalSkipped < fileLength) {
-                    currSkipped = valueStream.skip(fileLength - totalSkipped);
-                    totalSkipped += currSkipped;
-                  }
+                } catch (EOFException eof) {
+                  break;
                 }
-              } catch (EOFException eof) {
-                break;
+              }
+            } finally {
+              if (reader != null) {
+                reader.close();
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58be55b6/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 e13baa7..943f3cc 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
@@ -37,7 +37,7 @@ import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
 import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.UriInfo;
-
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -236,7 +236,6 @@ public class NMWebServices {
     }
     boolean downloadFile = parseBooleanParam(download);
     final long bytes = parseLongParam(size);
-
     try {
       final FileInputStream fis = ContainerLogsUtils.openLogFileForRead(
           containerIdStr, logFile, nmContext);
@@ -246,42 +245,46 @@ public class NMWebServices {
         @Override
         public void write(OutputStream os) throws IOException,
             WebApplicationException {
-          int bufferSize = 65536;
-          byte[] buf = new byte[bufferSize];
-          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;
+          try {
+            int bufferSize = 65536;
+            byte[] buf = new byte[bufferSize];
+            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
+            long curRead = 0;
+            long pendingRead = totalBytesToRead - curRead;
+            int toRead = pendingRead > buf.length ? buf.length
                 : (int) pendingRead;
-            len = fis.read(buf, 0, toRead);
+            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();
+          } finally {
+            IOUtils.closeQuietly(fis);
           }
-          os.flush();
         }
       };
       ResponseBuilder resp = Response.ok(stream);


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


[42/50] [abbrv] hadoop git commit: HADOOP-13220. Follow on fixups after upgraded mini-kdc using Kerby. Contributed by Jiajia Li

Posted by ae...@apache.org.
HADOOP-13220. Follow on fixups after upgraded mini-kdc using Kerby. Contributed by Jiajia Li


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

Branch: refs/heads/HDFS-7240
Commit: 723432b3387fe69e6bf2b56d2ea1a7b1bda16b97
Parents: 76f0800
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Jun 9 15:56:12 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Thu Jun 9 15:56:12 2016 +0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-auth/pom.xml       |  1 -
 hadoop-common-project/hadoop-common/pom.xml     |  1 -
 .../dev-support/findbugsExcludeFile.xml         | 28 ++++++++++++++++++++
 hadoop-common-project/hadoop-minikdc/pom.xml    | 14 +++++++++-
 hadoop-project/pom.xml                          |  6 +++++
 5 files changed, 47 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-common-project/hadoop-auth/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/pom.xml b/hadoop-common-project/hadoop-auth/pom.xml
index 27e4547..93dceb3 100644
--- a/hadoop-common-project/hadoop-auth/pom.xml
+++ b/hadoop-common-project/hadoop-auth/pom.xml
@@ -134,7 +134,6 @@
     <dependency>
       <groupId>org.apache.kerby</groupId>
       <artifactId>kerb-simplekdc</artifactId>
-      <version>1.0.0-RC2</version>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 8bf052c..059986f 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -298,7 +298,6 @@
     <dependency>
       <groupId>org.apache.kerby</groupId>
       <artifactId>kerb-simplekdc</artifactId>
-      <version>1.0.0-RC2</version>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-common-project/hadoop-minikdc/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-minikdc/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-minikdc/dev-support/findbugsExcludeFile.xml
new file mode 100644
index 0000000..9a1c4a7
--- /dev/null
+++ b/hadoop-common-project/hadoop-minikdc/dev-support/findbugsExcludeFile.xml
@@ -0,0 +1,28 @@
+<!--
+   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.
+-->
+<FindBugsFilter>
+  <!--
+    Caller is not supposed to modify returned values even though there's nothing
+    stopping them; we do this for performance reasons.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.minikdc.MiniKdc" />
+    <Method name="stop" />
+    <Bug pattern="SWL_SLEEP_WITH_LOCK_HELD" />
+  </Match>
+
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-common-project/hadoop-minikdc/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-minikdc/pom.xml b/hadoop-common-project/hadoop-minikdc/pom.xml
index 2e22ad0..3075cad 100644
--- a/hadoop-common-project/hadoop-minikdc/pom.xml
+++ b/hadoop-common-project/hadoop-minikdc/pom.xml
@@ -38,7 +38,6 @@
     <dependency>
       <groupId>org.apache.kerby</groupId>
       <artifactId>kerb-simplekdc</artifactId>
-      <version>1.0.0-RC2</version>
     </dependency>
     <dependency>
       <groupId>org.slf4j</groupId>
@@ -51,4 +50,17 @@
       <scope>compile</scope>
     </dependency>
   </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml
+          </excludeFilterFile>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index aa47f6c..2b6b162 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1011,6 +1011,12 @@
             <version>1.3.0</version>
         </dependency>
 
+        <dependency>
+          <groupId>org.apache.kerby</groupId>
+          <artifactId>kerb-simplekdc</artifactId>
+          <version>1.0.0-RC2</version>
+        </dependency>
+
     </dependencies>
   </dependencyManagement>
 


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


[12/50] [abbrv] 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-7240
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


[17/50] [abbrv] 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-7240
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


[47/50] [abbrv] hadoop git commit: HADOOP-12893. Verify LICENSE.txt and NOTICE.txt. Contributed by Xiao Chen, Akira Ajisaka, and Andrew Wang.

Posted by ae...@apache.org.
HADOOP-12893. Verify LICENSE.txt and NOTICE.txt. Contributed by Xiao Chen, Akira Ajisaka, and Andrew Wang.


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

Branch: refs/heads/HDFS-7240
Commit: e383b732c54c542482b0b836e2d2c46eb49b4e2d
Parents: 58c3486
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 9 13:54:14 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 9 13:54:14 2016 -0700

----------------------------------------------------------------------
 LICENSE.txt                 | 1017 +++++++++++++++++++++++++++++++++++++-
 NOTICE.txt                  |  266 ++++++++++
 hadoop-build-tools/pom.xml  |   41 ++
 hadoop-project-dist/pom.xml |    2 +
 hadoop-project/pom.xml      |   19 +-
 pom.xml                     |   26 +
 6 files changed, 1367 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 929e2a8..44880df 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -320,7 +320,9 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
 (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
 OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
-For com.google.re2j.* classes:
+The binary distribution of this product bundles these dependencies under the
+following license:
+re2j 1.0
 ---------------------------------------------------------------------
 This is a work derived from Russ Cox's RE2 in Go, whose license
 http://golang.org/LICENSE is as follows:
@@ -548,12 +550,14 @@ hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js
 hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css
 hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css
+And the binary distribution of this product bundles these dependencies under the
+following license:
+Mockito 1.8.5
+SLF4J 1.7.10
 --------------------------------------------------------------------------------
 
 The MIT License (MIT)
 
-Copyright (c) 2011-2016 Twitter, Inc.
-
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
 in the Software without restriction, including without limitation the rights
@@ -648,3 +652,1010 @@ hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js
 
 D3 is available under a 3-clause BSD license. For details, see:
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+HSQLDB Database 2.0.0
+--------------------------------------------------------------------------------
+"COPYRIGHTS AND LICENSES (based on BSD License)
+
+For work developed by the HSQL Development Group:
+
+Copyright (c) 2001-2016, The HSQL Development Group
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation
+and/or other materials provided with the distribution.
+
+Neither the name of the HSQL Development Group nor the names of its
+contributors may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG,
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+For work originally developed by the Hypersonic SQL Group:
+
+Copyright (c) 1995-2000 by the Hypersonic SQL Group.
+All rights reserved.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation
+and/or other materials provided with the distribution.
+
+Neither the name of the Hypersonic SQL Group nor the names of its
+contributors may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP,
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+This software consists of voluntary contributions made by many individuals on behalf of the
+Hypersonic SQL Group."
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+servlet-api 2.5
+jsp-api 2.1
+Streaming API for XML 1.0
+--------------------------------------------------------------------------------
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+1.�Definitions.�
+
+1.1.�Contributor�means each individual or entity
+that creates or contributes to the creation of
+Modifications.�
+
+1.2.�Contributor Version�means the combination of the
+Original Software, prior Modifications used by a Contributor (if any), and the
+Modifications made by that particular Contributor.�
+
+1.3.�Covered
+Software�means (a) the Original Software, or (b) Modifications, or (c) the
+combination of files containing Original Software with files containing
+Modifications, in each case including portions
+thereof.�
+
+1.4.�Executable�means the Covered Software in any form other
+than Source Code.�
+
+1.5.�Initial Developer�means the individual or entity
+that first makes Original Software available under this
+License.�
+
+1.6.�Larger Work�means a work which combines Covered Software or
+portions thereof with code not governed by the terms of this
+License.�
+
+1.7.�License�means this document.�
+
+1.8.�Licensable�means
+having the right to grant, to the maximum extent possible, whether at the time
+of the initial grant or subsequently acquired, any and all of the rights
+conveyed herein.�
+
+1.9.�Modifications�means the Source Code and Executable
+form of any of the following:
+A. Any file that results from an addition to,
+deletion from or modification of the contents of a file containing Original
+Software or previous Modifications;
+B. Any new file that contains any part of the Original Software
+or previous Modification; or
+C. Any new file that is contributed or otherwise made available
+under the terms of this License.�
+
+1.10.�Original Software�means the Source Code and Executable form of
+computer software code that is originally released under this License.�
+
+1.11.�Patent Claims�means any patent claim(s), now owned or
+hereafter acquired, including without limitation, method, process, and apparatus
+claims, in any patent Licensable by grantor.�
+
+1.12.�Source Code�means (a) the common form of computer software code in which
+modifications are made and (b) associated documentation included in or
+with such code.�
+
+1.13.�You (or Your)�means an individual or a legal entity exercising rights
+under, and complying with all of the terms of, this License. For legal entities,
+You includes any entity which controls, is controlled by, or is under common control
+with You. For purposes of this definition, control means (a)�the power, direct
+or indirect, to cause the direction or management of such entity, whether by
+contract or otherwise, or (b)�ownership of more than fifty percent (50%) of the
+outstanding shares or beneficial ownership of such entity.�
+
+2. License Grants.
+
+2.1. The Initial Developer Grant. Conditioned upon Your compliance
+with Section 3.1 below and subject to third party intellectual property claims,
+the Initial Developer hereby grants You a world-wide, royalty-free,
+non-exclusive license:�
+
+(a) under intellectual property rights (other than
+patent or trademark) Licensable by Initial Developer, to use, reproduce, modify,
+display, perform, sublicense and distribute the Original Software (or portions
+thereof), with or without Modifications, and/or as part of a Larger Work;
+and�
+
+(b) under Patent Claims infringed by the making, using or selling of
+Original Software, to make, have made, use, practice, sell, and offer for sale,
+and/or otherwise dispose of the Original Software (or portions
+thereof);
+
+(c) The licenses granted in Sections�2.1(a) and (b) are
+effective on the date Initial Developer first distributes or otherwise makes the
+Original Software available to a third party under the terms of this
+License;
+
+(d) Notwithstanding Section�2.1(b) above, no patent license is
+granted: (1)�for code that You delete from the Original Software, or (2)�for
+infringements caused by: (i)�the modification of the Original Software, or
+(ii)�the combination of the Original Software with other software or
+devices.�
+
+2.2. Contributor Grant. Conditioned upon Your compliance with
+Section 3.1 below and subject to third party intellectual property claims, each
+Contributor hereby grants You a world-wide, royalty-free, non-exclusive
+license:�
+
+(a) under intellectual property rights (other than patent or
+trademark) Licensable by Contributor to use, reproduce, modify, display,
+perform, sublicense and distribute the Modifications created by such Contributor
+(or portions thereof), either on an unmodified basis, with other Modifications,
+as Covered Software and/or as part of a Larger Work; and�
+
+(b) under Patent
+Claims infringed by the making, using, or selling of Modifications made by that
+Contributor either alone and/or in combination with its Contributor Version (or
+portions of such combination), to make, use, sell, offer for sale, have made,
+and/or otherwise dispose of: (1)�Modifications made by that Contributor (or
+portions thereof); and (2)�the combination of Modifications made by that
+Contributor with its Contributor Version (or portions of such
+combination).�
+
+(c) The licenses granted in Sections�2.2(a) and 2.2(b) are
+effective on the date Contributor first distributes or otherwise makes the
+Modifications available to a third party.
+
+(d) Notwithstanding Section�2.2(b)
+above, no patent license is granted: (1)�for any code that Contributor has
+deleted from the Contributor Version; (2)�for infringements caused by:
+(i)�third party modifications of Contributor Version, or (ii)�the combination
+of Modifications made by that Contributor with other software (except as part of
+the Contributor Version) or other devices; or (3)�under Patent Claims infringed
+by Covered Software in the absence of Modifications made by that
+Contributor.�
+
+3. Distribution Obligations.�
+
+3.1. Availability of Source
+Code. Any Covered Software that You distribute or otherwise make available in
+Executable form must also be made available in Source Code form and that Source
+Code form must be distributed only under the terms of this License. You must
+include a copy of this License with every copy of the Source Code form of the
+Covered Software You distribute or otherwise make available. You must inform
+recipients of any such Covered Software in Executable form as to how they can
+obtain such Covered Software in Source Code form in a reasonable manner on or
+through a medium customarily used for software exchange.�
+
+3.2.
+Modifications. The Modifications that You create or to which You contribute are
+governed by the terms of this License. You represent that You believe Your
+Modifications are Your original creation(s) and/or You have sufficient rights to
+grant the rights conveyed by this License.�
+
+3.3. Required Notices. You must
+include a notice in each of Your Modifications that identifies You as the
+Contributor of the Modification. You may not remove or alter any copyright,
+patent or trademark notices contained within the Covered Software, or any
+notices of licensing or any descriptive text giving attribution to any
+Contributor or the Initial Developer.�
+
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code
+form that alters or restricts the applicable version of this License or the
+recipients rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipients rights in the Source Code form from the rights set forth in this
+License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.�
+
+3.6. Larger Works. You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.�
+
+4. Versions of the License.�
+
+4.1.
+New Versions. Sun Microsystems, Inc. is the initial license steward and may
+publish revised and/or new versions of this License from time to time. Each
+version will be given a distinguishing version number. Except as provided in
+Section 4.3, no one other than the license steward has the right to modify this
+License.�
+
+4.2. Effect of New Versions. You may always continue to use,
+distribute or otherwise make the Covered Software available under the terms of
+the version of the License under which You originally received the Covered
+Software. If the Initial Developer includes a notice in the Original Software
+prohibiting it from being distributed or otherwise made available under any
+subsequent version of the License, You must distribute and make the Covered
+Software available under the terms of the version of the License under which You
+originally received the Covered Software. Otherwise, You may also choose to use,
+distribute or otherwise make the Covered Software available under the terms of
+any subsequent version of the License published by the license
+steward.�
+
+4.3. Modified Versions. When You are an Initial Developer and You
+want to create a new license for Your Original Software, You may create and use
+a modified version of this License if You: (a)�rename the license and remove
+any references to the name of the license steward (except to note that the
+license differs from this License); and (b)�otherwise make it clear that the
+license contains terms which differ from this License.�
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS,
+WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT
+LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS,
+MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS
+TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY
+COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER
+OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
+CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS
+LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER
+THIS DISCLAIMER.�
+
+6. TERMINATION.�
+
+6.1. This License and the rights
+granted hereunder will terminate automatically if You fail to comply with terms
+herein and fail to cure such breach within 30 days of becoming aware of the
+breach. Provisions which, by their nature, must remain in effect beyond the
+termination of this License shall survive.�
+
+6.2. If You assert a patent
+infringement claim (excluding declaratory judgment actions) against Initial
+Developer or a Contributor (the Initial Developer or Contributor against whom
+You assert such claim is referred to as Participant) alleging that the
+Participant Software (meaning the Contributor Version where the Participant is a
+Contributor or the Original Software where the Participant is the Initial
+Developer) directly or indirectly infringes any patent, then any and all rights
+granted directly or indirectly to You by such Participant, the Initial Developer
+(if the Initial Developer is not the Participant) and all Contributors under
+Sections�2.1 and/or 2.2 of this License shall, upon 60 days notice from
+Participant terminate prospectively and automatically at the expiration of such
+60 day notice period, unless if within such 60 day period You withdraw Your
+claim with respect to the Participant Software against such Participant either
+unilaterally or pursuant to a written agreement with Participant.�
+
+6.3. In
+the event of termination under Sections�6.1 or 6.2 above, all end user licenses
+that have been validly granted by You or any distributor hereunder prior to
+termination (excluding licenses granted to You by any distributor) shall survive
+termination.�
+
+7. LIMITATION OF LIABILITY.
+UNDER NO CIRCUMSTANCES AND UNDER
+NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE,
+SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
+PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY
+CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF
+GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER
+COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
+POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO
+LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO
+THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT
+ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO
+THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.�
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a commercial item, as that term is defined in
+48�C.F.R.�2.101 (Oct. 1995), consisting of commercial computer software (as
+that term is defined at 48 C.F.R. �252.227-7014(a)(1)) and commercial computer
+software documentation as such terms are used in 48�C.F.R.�12.212 (Sept.
+1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through
+227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software
+with only those rights set forth herein. This U.S. Government Rights clause is
+in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision
+that addresses Government rights in computer software under this
+License.�
+
+9. MISCELLANEOUS.
+This License represents the complete agreement
+concerning subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdictions conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.�
+
+10. RESPONSIBILITY FOR CLAIMS.
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.�
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+Jersey 1.9
+JAXB API bundle for GlassFish V3 2.2.2
+JAXB RI 2.2.3
+--------------------------------------------------------------------------------
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
+
+1. Definitions.
+
+1.1. \u201cContributor\u201d means each individual or entity that creates or
+contributes to the creation of Modifications.
+1.2. \u201cContributor Version\u201d means the combination of the Original Software,
+prior Modifications used by a Contributor (if any), and the Modifications made
+by that particular Contributor.
+1.3. \u201cCovered Software\u201d means (a) the Original Software, or (b)
+Modifications, or (c) the combination of files containing Original Software with
+files containing Modifications, in each case including portions thereof.
+1.4. \u201cExecutable\u201d means the Covered Software in any form other than Source
+Code.
+1.5. \u201cInitial Developer\u201d means the individual or entity that first makes
+Original Software available under this License.
+1.6. \u201cLarger Work\u201d means a work which combines Covered Software or portions
+thereof with code not governed by the terms of this License.
+1.7. \u201cLicense\u201d means this document.
+1.8. \u201cLicensable\u201d means having the right to grant, to the maximum extent
+possible, whether at the time of the initial grant or subsequently acquired, any
+and all of the rights conveyed herein.
+1.9. \u201cModifications\u201d means the Source Code and Executable form of any of the
+following:
+A. Any file that results from an addition to, deletion from or modification of
+the contents of a file containing Original Software or previous Modifications;
+B. Any new file that contains any part of the Original Software or previous
+Modification; or
+C. Any new file that is contributed or otherwise made available under the terms
+of this License.
+1.10. \u201cOriginal Software\u201d means the Source Code and Executable form of
+computer software code that is originally released under this License.
+1.11. \u201cPatent Claims\u201d means any patent claim(s), now owned or hereafter
+acquired, including without limitation, method, process, and apparatus claims,
+in any patent Licensable by grantor.
+1.12. \u201cSource Code\u201d means (a) the common form of computer software code in
+which modifications are made and (b) associated documentation included in or
+with such code.
+1.13. \u201cYou\u201d (or \u201cYour\u201d) means an individual or a legal entity exercising
+rights under, and complying with all of the terms of, this License. For legal
+entities, \u201cYou\u201d includes any entity which controls, is controlled by, or is
+under common control with You. For purposes of this definition, \u201ccontrol\u201d
+means (a) the power, direct or indirect, to cause the direction or management of
+such entity, whether by contract or otherwise, or (b) ownership of more than
+fifty percent (50%) of the outstanding shares or beneficial ownership of such
+entity.
+
+2. License Grants.
+
+2.1. The Initial Developer Grant.
+
+Conditioned upon Your compliance with Section 3.1 below and subject to
+third party intellectual property claims, the Initial Developer hereby grants
+You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual
+property rights (other than patent or trademark) Licensable by Initial
+Developer, to use, reproduce, modify, display, perform, sublicense and
+distribute the Original Software (or portions thereof), with or without
+Modifications, and/or as part of a Larger Work; and
+(b) under Patent Claims
+infringed by the making, using or selling of Original Software, to make, have
+made, use, practice, sell, and offer for sale, and/or otherwise dispose of the
+Original Software (or portions thereof).
+(c) The licenses granted in Sections
+2.1(a) and (b) are effective on the date Initial Developer first distributes or
+otherwise makes the Original Software available to a third party under the terms
+of this License.
+(d) Notwithstanding Section 2.1(b) above, no patent license is
+granted: (1) for code that You delete from the Original Software, or (2) for
+infringements caused by: (i) the modification of the Original Software, or (ii)
+the combination of the Original Software with other software or devices.
+
+2.2. Contributor Grant.
+
+Conditioned upon Your compliance with Section 3.1 below and
+subject to third party intellectual property claims, each Contributor hereby
+grants You a world-wide, royalty-free, non-exclusive license:
+(a) under
+intellectual property rights (other than patent or trademark) Licensable by
+Contributor to use, reproduce, modify, display, perform, sublicense and
+distribute the Modifications created by such Contributor (or portions thereof),
+either on an unmodified basis, with other Modifications, as Covered Software
+and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the
+making, using, or selling of Modifications made by that Contributor either alone
+and/or in combination with its Contributor Version (or portions of such
+combination), to make, use, sell, offer for sale, have made, and/or otherwise
+dispose of: (1) Modifications made by that Contributor (or portions thereof);
+and (2) the combination of Modifications made by that Contributor with its
+Contributor Version (or portions of such combination).
+(c) The licenses granted
+in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first
+distributes or otherwise makes the Modifications available to a third
+party.
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted:
+(1) for any code that Contributor has deleted from the Contributor Version; (2)
+for infringements caused by: (i) third party modifications of Contributor
+Version, or (ii) the combination of Modifications made by that Contributor with
+other software (except as part of the Contributor Version) or other devices; or
+(3) under Patent Claims infringed by Covered Software in the absence of
+Modifications made by that Contributor.
+
+3. Distribution Obligations.
+
+3.1. Availability of Source Code.
+Any Covered Software that You distribute or
+otherwise make available in Executable form must also be made available in
+Source Code form and that Source Code form must be distributed only under the
+terms of this License. You must include a copy of this License with every copy
+of the Source Code form of the Covered Software You distribute or otherwise make
+available. You must inform recipients of any such Covered Software in Executable
+form as to how they can obtain such Covered Software in Source Code form in a
+reasonable manner on or through a medium customarily used for software
+exchange.
+3.2. Modifications.
+The Modifications that You create or to which
+You contribute are governed by the terms of this License. You represent that You
+believe Your Modifications are Your original creation(s) and/or You have
+sufficient rights to grant the rights conveyed by this License.
+3.3. Required Notices.
+You must include a notice in each of Your Modifications that
+identifies You as the Contributor of the Modification. You may not remove or
+alter any copyright, patent or trademark notices contained within the Covered
+Software, or any notices of licensing or any descriptive text giving attribution
+to any Contributor or the Initial Developer.
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source
+Code form that alters or restricts the applicable version of this License or the
+recipients' rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+3.5. Distribution of Executable Versions.
+You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipient's rights in the Source Code form from the rights set forth in
+this License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.
+3.6. Larger Works.
+You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.
+
+4. Versions of the License.
+
+4.1. New Versions.
+Oracle is the initial license steward and may publish revised and/or
+new versions of this License from time to time. Each version will be given a
+distinguishing version number. Except as provided in Section 4.3, no one other
+than the license steward has the right to modify this License.
+4.2. Effect of New Versions.
+You may always continue to use, distribute or otherwise make the
+Covered Software available under the terms of the version of the License under
+which You originally received the Covered Software. If the Initial Developer
+includes a notice in the Original Software prohibiting it from being distributed
+or otherwise made available under any subsequent version of the License, You
+must distribute and make the Covered Software available under the terms of the
+version of the License under which You originally received the Covered Software.
+Otherwise, You may also choose to use, distribute or otherwise make the Covered
+Software available under the terms of any subsequent version of the License
+published by the license steward.
+4.3. Modified Versions.
+When You are an
+Initial Developer and You want to create a new license for Your Original
+Software, You may create and use a modified version of this License if You: (a)
+rename the license and remove any references to the name of the license steward
+(except to note that the license differs from this License); and (b) otherwise
+make it clear that the license contains terms which differ from this
+License.
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS
+LICENSE ON AN \u201cAS IS\u201d BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED
+OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE
+IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR
+NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED
+SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY
+RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE
+COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF
+WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED
+SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will
+terminate automatically if You fail to comply with terms herein and fail to cure
+such breach within 30 days of becoming aware of the breach. Provisions which, by
+their nature, must remain in effect beyond the termination of this License shall
+survive.
+6.2. If You assert a patent infringement claim (excluding declaratory
+judgment actions) against Initial Developer or a Contributor (the Initial
+Developer or Contributor against whom You assert such claim is referred to as
+\u201cParticipant\u201d) alleging that the Participant Software (meaning the
+Contributor Version where the Participant is a Contributor or the Original
+Software where the Participant is the Initial Developer) directly or indirectly
+infringes any patent, then any and all rights granted directly or indirectly to
+You by such Participant, the Initial Developer (if the Initial Developer is not
+the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this
+License shall, upon 60 days notice from Participant terminate prospectively and
+automatically at the expiration of such 60 day notice period, unless if within
+such 60 day period You withdraw Your claim with respect to the Participant
+Software against such Participant either unilaterally or pursuant to a written
+agreement with Participant.
+6.3. If You assert a patent infringement claim
+against Participant alleging that the Participant Software directly or
+indirectly infringes any patent where such claim is resolved (such as by license
+or settlement) prior to the initiation of patent infringement litigation, then
+the reasonable value of the licenses granted by such Participant under Sections
+2.1 or 2.2 shall be taken into account in determining the amount or value of any
+payment or license.
+6.4. In the event of termination under Sections 6.1 or 6.2
+above, all end user licenses that have been validly granted by You or any
+distributor hereunder prior to termination (excluding licenses granted to You by
+any distributor) shall survive termination.
+
+7. LIMITATION OF LIABILITY.
+
+UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT
+(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL
+DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY
+SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT,
+SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING,
+WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER
+FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN
+IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS
+LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL
+INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW
+PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR
+LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND
+LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered
+Software is a \u201ccommercial item,\u201d as that term is defined in 48 C.F.R. 2.101
+(Oct. 1995), consisting of \u201ccommercial computer software\u201d (as that term is
+defined at 48 C.F.R. � 252.227-7014(a)(1)) and \u201ccommercial computer software
+documentation\u201d as such terms are used in 48 C.F.R. 12.212 (Sept. 1995).
+Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4
+(June 1995), all U.S. Government End Users acquire Covered Software with only
+those rights set forth herein. This U.S. Government Rights clause is in lieu of,
+and supersedes, any other FAR, DFAR, or other clause or provision that addresses
+Government rights in computer software under this License.
+
+9. MISCELLANEOUS.
+
+This License represents the complete agreement concerning
+subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdiction's conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys' fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+Protocol Buffer Java API 2.5.0
+--------------------------------------------------------------------------------
+This license applies to all parts of Protocol Buffers except the following:
+
+  - Atomicops support for generic gcc, located in
+    src/google/protobuf/stubs/atomicops_internals_generic_gcc.h.
+    This file is copyrighted by Red Hat Inc.
+
+  - Atomicops support for AIX/POWER, located in
+    src/google/protobuf/stubs/atomicops_internals_power.h.
+    This file is copyrighted by Bloomberg Finance LP.
+
+Copyright 2014, Google Inc.  All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it.  This code is not
+standalone and requires a support library to be linked with it.  This
+support library is itself covered by the above license.
+
+For:
+XML Commons External Components XML APIs 1.3.04
+--------------------------------------------------------------------------------
+By obtaining, using and/or copying this work, you (the licensee) agree that you
+have read, understood, and will comply with the following terms and conditions.
+
+Permission to copy, modify, and distribute this software and its documentation,
+with or without modification, for any purpose and without fee or royalty is
+hereby granted, provided that you include the following on ALL copies of the
+software and documentation or portions thereof, including modifications:
+- The full text of this NOTICE in a location viewable to users of the
+redistributed or derivative work.
+- Any pre-existing intellectual property disclaimers, notices, or terms and
+conditions. If none exist, the W3C Software Short Notice should be included
+(hypertext is preferred, text is permitted) within the body of any redistributed
+or derivative code.
+- Notice of any changes or modifications to the files, including the date changes
+were made. (We recommend you provide URIs to the location from which the code is
+derived.)
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+JUnit 4.11
+ecj-4.3.1.jar
+--------------------------------------------------------------------------------
+Eclipse Public License - v 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
+LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
+CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+1. DEFINITIONS
+
+"Contribution" means:
+
+a) in the case of the initial Contributor, the initial code and documentation
+distributed under this Agreement, and
+b) in the case of each subsequent Contributor:
+i) changes to the Program, and
+ii) additions to the Program;
+where such changes and/or additions to the Program originate from and are
+distributed by that particular Contributor. A Contribution 'originates' from a
+Contributor if it was added to the Program by such Contributor itself or anyone
+acting on such Contributor's behalf. Contributions do not include additions to
+the Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii) are not
+derivative works of the Program.
+"Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents" mean patent claims licensable by a Contributor which are
+necessarily infringed by the use or sale of its Contribution alone or when
+combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement,
+including all Contributors.
+
+2. GRANT OF RIGHTS
+
+a) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free copyright license to
+reproduce, prepare derivative works of, publicly display, publicly perform,
+distribute and sublicense the Contribution of such Contributor, if any, and such
+derivative works, in source code and object code form.
+b) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed
+Patents to make, use, sell, offer to sell, import and otherwise transfer the
+Contribution of such Contributor, if any, in source code and object code form.
+This patent license shall apply to the combination of the Contribution and the
+Program if, at the time the Contribution is added by the Contributor, such
+addition of the Contribution causes such combination to be covered by the
+Licensed Patents. The patent license shall not apply to any other combinations
+which include the Contribution. No hardware per se is licensed hereunder.
+c) Recipient understands that although each Contributor grants the licenses to
+its Contributions set forth herein, no assurances are provided by any
+Contributor that the Program does not infringe the patent or other intellectual
+property rights of any other entity. Each Contributor disclaims any liability to
+Recipient for claims brought by any other entity based on infringement of
+intellectual property rights or otherwise. As a condition to exercising the
+rights and licenses granted hereunder, each Recipient hereby assumes sole
+responsibility to secure any other intellectual property rights needed, if any.
+For example, if a third party patent license is required to allow Recipient to
+distribute the Program, it is Recipient's responsibility to acquire that license
+before distributing the Program.
+d) Each Contributor represents that to its knowledge it has sufficient copyright
+rights in its Contribution, if any, to grant the copyright license set forth in
+this Agreement.
+3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under its
+own license agreement, provided that:
+
+a) it complies with the terms and conditions of this Agreement; and
+b) its license agreement:
+i) effectively disclaims on behalf of all Contributors all warranties and
+conditions, express and implied, including warranties or conditions of title and
+non-infringement, and implied warranties or conditions of merchantability and
+fitness for a particular purpose;
+ii) effectively excludes on behalf of all Contributors all liability for
+damages, including direct, indirect, special, incidental and consequential
+damages, such as lost profits;
+iii) states that any provisions which differ from this Agreement are offered by
+that Contributor alone and not by any other party; and
+iv) states that source code for the Program is available from such Contributor,
+and informs licensees how to obtain it in a reasonable manner on or through a
+medium customarily used for software exchange.
+When the Program is made available in source code form:
+
+a) it must be made available under this Agreement; and
+b) a copy of this Agreement must be included with each copy of the Program.
+Contributors may not remove or alter any copyright notices contained within the
+Program.
+
+Each Contributor must identify itself as the originator of its Contribution, if
+any, in a manner that reasonably allows subsequent Recipients to identify the
+originator of the Contribution.
+
+4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with
+respect to end users, business partners and the like. While this license is
+intended to facilitate the commercial use of the Program, the Contributor who
+includes the Program in a commercial product offering should do so in a manner
+which does not create potential liability for other Contributors. Therefore, if
+a Contributor includes the Program in a commercial product offering, such
+Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
+every other Contributor ("Indemnified Contributor") against any losses, damages
+and costs (collectively "Losses") arising from claims, lawsuits and other legal
+actions brought by a third party against the Indemnified Contributor to the
+extent caused by the acts or omissions of such Commercial Contributor in
+connection with its distribution of the Program in a commercial product
+offering. The obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In order
+to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
+Contributor in writing of such claim, and b) allow the Commercial Contributor to
+control, and cooperate with the Commercial Contributor in, the defense and any
+related settlement negotiations. The Indemnified Contributor may participate in
+any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product
+offering, Product X. That Contributor is then a Commercial Contributor. If that
+Commercial Contributor then makes performance claims, or offers warranties
+related to Product X, those performance claims and warranties are such
+Commercial Contributor's responsibility alone. Under this section, the
+Commercial Contributor would have to defend claims against the other
+Contributors related to those performance claims and warranties, and if a court
+requires any other Contributor to pay any damages as a result, the Commercial
+Contributor must pay those damages.
+
+5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR
+IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,
+NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each
+Recipient is solely responsible for determining the appropriateness of using and
+distributing the Program and assumes all risks associated with its exercise of
+rights under this Agreement , including but not limited to the risks and costs
+of program errors, compliance with applicable laws, damage to or loss of data,
+programs or equipment, and unavailability or interruption of operations.
+
+6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
+CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST
+PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
+GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable
+law, it shall not affect the validity or enforceability of the remainder of the
+terms of this Agreement, and without further action by the parties hereto, such
+provision shall be reformed to the minimum extent necessary to make such
+provision valid and enforceable.
+
+If Recipient institutes patent litigation against any entity (including a
+cross-claim or counterclaim in a lawsuit) alleging that the Program itself
+(excluding combinations of the Program with other software or hardware)
+infringes such Recipient's patent(s), then such Recipient's rights granted under
+Section 2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to
+comply with any of the material terms or conditions of this Agreement and does
+not cure such failure in a reasonable period of time after becoming aware of
+such noncompliance. If all Recipient's rights under this Agreement terminate,
+Recipient agrees to cease use and distribution of the Program as soon as
+reasonably practicable. However, Recipient's obligations under this Agreement
+and any licenses granted by Recipient relating to the Program shall continue and
+survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in
+order to avoid inconsistency the Agreement is copyrighted and may only be
+modified in the following manner. The Agreement Steward reserves the right to
+publish new versions (including revisions) of this Agreement from time to time.
+No one other than the Agreement Steward has the right to modify this Agreement.
+The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation
+may assign the responsibility to serve as the Agreement Steward to a suitable
+separate entity. Each new version of the Agreement will be given a
+distinguishing version number. The Program (including Contributions) may always
+be distributed subject to the version of the Agreement under which it was
+received. In addition, after a new version of the Agreement is published,
+Contributor may elect to distribute the Program (including its Contributions)
+under the new version. Except as expressly stated in Sections 2(a) and 2(b)
+above, Recipient receives no rights or licenses to the intellectual property of
+any Contributor under this Agreement, whether expressly, by implication,
+estoppel or otherwise. All rights in the Program not expressly granted under
+this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the
+intellectual property laws of the United States of America. No party to this
+Agreement will bring a legal action under this Agreement more than one year
+after the cause of action arose. Each party waives its rights to a jury trial in
+any resulting litigation.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+ASM Core 3.2
+JSch 0.1.51
+ParaNamer Core 2.3
+JLine 0.9.94
+leveldbjni-all 1.8
+Hamcrest Core 1.3
+xmlenc Library 0.52
+--------------------------------------------------------------------------------
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
+    * Neither the name of the <organization> nor the
+      names of its contributors may be used to endorse or promote products
+      derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL <COPYRIGHT HOLDER> BE LIABLE FOR ANY
+DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+FindBugs-jsr305 3.0.0
+--------------------------------------------------------------------------------
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The views and conclusions contained in the software and documentation are those
+of the authors and should not be interpreted as representing official policies,
+either expressed or implied, of the FreeBSD Project.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/NOTICE.txt
----------------------------------------------------------------------
diff --git a/NOTICE.txt b/NOTICE.txt
index efa865f..63fbc9d 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -15,3 +15,269 @@ which has the following notices:
 * This product includes software developed at
   Progress Software Corporation and/or its  subsidiaries or affiliates.
 * This product includes software developed by IBM Corporation and others.
+
+The binary distribution of this product bundles binaries of
+AWS Java SDK 1.10.6,
+which has the following notices:
+ * This software includes third party software subject to the following
+ copyrights: - XML parsing and utility functions from JetS3t - Copyright
+ 2006-2009 James Murty. - JSON parsing and utility functions from JSON.org -
+ Copyright 2002 JSON.org. - PKCS#1 PEM encoded private key parsing and utility
+ functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc.
+
+The binary distribution of this product bundles binaries of
+Gson 2.2.4,
+which has the following notices:
+
+                            The Netty Project
+                            =================
+
+Please visit the Netty web site for more information:
+
+  * http://netty.io/
+
+Copyright 2014 The Netty Project
+
+The Netty Project 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.
+
+Also, please refer to each LICENSE.<component>.txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+  * LICENSE:
+    * license/LICENSE.jsr166y.txt (Public Domain)
+  * HOMEPAGE:
+    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.base64.txt (Public Domain)
+  * HOMEPAGE:
+    * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified portion of 'Webbit', an event based
+WebSocket and HTTP server, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.webbit.txt (BSD License)
+  * HOMEPAGE:
+    * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.slf4j.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.slf4j.org/
+
+This product contains a modified portion of 'ArrayDeque', written by Josh
+Bloch of Google, Inc:
+
+  * LICENSE:
+    * license/LICENSE.deque.txt (Public Domain)
+
+This product contains a modified portion of 'Apache Harmony', an open source
+Java SE, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.harmony.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://archive.apache.org/dist/harmony/
+
+This product contains a modified version of Roland Kuhn's ASL2
+AbstractNodeQueue, which is based on Dmitriy Vyukov's non-intrusive MPSC queue.
+It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.abstractnodequeue.txt (Public Domain)
+  * HOMEPAGE:
+    * https://github.com/akka/akka/blob/wip-2.2.3-for-scala-2.11/akka-actor/src/main/java/akka/dispatch/AbstractNodeQueue.java
+
+This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+and decompression library written by Matthew J. Francis. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jbzip2.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jbzip2/
+
+This product contains a modified portion of 'libdivsufsort', a C API library to construct
+the suffix array and the Burrows-Wheeler transformed string for any input string of
+a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.libdivsufsort.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/libdivsufsort/
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jctools.txt (ASL2 License)
+  * HOMEPAGE:
+    * https://github.com/JCTools/JCTools
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jzlib.txt (BSD style License)
+  * HOMEPAGE:
+    * http://www.jcraft.com/jzlib/
+
+This product optionally depends on 'Compress-LZF', a Java library for encoding and
+decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/ning/compress
+
+This product optionally depends on 'lz4', a LZ4 Java compression
+and decompression library written by Adrien Grand. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lz4.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jpountz/lz4-java
+
+This product optionally depends on 'lzma-java', a LZMA Java compression
+and decompression library, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jponge/lzma-java
+
+This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+and decompression library written by William Kinney. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jfastlz.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jfastlz/
+
+This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.protobuf.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/protobuf/
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality.  It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.bouncycastle.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.bouncycastle.org/
+
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.snappy.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/snappy/
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+  * HOMEPAGE:
+    * http://www.jboss.org/jbossmarshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.caliper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://code.google.com/p/caliper/
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.log4j.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://logging.apache.org/log4j/
+
+This product optionally depends on 'Aalto XML', an ultra-high performance
+non-blocking XML processor, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://wiki.fasterxml.com/AaltoHome
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.hpack.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/twitter/hpack
+
+This product contains a modified portion of 'Apache Commons Lang', a Java library
+provides utilities for the java.lang API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://commons.apache.org/proper/commons-lang/
+
+The binary distribution of this product bundles binaries of
+Commons Codec 1.4,
+which has the following notices:
+ * src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.javacontains test data from http://aspell.net/test/orig/batch0.tab.Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+  ===============================================================================
+  The content of package org.apache.commons.codec.language.bm has been translated
+  from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+  with permission from the original authors.
+  Original source copyright:Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+The binary distribution of this product bundles binaries of
+Commons Lang 2.6,
+which has the following notices:
+ * This product includes software from the Spring Framework,under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+The binary distribution of this product bundles binaries of
+Apache Log4j 1.2.17,
+which has the following notices:
+ * ResolverUtil.java
+    Copyright 2005-2006 Tim Fennell
+  Dumbster SMTP test server
+    Copyright 2004 Jason Paul Kitchen
+  TypeUtil.java
+    Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/hadoop-build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/pom.xml b/hadoop-build-tools/pom.xml
index d08cb89..e84c94f 100644
--- a/hadoop-build-tools/pom.xml
+++ b/hadoop-build-tools/pom.xml
@@ -28,4 +28,45 @@
   <properties>
     <failIfNoTests>false</failIfNoTests>
   </properties>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-remote-resources-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>bundle</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <includes>
+            <include>META-INF/LICENSE.txt</include>
+            <include>META-INF/NOTICE.txt</include>
+          </includes>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>dummy</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index 2cccbbe..ba4c0a8 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -80,6 +80,8 @@
               <includes>
                 <include>**/*.class</include>
                 <include>webapps/**</include>
+                <include>META-INF/LICENSE.txt</include>
+                <include>META-INF/NOTICE.txt</include>
               </includes>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 35166b1..7558b04 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -111,6 +111,7 @@
     <maven-source-plugin.version>2.3</maven-source-plugin.version>
     <maven-project-info-reports-plugin.version>2.7</maven-project-info-reports-plugin.version>
     <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
+    <maven-remote-resources-plugin.version>1.5</maven-remote-resources-plugin.version>
     <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
     <exec-maven-plugin.version>1.3.1</exec-maven-plugin.version>
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
@@ -1134,7 +1135,23 @@
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
       </plugin>
-      
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-remote-resources-plugin</artifactId>
+        <version>${maven-remote-resources-plugin.version}</version>
+        <configuration>
+          <resourceBundles>
+            <resourceBundle>org.apache.hadoop:hadoop-build-tools:${project.version}</resourceBundle>
+          </resourceBundles>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>process</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
        <plugin>
          <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c908340..aec6722 100644
--- a/pom.xml
+++ b/pom.xml
@@ -412,6 +412,32 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
         <artifactId>dependency-check-maven</artifactId>
         <version>${dependency-check-maven.version}</version>
       </plugin>
+      <plugin>
+        <!-- Copy license and notice files into hadoop-build-tools'
+             metadata, which will then be bundled into jars.
+        -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy-files</id>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <target>
+                <copy todir="${project.build.directory}/../hadoop-build-tools/src/main/resources/META-INF/">
+                  <fileset dir="${project.build.directory}/../">
+                    <include name="LICENSE.txt"/>
+                    <include name="NOTICE.txt"/>
+                  </fileset>
+                </copy>
+              </target>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 


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


[36/50] [abbrv] hadoop git commit: YARN-5080. Addendum fix to the original patch to fix YARN logs CLI. Contributed by Xuan Gong

Posted by ae...@apache.org.
YARN-5080. Addendum fix to the original patch to fix YARN logs CLI. 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/5a43583c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5a43583c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5a43583c

Branch: refs/heads/HDFS-7240
Commit: 5a43583c0bbb9650ea6a9f48d9544ec3ec24b580
Parents: 3344ba7
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Wed Jun 8 09:49:55 2016 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Wed Jun 8 09:49:55 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a43583c/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 bbe636f..d62ee5e 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
@@ -278,7 +278,9 @@ public class LogsCLI extends Configured implements Tool {
       Configuration conf, String appId) throws ClientHandlerException,
       UniformInterfaceException, JSONException {
     Client webServiceClient = Client.create();
-    String webAppAddress = WebAppUtils.getRMWebAppURLWithScheme(conf);
+    String webAppAddress = WebAppUtils.getHttpSchemePrefix(conf) +
+        WebAppUtils.getWebAppBindURL(conf, YarnConfiguration.RM_BIND_HOST,
+        WebAppUtils.getRMWebAppURLWithoutScheme(conf));
     WebResource webResource = webServiceClient.resource(webAppAddress);
 
     ClientResponse response =


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


[15/50] [abbrv] 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-7240
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


[29/50] [abbrv] hadoop git commit: YARN-4837. User facing aspects of 'AM blacklisting' feature need fixing. (vinodkv via wangda)

Posted by ae...@apache.org.
YARN-4837. User facing aspects of 'AM blacklisting' feature need fixing. (vinodkv via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: 620325e81696fca140195b74929ed9eda2d5eb16
Parents: be34e85
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Jun 7 15:06:42 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Jun 7 15:06:42 2016 -0700

----------------------------------------------------------------------
 .../yarn/api/records/AMBlackListingRequest.java |  67 -----
 .../records/ApplicationSubmissionContext.java   |  23 --
 .../hadoop/yarn/conf/YarnConfiguration.java     |  25 +-
 .../src/main/proto/yarn_protos.proto            |   5 -
 .../yarn/conf/TestYarnConfigurationFields.java  |   7 +
 .../impl/pb/AMBlackListingRequestPBImpl.java    | 104 --------
 .../pb/ApplicationSubmissionContextPBImpl.java  |  40 ---
 .../src/main/resources/yarn-default.xml         |  19 --
 .../hadoop/yarn/api/TestPBImplRecords.java      |  10 -
 .../blacklist/BlacklistManager.java             |   9 +-
 .../blacklist/BlacklistUpdates.java             |  47 ----
 .../blacklist/DisabledBlacklistManager.java     |  12 +-
 .../blacklist/SimpleBlacklistManager.java       |  17 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |  79 ++----
 .../rmapp/attempt/RMAppAttempt.java             |   2 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |  85 +++++--
 .../scheduler/AbstractYarnScheduler.java        |   2 +-
 .../scheduler/AppSchedulingInfo.java            |  74 +++---
 .../scheduler/SchedulerAppUtils.java            |  16 +-
 .../scheduler/SchedulerApplicationAttempt.java  |  33 ++-
 .../scheduler/capacity/CapacityScheduler.java   |  11 +-
 .../allocator/RegularContainerAllocator.java    |   2 +-
 .../scheduler/fair/FSLeafQueue.java             |   2 +-
 .../scheduler/fair/FairScheduler.java           |   8 +-
 .../scheduler/fifo/FifoScheduler.java           |  12 +-
 .../webapp/RMAppAttemptBlock.java               |   9 +-
 .../resourcemanager/webapp/RMAppBlock.java      |  13 +-
 .../resourcemanager/webapp/RMWebServices.java   |  21 +-
 .../webapp/dao/AMBlackListingRequestInfo.java   |  61 -----
 .../webapp/dao/AppAttemptInfo.java              |   8 +-
 .../dao/ApplicationSubmissionContextInfo.java   |  13 -
 .../TestNodeBlacklistingOnAMFailures.java       | 251 +++++++++++++++++++
 .../applicationsmanager/TestAMRestart.java      | 177 +------------
 .../blacklist/TestBlacklistManager.java         |  29 +--
 .../rmapp/TestRMAppTransitions.java             |  58 -----
 .../scheduler/TestAppSchedulingInfo.java        |  12 +-
 .../capacity/TestCapacityScheduler.java         |   8 +-
 .../scheduler/fair/TestFSAppAttempt.java        |  12 +-
 .../scheduler/fair/TestFairScheduler.java       |   9 +-
 .../TestRMWebServicesAppsModification.java      |  39 ++-
 40 files changed, 536 insertions(+), 895 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMBlackListingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMBlackListingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMBlackListingRequest.java
deleted file mode 100644
index 4aec2ba..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMBlackListingRequest.java
+++ /dev/null
@@ -1,67 +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.yarn.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.util.Records;
-
-/**
- * Specific AMBlacklistingRequest from AM to enable/disable blacklisting.
- */
-@Public
-@Evolving
-public abstract class AMBlackListingRequest {
-
-  @Private
-  @Unstable
-  public static AMBlackListingRequest newInstance(
-      boolean isAMBlackListingEnabled, float disableFailureThreshold) {
-    AMBlackListingRequest blackListRequest = Records
-        .newRecord(AMBlackListingRequest.class);
-    blackListRequest.setBlackListingEnabled(isAMBlackListingEnabled);
-    blackListRequest
-        .setBlackListingDisableFailureThreshold(disableFailureThreshold);
-    return blackListRequest;
-  }
-
-  /**
-   * @return AM Blacklisting is enabled.
-   */
-  @Public
-  @Evolving
-  public abstract boolean isAMBlackListingEnabled();
-
-  /**
-   * @return AM Blacklisting disable failure threshold
-   */
-  @Public
-  @Evolving
-  public abstract float getBlackListingDisableFailureThreshold();
-
-  @Private
-  @Unstable
-  public abstract void setBlackListingEnabled(boolean isAMBlackListingEnabled);
-
-  @Private
-  @Unstable
-  public abstract void setBlackListingDisableFailureThreshold(
-      float disableFailureThreshold);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
index fe833f7..21cd1bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
@@ -535,27 +535,4 @@ public abstract class ApplicationSubmissionContext {
   @Public
   @Unstable
   public abstract void setReservationID(ReservationId reservationID);
-
-  /**
-   * Get AM Blacklisting request object to know whether application needs any
-   * specific blacklisting for AM Nodes.
-   *
-   * @return AMBlackListingRequest object which has blacklisting information.
-   */
-  @Public
-  @Unstable
-  public abstract AMBlackListingRequest getAMBlackListRequest();
-
-  /**
-   * Get AM Blacklisting request object to know whether application needs any
-   * specific blacklisting for AM Nodes.
-   *
-   * @param blackListRequest
-   *          object which has blacklisting information such as
-   *          "enable/disable AM blacklisting" and "disable failure threshold".
-   */
-  @Public
-  @Unstable
-  public abstract void setAMBlackListRequest(
-      AMBlackListingRequest blackListRequest);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 84ec894..0beec62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2553,14 +2553,25 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS =
       30 * 60 * 1000;
 
-  public static final String AM_BLACKLISTING_ENABLED =
-      YARN_PREFIX + "am.blacklisting.enabled";
-  public static final boolean DEFAULT_AM_BLACKLISTING_ENABLED = true;
-
-  public static final String AM_BLACKLISTING_DISABLE_THRESHOLD =
-      YARN_PREFIX + "am.blacklisting.disable-failure-threshold";
-  public static final float DEFAULT_AM_BLACKLISTING_DISABLE_THRESHOLD = 0.8f;
+  @Private
+  /**
+   * This is a private feature that isn't supposed to be used by end-users.
+   */
+  public static final String AM_SCHEDULING_NODE_BLACKLISTING_ENABLED =
+      RM_PREFIX + "am-scheduling.node-blacklisting-enabled";
+  @Private
+  public static final boolean DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_ENABLED =
+      true;
 
+  @Private
+  /**
+   * This is a private feature that isn't supposed to be used by end-users.
+   */
+  public static final String AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD =
+      RM_PREFIX + "am-scheduling.node-blacklisting-disable-threshold";
+  @Private
+  public static final float
+      DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD = 0.8f;
 
   private static final String NM_SCRIPT_BASED_NODE_LABELS_PROVIDER_PREFIX =
       NM_NODE_LABELS_PROVIDER_PREFIX + "script.";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/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 ca33b28..0649f8e 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
@@ -291,10 +291,6 @@ enum ExecutionTypeProto {
   OPPORTUNISTIC = 2;
 }
 
-message AMBlackListingRequestProto {
-  optional bool blacklisting_enabled = 1 [default = false];
-  optional float blacklisting_failure_threshold = 2;
-}
 ////////////////////////////////////////////////////////////////////////
 ////// From AM_RM_Protocol /////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////
@@ -366,7 +362,6 @@ message ApplicationSubmissionContextProto {
   optional ReservationIdProto reservation_id = 15;
   optional string node_label_expression = 16;
   optional ResourceRequestProto am_container_resource_request = 17;
-  optional AMBlackListingRequestProto am_blacklisting_request = 18;
 }
 
 message LogAggregationContextProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 61b698d..2c45b87 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -92,6 +92,13 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.DEFAULT_AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
 
+    // Ignore blacklisting nodes for AM failures feature since it is still a
+    // "work in progress"
+    configurationPropsToSkipCompare.add(YarnConfiguration.
+        AM_SCHEDULING_NODE_BLACKLISTING_ENABLED);
+    configurationPropsToSkipCompare.add(YarnConfiguration.
+        AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD);
+
     // Ignore all YARN Application Timeline Service (version 1) properties
     configurationPrefixToSkipCompare.add("yarn.timeline-service.");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMBlackListingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMBlackListingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMBlackListingRequestPBImpl.java
deleted file mode 100644
index 1d04dd2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMBlackListingRequestPBImpl.java
+++ /dev/null
@@ -1,104 +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.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.AMBlackListingRequest;
-import org.apache.hadoop.yarn.proto.YarnProtos.AMBlackListingRequestProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.AMBlackListingRequestProtoOrBuilder;
-
-import com.google.protobuf.TextFormat;
-
-@Private
-@Unstable
-public class AMBlackListingRequestPBImpl extends AMBlackListingRequest {
-  AMBlackListingRequestProto proto = AMBlackListingRequestProto
-      .getDefaultInstance();
-  AMBlackListingRequestProto.Builder builder = null;
-  boolean viaProto = false;
-
-  public AMBlackListingRequestPBImpl() {
-    builder = AMBlackListingRequestProto.newBuilder();
-  }
-
-  public AMBlackListingRequestPBImpl(AMBlackListingRequestProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  public AMBlackListingRequestProto getProto() {
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = AMBlackListingRequestProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
-  @Override
-  public boolean isAMBlackListingEnabled() {
-    AMBlackListingRequestProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getBlacklistingEnabled();
-  }
-
-  @Override
-  public float getBlackListingDisableFailureThreshold() {
-    AMBlackListingRequestProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getBlacklistingFailureThreshold();
-  }
-
-  @Override
-  public void setBlackListingEnabled(boolean isAMBlackListingEnabled) {
-    maybeInitBuilder();
-    builder.setBlacklistingEnabled(isAMBlackListingEnabled);
-  }
-
-  @Override
-  public void setBlackListingDisableFailureThreshold(
-      float disableFailureThreshold) {
-    maybeInitBuilder();
-    builder.setBlacklistingFailureThreshold(disableFailureThreshold);
-  }
-
-  @Override
-  public int hashCode() {
-    return getProto().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == null) {
-      return false;
-    }
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      return this.getProto().equals(this.getClass().cast(other).getProto());
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return TextFormat.shortDebugString(getProto());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
index b39258e..67e3a84 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
@@ -24,7 +24,6 @@ import java.util.Set;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -34,7 +33,6 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.proto.YarnProtos.AMBlackListingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProtoOrBuilder;
@@ -65,7 +63,6 @@ extends ApplicationSubmissionContext {
   private ResourceRequest amResourceRequest = null;
   private LogAggregationContext logAggregationContext = null;
   private ReservationId reservationId = null;
-  private AMBlackListingRequest amBlackListRequest = null;
 
   public ApplicationSubmissionContextPBImpl() {
     builder = ApplicationSubmissionContextProto.newBuilder();
@@ -134,10 +131,6 @@ extends ApplicationSubmissionContext {
     if (this.reservationId != null) {
       builder.setReservationId(convertToProtoFormat(this.reservationId));
     }
-    if (this.amBlackListRequest != null) {
-      builder.setAmBlacklistingRequest(
-          convertToProtoFormat(this.amBlackListRequest));
-    }
   }
 
   private void mergeLocalToProto() {
@@ -420,29 +413,6 @@ extends ApplicationSubmissionContext {
     return p.getKeepContainersAcrossApplicationAttempts();
   }
 
-  @Override
-  public AMBlackListingRequest getAMBlackListRequest() {
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    if (amBlackListRequest != null) {
-      return amBlackListRequest;
-    }
-    if (!p.hasAmBlacklistingRequest()) {
-      return null;
-    }
-    amBlackListRequest = convertFromProtoFormat(p.getAmBlacklistingRequest());
-    return amBlackListRequest;
-  }
-
-  @Override
-  public void setAMBlackListRequest(AMBlackListingRequest amBlackListRequest) {
-    maybeInitBuilder();
-    if (amBlackListRequest == null) {
-      builder.clearAmBlacklistingRequest();
-      return;
-    }
-    this.amBlackListRequest = amBlackListRequest;
-  }
-
   private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
     return new PriorityPBImpl(p);
   }
@@ -485,16 +455,6 @@ extends ApplicationSubmissionContext {
     return ((ResourcePBImpl)t).getProto();
   }
 
-  private AMBlackListingRequestPBImpl convertFromProtoFormat(
-      AMBlackListingRequestProto a) {
-    return new AMBlackListingRequestPBImpl(a);
-  }
-
-  private AMBlackListingRequestProto convertToProtoFormat(
-      AMBlackListingRequest a) {
-    return ((AMBlackListingRequestPBImpl) a).getProto();
-  }
-
   @Override
   public String getNodeLabelExpression() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index b3b2e2d..eabb679 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2670,25 +2670,6 @@
     <value>4096</value>
   </property>
 
- <property>
-    <description>
-    Enable/disable blacklisting of hosts for AM based on AM failures on those
-    hosts.
-    </description>
-    <name>yarn.am.blacklisting.enabled</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <description>
-    Threshold of ratio number of NodeManager hosts that are allowed to be
-    blacklisted for AM. Beyond this ratio there is no blacklisting to avoid
-    danger of blacklisting the entire cluster.
-    </description>
-    <name>yarn.am.blacklisting.disable-failure-threshold</name>
-    <value>0.8f</value>
-  </property>
-
   <property>
     <description>
     Choose different implementation of node label's storage

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/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 91d65b1..55b1233 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
@@ -109,7 +109,6 @@ 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.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -155,7 +154,6 @@ import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
-import org.apache.hadoop.yarn.api.records.impl.pb.AMBlackListingRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptReportPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
@@ -190,7 +188,6 @@ import org.apache.hadoop.yarn.api.records.impl.pb.StrictPreemptionContractPBImpl
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.URLPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.YarnClusterMetricsPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.AMBlackListingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
@@ -515,7 +512,6 @@ public class TestPBImplRecords {
     generateByNewInstance(ResourceAllocationRequest.class);
     generateByNewInstance(ReservationAllocationState.class);
     generateByNewInstance(ResourceUtilization.class);
-    generateByNewInstance(AMBlackListingRequest.class);
   }
 
   private class GetSetPair {
@@ -1355,10 +1351,4 @@ public class TestPBImplRecords {
     validatePBImplRecord(CheckForDecommissioningNodesResponsePBImpl.class,
         CheckForDecommissioningNodesResponseProto.class);
   }
-
-  @Test
-  public void testAMBlackListingRequestPBImpl() throws Exception {
-    validatePBImplRecord(AMBlackListingRequestPBImpl.class,
-        AMBlackListingRequestProto.class);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistManager.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/blacklist/BlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistManager.java
index f03b421..f343603 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistManager.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 
 /**
  * Tracks blacklists based on failures reported on nodes.
@@ -33,14 +34,14 @@ public interface BlacklistManager {
   void addNode(String node);
 
   /**
-   * Get {@link BlacklistUpdates} that indicate which nodes should be
+   * Get {@link ResourceBlacklistRequest} that indicate which nodes should be
    * added or to removed from the blacklist.
-   * @return {@link BlacklistUpdates}
+   * @return {@link ResourceBlacklistRequest}
    */
-  BlacklistUpdates getBlacklistUpdates();
+  ResourceBlacklistRequest getBlacklistUpdates();
 
   /**
-   * Refresh the number of nodemanager hosts available for scheduling.
+   * Refresh the number of NodeManagers available for scheduling.
    * @param nodeHostCount is the number of node hosts.
    */
   void refreshNodeHostCount(int nodeHostCount);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistUpdates.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/blacklist/BlacklistUpdates.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistUpdates.java
deleted file mode 100644
index c76dfb4..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistUpdates.java
+++ /dev/null
@@ -1,47 +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.yarn.server.resourcemanager.blacklist;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-
-import java.util.List;
-
-/**
- * Class to track blacklist additions and removals.
- */
-@Private
-public class BlacklistUpdates {
-
-  private List<String> additions;
-  private List<String> removals;
-
-  public BlacklistUpdates(List<String> additions,
-      List<String> removals) {
-    this.additions = additions;
-    this.removals = removals;
-  }
-
-  public List<String> getAdditions() {
-    return additions;
-  }
-
-  public List<String> getRemovals() {
-    return removals;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/DisabledBlacklistManager.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/blacklist/DisabledBlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/DisabledBlacklistManager.java
index f155b45..8bb308d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/DisabledBlacklistManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/DisabledBlacklistManager.java
@@ -20,21 +20,23 @@ package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
 import java.util.ArrayList;
 
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+
 /**
  * A {@link BlacklistManager} that returns no blacklists.
  */
-public class DisabledBlacklistManager implements BlacklistManager{
+public class DisabledBlacklistManager implements BlacklistManager {
 
   private static final ArrayList<String> EMPTY_LIST = new ArrayList<String>();
-  private BlacklistUpdates noBlacklist =
-      new BlacklistUpdates(EMPTY_LIST, EMPTY_LIST);
+  private ResourceBlacklistRequest noBlacklist =
+      ResourceBlacklistRequest.newInstance(EMPTY_LIST, EMPTY_LIST);
 
   @Override
   public void addNode(String node) {
   }
 
   @Override
-  public BlacklistUpdates getBlacklistUpdates() {
+  public ResourceBlacklistRequest getBlacklistUpdates() {
     return noBlacklist;
   }
 
@@ -42,4 +44,4 @@ public class DisabledBlacklistManager implements BlacklistManager{
   public void refreshNodeHostCount(int nodeHostCount) {
     // Do nothing
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/SimpleBlacklistManager.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/blacklist/SimpleBlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/SimpleBlacklistManager.java
index c7bd0f8..f10e885 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/SimpleBlacklistManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/SimpleBlacklistManager.java
@@ -18,14 +18,15 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+
 /**
  * Maintains a list of failed nodes and returns that as long as number of
  * blacklisted nodes is below a threshold percentage of total nodes. If more
@@ -58,8 +59,8 @@ public class SimpleBlacklistManager implements BlacklistManager {
   }
 
   @Override
-  public BlacklistUpdates getBlacklistUpdates() {
-    BlacklistUpdates ret;
+  public ResourceBlacklistRequest getBlacklistUpdates() {
+    ResourceBlacklistRequest ret;
     List<String> blacklist = new ArrayList<>(blacklistNodes);
     final int currentBlacklistSize = blacklist.size();
     final double failureThreshold = this.blacklistDisableFailureThreshold *
@@ -70,13 +71,15 @@ public class SimpleBlacklistManager implements BlacklistManager {
             "failure threshold ratio " + blacklistDisableFailureThreshold +
             " out of total usable nodes " + numberOfNodeManagerHosts);
       }
-      ret = new BlacklistUpdates(blacklist, EMPTY_LIST);
+      ret = ResourceBlacklistRequest.newInstance(blacklist, EMPTY_LIST);
     } else {
       LOG.warn("Ignoring Blacklists, blacklist size " + currentBlacklistSize
           + " is more than failure threshold ratio "
           + blacklistDisableFailureThreshold + " out of total usable nodes "
           + numberOfNodeManagerHosts);
-      ret = new BlacklistUpdates(EMPTY_LIST, blacklist);
+      // TODO: After the threshold hits, we will keep sending a long list
+      // every time a new AM is to be scheduled.
+      ret = ResourceBlacklistRequest.newInstance(EMPTY_LIST, blacklist);
     }
     return ret;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 07d5a74..53d8ae5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -391,8 +391,8 @@ public class RMAppImpl implements RMApp, Recoverable {
                                                                  stateMachine;
 
   private static final int DUMMY_APPLICATION_ATTEMPT_NUMBER = -1;
-  private static final float MINIMUM_THRESHOLD_VALUE = 0.0f;
-  private static final float MAXIMUM_THRESHOLD_VALUE = 1.0f;
+  private static final float MINIMUM_AM_BLACKLIST_THRESHOLD_VALUE = 0.0f;
+  private static final float MAXIMUM_AM_BLACKLIST_THRESHOLD_VALUE = 1.0f;
 
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
       Configuration config, String name, String user, String queue,
@@ -471,42 +471,24 @@ public class RMAppImpl implements RMApp, Recoverable {
         YarnConfiguration.RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY,
         YarnConfiguration.DEFAULT_RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY);
 
-    // amBlacklistingEnabled can be configured globally and by each
-    // application.
-    // Case 1: If AMBlackListRequest is available in submission context, we
-    // will consider only app level request (RM level configuration will be
-    // skipped).
-    // Case 2: AMBlackListRequest is available in submission context and
-    // amBlacklisting is disabled. In this case, AM blacklisting wont be
-    // enabled for this app even if this feature is enabled in RM level.
-    // Case 3: AMBlackListRequest is not available through submission context.
-    // RM level AM black listing configuration will be considered.
-    if (null != submissionContext.getAMBlackListRequest()) {
-      amBlacklistingEnabled = submissionContext.getAMBlackListRequest()
-          .isAMBlackListingEnabled();
-      blacklistDisableThreshold = 0.0f;
-      if (amBlacklistingEnabled) {
-        blacklistDisableThreshold = submissionContext.getAMBlackListRequest()
-            .getBlackListingDisableFailureThreshold();
-
-        // Verify whether blacklistDisableThreshold is valid. And for invalid
-        // threshold, reset to global level blacklistDisableThreshold
-        // configured.
-        if (blacklistDisableThreshold < MINIMUM_THRESHOLD_VALUE
-            || blacklistDisableThreshold > MAXIMUM_THRESHOLD_VALUE) {
-          blacklistDisableThreshold = conf.getFloat(
-              YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-              YarnConfiguration.DEFAULT_AM_BLACKLISTING_DISABLE_THRESHOLD);
-        }
-      }
-    } else {
-      amBlacklistingEnabled = conf.getBoolean(
-          YarnConfiguration.AM_BLACKLISTING_ENABLED,
-          YarnConfiguration.DEFAULT_AM_BLACKLISTING_ENABLED);
-      if (amBlacklistingEnabled) {
-        blacklistDisableThreshold = conf.getFloat(
-            YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-            YarnConfiguration.DEFAULT_AM_BLACKLISTING_DISABLE_THRESHOLD);
+    // amBlacklistingEnabled can be configured globally
+    // Just use the global values
+    amBlacklistingEnabled =
+        conf.getBoolean(
+          YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+          YarnConfiguration.DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_ENABLED);
+    if (amBlacklistingEnabled) {
+      blacklistDisableThreshold = conf.getFloat(
+          YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+          YarnConfiguration.
+          DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD);
+      // Verify whether blacklistDisableThreshold is valid. And for invalid
+      // threshold, reset to global level blacklistDisableThreshold
+      // configured.
+      if (blacklistDisableThreshold < MINIMUM_AM_BLACKLIST_THRESHOLD_VALUE ||
+          blacklistDisableThreshold > MAXIMUM_AM_BLACKLIST_THRESHOLD_VALUE) {
+        blacklistDisableThreshold = YarnConfiguration.
+            DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD;
       }
     }
   }
@@ -877,15 +859,16 @@ public class RMAppImpl implements RMApp, Recoverable {
   }
 
   private void createNewAttempt(ApplicationAttemptId appAttemptId) {
-    BlacklistManager currentAMBlacklist;
+    BlacklistManager currentAMBlacklistManager;
     if (currentAttempt != null) {
-      currentAMBlacklist = currentAttempt.getAMBlacklist();
+      // Transfer over the blacklist from the previous app-attempt.
+      currentAMBlacklistManager = currentAttempt.getAMBlacklistManager();
     } else {
       if (amBlacklistingEnabled) {
-        currentAMBlacklist = new SimpleBlacklistManager(
+        currentAMBlacklistManager = new SimpleBlacklistManager(
             scheduler.getNumClusterNodes(), blacklistDisableThreshold);
       } else {
-        currentAMBlacklist = new DisabledBlacklistManager();
+        currentAMBlacklistManager = new DisabledBlacklistManager();
       }
     }
     RMAppAttempt attempt =
@@ -896,7 +879,7 @@ public class RMAppImpl implements RMApp, Recoverable {
           // hardware error and NM resync) + 1) equal to the max-attempt
           // limit.
           maxAppAttempts == (getNumFailedAppAttempts() + 1), amReq,
-          currentAMBlacklist);
+          currentAMBlacklistManager);
     attempts.put(appAttemptId, attempt);
     currentAttempt = attempt;
   }
@@ -1825,16 +1808,6 @@ public class RMAppImpl implements RMApp, Recoverable {
     rmContext.getSystemMetricsPublisher().appCreated(app, startTime);
   }
 
-  @VisibleForTesting
-  public boolean isAmBlacklistingEnabled() {
-    return amBlacklistingEnabled;
-  }
-
-  @VisibleForTesting
-  public float getAmBlacklistingDisableThreshold() {
-    return blacklistDisableThreshold;
-  }
-
   @Private
   @VisibleForTesting
   public int getNextAttemptId() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.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/RMAppAttempt.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/RMAppAttempt.java
index f2e60d1..cfd91e9 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/RMAppAttempt.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/RMAppAttempt.java
@@ -190,7 +190,7 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
    * Get the {@link BlacklistManager} that manages blacklists for AM failures
    * @return the {@link BlacklistManager} that tracks AM failures.
    */
-  BlacklistManager getAMBlacklist();
+  BlacklistManager getAMBlacklistManager();
 
   /**
    * the start time of the application.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/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 75090fe..d210b53 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
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -73,7 +74,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager;
-import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
@@ -492,7 +492,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       ApplicationMasterService masterService,
       ApplicationSubmissionContext submissionContext,
       Configuration conf, boolean maybeLastAttempt, ResourceRequest amReq,
-      BlacklistManager amBlacklist) {
+      BlacklistManager amBlacklistManager) {
     this.conf = conf;
     this.applicationAttemptId = appAttemptId;
     this.rmContext = rmContext;
@@ -511,9 +511,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
     this.attemptMetrics =
         new RMAppAttemptMetrics(applicationAttemptId, rmContext);
-    
+
     this.amReq = amReq;
-    this.blacklistedNodesForAM = amBlacklist;
+    this.blacklistedNodesForAM = amBlacklistManager;
   }
 
   @Override
@@ -1031,15 +1031,15 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         appAttempt.amReq.setResourceName(ResourceRequest.ANY);
         appAttempt.amReq.setRelaxLocality(true);
 
-        appAttempt.getAMBlacklist().refreshNodeHostCount(
+        appAttempt.getAMBlacklistManager().refreshNodeHostCount(
             appAttempt.scheduler.getNumClusterNodes());
 
-        BlacklistUpdates amBlacklist = appAttempt.getAMBlacklist()
-            .getBlacklistUpdates();
+        ResourceBlacklistRequest amBlacklist =
+            appAttempt.getAMBlacklistManager().getBlacklistUpdates();
         if (LOG.isDebugEnabled()) {
           LOG.debug("Using blacklist for AM: additions(" +
-              amBlacklist.getAdditions() + ") and removals(" +
-              amBlacklist.getRemovals() + ")");
+              amBlacklist.getBlacklistAdditions() + ") and removals(" +
+              amBlacklist.getBlacklistRemovals() + ")");
         }
         // AM resource has been checked when submission
         Allocation amContainerAllocation =
@@ -1047,8 +1047,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
                 appAttempt.applicationAttemptId,
                 Collections.singletonList(appAttempt.amReq),
                 EMPTY_CONTAINER_RELEASE_LIST,
-                amBlacklist.getAdditions(),
-                amBlacklist.getRemovals(), null, null);
+                amBlacklist.getBlacklistAdditions(),
+                amBlacklist.getBlacklistRemovals(), null, null);
         if (amContainerAllocation != null
             && amContainerAllocation.getContainers() != null) {
           assert (amContainerAllocation.getContainers().size() == 0);
@@ -1481,9 +1481,36 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
-  private boolean shouldCountTowardsNodeBlacklisting(int exitStatus) {
-    return !(exitStatus == ContainerExitStatus.SUCCESS
-        || exitStatus == ContainerExitStatus.PREEMPTED);
+  private static boolean shouldCountTowardsNodeBlacklisting(int exitStatus) {
+    switch (exitStatus) {
+    case ContainerExitStatus.PREEMPTED:
+    case ContainerExitStatus.KILLED_BY_RESOURCEMANAGER:
+    case ContainerExitStatus.KILLED_BY_APPMASTER:
+    case ContainerExitStatus.KILLED_AFTER_APP_COMPLETION:
+    case ContainerExitStatus.ABORTED:
+      // Neither the app's fault nor the system's fault. This happens by design,
+      // so no need for skipping nodes
+      return false;
+    case ContainerExitStatus.DISKS_FAILED:
+      // This container is marked with this exit-status means that the node is
+      // already marked as unhealthy given that most of the disks failed. So, no
+      // need for any explicit skipping of nodes.
+      return false;
+    case ContainerExitStatus.KILLED_EXCEEDED_VMEM:
+    case ContainerExitStatus.KILLED_EXCEEDED_PMEM:
+      // No point in skipping the node as it's not the system's fault
+      return false;
+    case ContainerExitStatus.SUCCESS:
+      return false;
+    case ContainerExitStatus.INVALID:
+      // Ideally, this shouldn't be considered for skipping a node. But in
+      // reality, it seems like there are cases where we are not setting
+      // exit-code correctly and so it's better to be conservative. See
+      // YARN-4284.
+      return true;
+    default:
+      return true;
+    }
   }
 
   private static final class UnmanagedAMAttemptSavedTransition
@@ -1803,7 +1830,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       if (appAttempt.masterContainer != null
           && appAttempt.masterContainer.getId().equals(
               containerStatus.getContainerId())) {
-        appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
+        appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
 
         // Remember the follow up transition and save the final attempt state.
         appAttempt.rememberTargetTransitionsAndStoreState(event,
@@ -1848,13 +1875,17 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
   // Add am container to the list so that am container instance will be
   // removed from NMContext.
-  private void sendAMContainerToNM(RMAppAttemptImpl appAttempt,
+  private static void amContainerFinished(RMAppAttemptImpl appAttempt,
       RMAppAttemptContainerFinishedEvent containerFinishedEvent) {
+
     NodeId nodeId = containerFinishedEvent.getNodeId();
-    if (containerFinishedEvent.getContainerStatus() != null) {
-      if (shouldCountTowardsNodeBlacklisting(containerFinishedEvent
-          .getContainerStatus().getExitStatus())) {
-        appAttempt.addAMNodeToBlackList(containerFinishedEvent.getNodeId());
+
+    ContainerStatus containerStatus =
+        containerFinishedEvent.getContainerStatus();
+    if (containerStatus != null) {
+      int exitStatus = containerStatus.getExitStatus();
+      if (shouldCountTowardsNodeBlacklisting(exitStatus)) {
+        appAttempt.addAMNodeToBlackList(nodeId);
       }
     } else {
       LOG.warn("No ContainerStatus in containerFinishedEvent");
@@ -1862,14 +1893,13 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
     if (!appAttempt.getSubmissionContext()
         .getKeepContainersAcrossApplicationAttempts()) {
-      finishedContainersSentToAM.putIfAbsent(nodeId,
+      appAttempt.finishedContainersSentToAM.putIfAbsent(nodeId,
           new ArrayList<ContainerStatus>());
-      appAttempt.finishedContainersSentToAM.get(nodeId).add(
-          containerFinishedEvent.getContainerStatus());
+      appAttempt.finishedContainersSentToAM.get(nodeId).add(containerStatus);
       appAttempt.sendFinishedContainersToNM();
     } else {
       appAttempt.sendFinishedAMContainerToNM(nodeId,
-          containerFinishedEvent.getContainerStatus().getContainerId());
+          containerStatus.getContainerId());
     }
   }
 
@@ -1884,7 +1914,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   }
 
   @Override
-  public BlacklistManager getAMBlacklist() {
+  public BlacklistManager getAMBlacklistManager() {
     return blacklistedNodesForAM;
   }
 
@@ -1943,7 +1973,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           containerStatus.getContainerId())) {
         new FinalTransition(RMAppAttemptState.FINISHED).transition(
             appAttempt, containerFinishedEvent);
-        appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
+        appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
         return RMAppAttemptState.FINISHED;
       }
       // Add all finished containers so that they can be acked to NM.
@@ -1968,7 +1998,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // Thus, we still return FINAL_SAVING state here.
       if (appAttempt.masterContainer.getId().equals(
         containerStatus.getContainerId())) {
-        appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
+
+        appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
 
         if (appAttempt.targetedFinalState.equals(RMAppAttemptState.FAILED)
             || appAttempt.targetedFinalState.equals(RMAppAttemptState.KILLED)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/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 354dcb2..3066339 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
@@ -178,7 +178,7 @@ public abstract class AbstractYarnScheduler
     NodeFilter nodeFilter = new NodeFilter() {
       @Override
       public boolean accept(SchedulerNode node) {
-        return SchedulerAppUtils.isBlacklisted(app, node, LOG);
+        return SchedulerAppUtils.isPlaceBlacklisted(app, node, LOG);
       }
     };
     return nodeTracker.getNodes(nodeFilter);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.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/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 463bebd..8d42c97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -73,9 +73,13 @@ public class AppSchedulingInfo {
   private ActiveUsersManager activeUsersManager;
   private boolean pending = true; // whether accepted/allocated by scheduler
   private ResourceUsage appResourceUsage;
+
   private AtomicBoolean userBlacklistChanged = new AtomicBoolean(false);
-  private final Set<String> amBlacklist = new HashSet<>();
-  private Set<String> userBlacklist = new HashSet<>();
+  // Set of places (nodes / racks) blacklisted by the system. Today, this only
+  // has places blacklisted for AM containers.
+  private final Set<String> placesBlacklistedBySystem = new HashSet<>();
+  private Set<String> placesBlacklistedByApp = new HashSet<>();
+
   private Set<String> requestedPartitions = new HashSet<>();
 
   final Set<Priority> priorities = new TreeSet<>(COMPARATOR);
@@ -447,32 +451,38 @@ public class AppSchedulingInfo {
   }
 
   /**
-   * The ApplicationMaster is updating the userBlacklist used for containers
-   * other than AMs.
+   * The ApplicationMaster is updating the placesBlacklistedByApp used for
+   * containers other than AMs.
    *
-   * @param blacklistAdditions resources to be added to the userBlacklist
-   * @param blacklistRemovals resources to be removed from the userBlacklist
+   * @param blacklistAdditions
+   *          resources to be added to the userBlacklist
+   * @param blacklistRemovals
+   *          resources to be removed from the userBlacklist
    */
-  public void updateBlacklist(
+  public void updatePlacesBlacklistedByApp(
       List<String> blacklistAdditions, List<String> blacklistRemovals) {
-    if (updateUserOrAMBlacklist(userBlacklist, blacklistAdditions,
+    if (updateBlacklistedPlaces(placesBlacklistedByApp, blacklistAdditions,
         blacklistRemovals)) {
       userBlacklistChanged.set(true);
     }
   }
 
   /**
-   * RM is updating blacklist for AM containers.
-   * @param blacklistAdditions resources to be added to the amBlacklist
-   * @param blacklistRemovals resources to be added to the amBlacklist
+   * Update the list of places that are blacklisted by the system. Today the
+   * system only blacklists places when it sees that AMs failed there
+   *
+   * @param blacklistAdditions
+   *          resources to be added to placesBlacklistedBySystem
+   * @param blacklistRemovals
+   *          resources to be removed from placesBlacklistedBySystem
    */
-  public void updateAMBlacklist(
+  public void updatePlacesBlacklistedBySystem(
       List<String> blacklistAdditions, List<String> blacklistRemovals) {
-    updateUserOrAMBlacklist(amBlacklist, blacklistAdditions,
+    updateBlacklistedPlaces(placesBlacklistedBySystem, blacklistAdditions,
         blacklistRemovals);
   }
 
-  boolean updateUserOrAMBlacklist(Set<String> blacklist,
+  private static boolean updateBlacklistedPlaces(Set<String> blacklist,
       List<String> blacklistAdditions, List<String> blacklistRemovals) {
     boolean changed = false;
     synchronized (blacklist) {
@@ -481,9 +491,7 @@ public class AppSchedulingInfo {
       }
 
       if (blacklistRemovals != null) {
-        if (blacklist.removeAll(blacklistRemovals)) {
-          changed = true;
-        }
+        changed = blacklist.removeAll(blacklistRemovals) || changed;
       }
     }
     return changed;
@@ -522,20 +530,24 @@ public class AppSchedulingInfo {
   }
 
   /**
-   * Returns if the node is either blacklisted by the user or the system
-   * @param resourceName the resourcename
-   * @param useAMBlacklist true if it should check amBlacklist
+   * Returns if the place (node/rack today) is either blacklisted by the
+   * application (user) or the system
+   *
+   * @param resourceName
+   *          the resourcename
+   * @param blacklistedBySystem
+   *          true if it should check amBlacklist
    * @return true if its blacklisted
    */
-  public boolean isBlacklisted(String resourceName,
-      boolean useAMBlacklist) {
-    if (useAMBlacklist){
-      synchronized (amBlacklist) {
-        return amBlacklist.contains(resourceName);
+  public boolean isPlaceBlacklisted(String resourceName,
+      boolean blacklistedBySystem) {
+    if (blacklistedBySystem){
+      synchronized (placesBlacklistedBySystem) {
+        return placesBlacklistedBySystem.contains(resourceName);
       }
     } else {
-      synchronized (userBlacklist) {
-        return userBlacklist.contains(resourceName);
+      synchronized (placesBlacklistedByApp) {
+        return placesBlacklistedByApp.contains(resourceName);
       }
     }
   }
@@ -773,12 +785,12 @@ public class AppSchedulingInfo {
   }
 
   public Set<String> getBlackList() {
-    return this.userBlacklist;
+    return this.placesBlacklistedByApp;
   }
 
   public Set<String> getBlackListCopy() {
-    synchronized (userBlacklist) {
-      return new HashSet<>(this.userBlacklist);
+    synchronized (placesBlacklistedByApp) {
+      return new HashSet<>(this.placesBlacklistedByApp);
     }
   }
 
@@ -786,7 +798,7 @@ public class AppSchedulingInfo {
       AppSchedulingInfo appInfo) {
     // This should not require locking the userBlacklist since it will not be
     // used by this instance until after setCurrentAppAttempt.
-    this.userBlacklist = appInfo.getBlackList();
+    this.placesBlacklistedByApp = appInfo.getBlackList();
   }
 
   public synchronized void recoverContainer(RMContainer rmContainer) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.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/SchedulerAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.java
index 36a1244..631da67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.java
@@ -22,20 +22,20 @@ import org.apache.commons.logging.Log;
 
 public class SchedulerAppUtils {
 
-  public static  boolean isBlacklisted(SchedulerApplicationAttempt application,
-      SchedulerNode node, Log LOG) {
-    if (application.isBlacklisted(node.getNodeName())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Skipping 'host' " + node.getNodeName() +
+  public static boolean isPlaceBlacklisted(
+      SchedulerApplicationAttempt application, SchedulerNode node, Log log) {
+    if (application.isPlaceBlacklisted(node.getNodeName())) {
+      if (log.isDebugEnabled()) {
+        log.debug("Skipping 'host' " + node.getNodeName() +
             " for " + application.getApplicationId() +
             " since it has been blacklisted");
       }
       return true;
     }
 
-    if (application.isBlacklisted(node.getRackName())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Skipping 'rack' " + node.getRackName() +
+    if (application.isPlaceBlacklisted(node.getRackName())) {
+      if (log.isDebugEnabled()) {
+        log.debug("Skipping 'rack' " + node.getRackName() +
             " for " + application.getApplicationId() +
             " since it has been blacklisted");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index ffb8657..b48b272 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -592,27 +592,26 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return (!unmanagedAM && appAttempt.getMasterContainer() == null);
   }
 
-  // Blacklist used for user containers
-  public synchronized void updateBlacklist(
-      List<String> blacklistAdditions, List<String> blacklistRemovals) {
+  public synchronized void updateBlacklist(List<String> blacklistAdditions,
+      List<String> blacklistRemovals) {
     if (!isStopped) {
-      this.appSchedulingInfo.updateBlacklist(
-          blacklistAdditions, blacklistRemovals);
-    }
-  }
-
-  // Blacklist used for AM containers
-  public synchronized void updateAMBlacklist(
-      List<String> blacklistAdditions, List<String> blacklistRemovals) {
-    if (!isStopped) {
-      this.appSchedulingInfo.updateAMBlacklist(
-          blacklistAdditions, blacklistRemovals);
+      if (isWaitingForAMContainer()) {
+        // The request is for the AM-container, and the AM-container is launched
+        // by the system. So, update the places that are blacklisted by system
+        // (as opposed to those blacklisted by the application).
+        this.appSchedulingInfo.updatePlacesBlacklistedBySystem(
+            blacklistAdditions, blacklistRemovals);
+      } else {
+        this.appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
+            blacklistRemovals);
+      }
     }
   }
 
-  public boolean isBlacklisted(String resourceName) {
-    boolean useAMBlacklist = isWaitingForAMContainer();
-    return this.appSchedulingInfo.isBlacklisted(resourceName, useAMBlacklist);
+  public boolean isPlaceBlacklisted(String resourceName) {
+    boolean forAMContainer = isWaitingForAMContainer();
+    return this.appSchedulingInfo.isPlaceBlacklisted(resourceName,
+      forAMContainer);
   }
 
   public synchronized int addMissedNonPartitionedRequestSchedulingOpportunity(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index 920e983..ee62a70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -996,15 +996,8 @@ public class CapacityScheduler extends
           application.showRequests();
         }
       }
-      
-      if (application.isWaitingForAMContainer()) {
-        // Allocate is for AM and update AM blacklist for this
-        application.updateAMBlacklist(
-            blacklistAdditions, blacklistRemovals);
-      } else {
-        application.updateBlacklist(blacklistAdditions, blacklistRemovals);
-      }
-      
+
+      application.updateBlacklist(blacklistAdditions, blacklistRemovals);
 
       allocation = application.getAllocation(getResourceCalculator(),
           getClusterResource(), getMinimumResourceCapability());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.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/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
index b2d4bbe..aae5292 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
@@ -81,7 +81,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
       ResourceLimits resourceLimits, Priority priority) {
-    if (SchedulerAppUtils.isBlacklisted(application, node, LOG)) {
+    if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
       application.updateAppSkipNodeDiagnostics(
           CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE);
       return ContainerAllocation.APP_SKIPPED;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.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/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index a398906..d8b51f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -332,7 +332,7 @@ public class FSLeafQueue extends FSQueue {
       readLock.unlock();
     }
     for (FSAppAttempt sched : pendingForResourceApps) {
-      if (SchedulerAppUtils.isBlacklisted(sched, node, LOG)) {
+      if (SchedulerAppUtils.isPlaceBlacklisted(sched, node, LOG)) {
         continue;
       }
       assigned = sched.assignContainer(node);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index c8e8406..bc953ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -993,13 +993,7 @@ public class FairScheduler extends
         preemptionContainerIds.add(container.getContainerId());
       }
 
-      if (application.isWaitingForAMContainer()) {
-        // Allocate is for AM and update AM blacklist for this
-        application.updateAMBlacklist(
-            blacklistAdditions, blacklistRemovals);
-      } else {
-        application.updateBlacklist(blacklistAdditions, blacklistRemovals);
-      }
+      application.updateBlacklist(blacklistAdditions, blacklistRemovals);
 
       List<Container> newlyAllocatedContainers =
           application.pullNewlyAllocatedContainers();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.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/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index 796b0cf..eaab495 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -375,13 +375,7 @@ public class FifoScheduler extends
             " #ask=" + ask.size());
       }
 
-      if (application.isWaitingForAMContainer()) {
-        // Allocate is for AM and update AM blacklist for this
-        application.updateAMBlacklist(
-            blacklistAdditions, blacklistRemovals);
-      } else {
-        application.updateBlacklist(blacklistAdditions, blacklistRemovals);
-      }
+      application.updateBlacklist(blacklistAdditions, blacklistRemovals);
 
       Resource headroom = application.getHeadroom();
       application.setApplicationHeadroomForMetrics(headroom);
@@ -516,10 +510,10 @@ public class FifoScheduler extends
       application.showRequests();
       synchronized (application) {
         // Check if this resource is on the blacklist
-        if (SchedulerAppUtils.isBlacklisted(application, node, LOG)) {
+        if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
           continue;
         }
-        
+
         for (Priority priority : application.getPriorities()) {
           int maxContainers = 
             getMaxAllocatableContainers(application, priority, node, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.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/webapp/RMAppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
index 6fef367..d2da8e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
@@ -220,8 +220,9 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
     String appBlacklistedNodes =
         getNodeString(rmAppAttempt.getBlacklistedNodes());
     // nodes which are blacklisted by the RM for AM launches
-    String rmBlackListedNodes = getNodeString(
-        rmAppAttempt.getAMBlacklist().getBlacklistUpdates().getAdditions());
+    String rmBlackListedNodes =
+        getNodeString(rmAppAttempt.getAMBlacklistManager()
+          .getBlacklistUpdates().getBlacklistAdditions());
 
     info("Application Attempt Overview")
       ._(
@@ -256,8 +257,8 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
         "Diagnostics Info:",
         appAttempt.getDiagnosticsInfo() == null ? "" : appAttempt
           .getDiagnosticsInfo())
-      ._("Application Blacklisted Nodes:", appBlacklistedNodes)
-      ._("RM Blacklisted Nodes(for AM launches)", rmBlackListedNodes);
+      ._("Nodes blacklisted by the application:", appBlacklistedNodes)
+      ._("Nodes blacklisted by the system:", rmBlackListedNodes);
   }
 
   private String getNodeString(Collection<String> nodes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.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/webapp/RMAppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
index 2d822c1..cfa21dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
@@ -113,10 +113,10 @@ public class RMAppBlock extends AppBlock{
     Hamlet.TBODY<Hamlet.TABLE<Hamlet>> tbody =
         html.table("#attempts").thead().tr().th(".id", "Attempt ID")
             .th(".started", "Started").th(".node", "Node").th(".logs", "Logs")
-            .th(".appBlacklistednodes", "Nodes black listed by the application",
-                "App Blacklisted Nodes")
-            .th(".rmBlacklistednodes", "Nodes black listed by the RM for the"
-                + " app", "RM Blacklisted Nodes")._()._().tbody();
+            .th(".appBlacklistednodes", "Nodes blacklisted by the application",
+                "Nodes blacklisted by the app")
+            .th(".rmBlacklistednodes", "Nodes blacklisted by the RM for the"
+                + " app", "Nodes blacklisted by the system")._()._().tbody();
 
     RMApp rmApp = this.rm.getRMContext().getRMApps().get(this.appID);
     if (rmApp == null) {
@@ -136,8 +136,9 @@ public class RMAppBlock extends AppBlock{
       // nodes which are blacklisted by the application
       String appBlacklistedNodesCount = String.valueOf(nodes.size());
       // nodes which are blacklisted by the RM for AM launches
-      String rmBlacklistedNodesCount = String.valueOf(rmAppAttempt
-          .getAMBlacklist().getBlacklistUpdates().getAdditions().size());
+      String rmBlacklistedNodesCount =
+          String.valueOf(rmAppAttempt.getAMBlacklistManager()
+            .getBlacklistUpdates().getBlacklistAdditions().size());
       String nodeLink = attemptInfo.getNodeHttpAddress();
       if (nodeLink != null) {
         nodeLink = WebAppUtils.getHttpSchemePrefix(conf) + nodeLink;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 7de1d46..d05d952 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -85,12 +85,15 @@ import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -130,7 +133,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AMBlackListingRequestInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
@@ -168,10 +170,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationReque
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateResponseInfo;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
@@ -1499,10 +1497,6 @@ public class RMWebServices extends WebServices {
           reservationIdStr);
       appContext.setReservationID(reservationId);
     }
-    if (newApp.getAMBlackListingRequestInfo() != null) {
-      appContext.setAMBlackListRequest(createAMBlackListingRequest(
-          newApp.getAMBlackListingRequestInfo()));
-    }
     return appContext;
   }
 
@@ -1651,13 +1645,6 @@ public class RMWebServices extends WebServices {
         logAggregationContextInfo.getLogAggregationPolicyParameters());
   }
 
-  private AMBlackListingRequest createAMBlackListingRequest(
-      AMBlackListingRequestInfo amBlackListingRequestInfo) {
-    return AMBlackListingRequest.newInstance(
-        amBlackListingRequestInfo.getAMBlackListingEnabled(),
-        amBlackListingRequestInfo.getBlackListingDisableFailureThreshold());
-  }
-
   @POST
   @Path("/delegation-token")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AMBlackListingRequestInfo.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/webapp/dao/AMBlackListingRequestInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AMBlackListingRequestInfo.java
deleted file mode 100644
index 0084b0b..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AMBlackListingRequestInfo.java
+++ /dev/null
@@ -1,61 +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.yarn.server.resourcemanager.webapp.dao;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * Simple class to allow users to send information required to create a
- * AMBlackListingRequest which can then be used as part of the
- * ApplicationSubmissionContext
- *
- */
-@XmlRootElement(name = "am-black-listing-requests")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class AMBlackListingRequestInfo {
-
-  @XmlElement(name = "am-black-listing-enabled")
-  boolean isAMBlackListingEnabled;
-
-  @XmlElement(name = "disable-failure-threshold")
-  float disableFailureThreshold;
-
-  public AMBlackListingRequestInfo() {
-  }
-
-  public boolean getAMBlackListingEnabled() {
-    return isAMBlackListingEnabled;
-  }
-
-  public void setAMBlackListingEnabled(boolean isAMBlackListingEnabled) {
-    this.isAMBlackListingEnabled = isAMBlackListingEnabled;
-  }
-
-  public float getBlackListingDisableFailureThreshold() {
-    return disableFailureThreshold;
-  }
-
-  public void setBlackListingDisableFailureThreshold(
-      float disableFailureThreshold) {
-    this.disableFailureThreshold = disableFailureThreshold;
-  }
-}


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


[10/50] [abbrv] 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-7240
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


[08/50] [abbrv] 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-7240
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


[50/50] [abbrv] hadoop git commit: Merge branch 'trunk' into HDFS-7240

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


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

Branch: refs/heads/HDFS-7240
Commit: 23923086d1aff381fcf3a0d0e1dd19ed029bda18
Parents: 54f7975 9581fb7
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Jun 9 20:34:39 2016 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Thu Jun 9 20:34:39 2016 -0700

----------------------------------------------------------------------
 LICENSE.txt                                     | 1017 +++++++++++++++-
 NOTICE.txt                                      |  266 +++++
 dev-support/bin/qbt                             |   18 +
 dev-support/bin/yetus-wrapper                   |    2 +-
 dev-support/checkstyle/suppressions.xml         |   21 +
 hadoop-build-tools/pom.xml                      |   41 +
 hadoop-common-project/hadoop-auth/pom.xml       |   27 +-
 .../authentication/util/KerberosUtil.java       |   18 +-
 .../TestKerberosAuthenticationHandler.java      |    1 -
 .../authentication/util/TestKerberosUtil.java   |   26 +-
 .../hadoop-common/HadoopCommon.cmake            |    9 +-
 .../dev-support/findbugsExcludeFile.xml         |    8 +-
 hadoop-common-project/hadoop-common/pom.xml     |    4 +
 .../KeyProviderDelegationTokenExtension.java    |   30 +-
 .../org/apache/hadoop/crypto/key/KeyShell.java  |  182 +--
 .../crypto/key/kms/KMSClientProvider.java       |  158 ++-
 .../key/kms/LoadBalancingKMSClientProvider.java |   41 +-
 .../hadoop/crypto/key/kms/ValueQueue.java       |    2 +-
 .../hadoop/fs/CommonConfigurationKeys.java      |    3 +
 .../hadoop/fs/InvalidRequestException.java      |    4 +
 .../org/apache/hadoop/fs/LocalDirAllocator.java |  153 ++-
 .../hadoop/fs/PathAccessDeniedException.java    |   12 +-
 .../apache/hadoop/fs/PathNotFoundException.java |   20 +-
 .../hadoop/fs/PathPermissionException.java      |   16 +-
 .../apache/hadoop/io/FastByteComparisons.java   |    2 +-
 .../apache/hadoop/io/erasurecode/CodecUtil.java |  113 +-
 .../io/erasurecode/ErasureCoderOptions.java     |   89 ++
 .../erasurecode/coder/HHXORErasureDecoder.java  |   18 +-
 .../erasurecode/coder/HHXORErasureEncoder.java  |   15 +-
 .../io/erasurecode/coder/RSErasureDecoder.java  |    6 +-
 .../io/erasurecode/coder/RSErasureEncoder.java  |    6 +-
 .../io/erasurecode/coder/XORErasureDecoder.java |    6 +-
 .../io/erasurecode/coder/XORErasureEncoder.java |    6 +-
 .../rawcoder/AbstractRawErasureCoder.java       |  220 ----
 .../rawcoder/AbstractRawErasureDecoder.java     |  181 ---
 .../rawcoder/AbstractRawErasureEncoder.java     |  146 ---
 .../rawcoder/ByteArrayDecodingState.java        |  111 ++
 .../rawcoder/ByteArrayEncodingState.java        |   81 ++
 .../rawcoder/ByteBufferDecodingState.java       |  134 +++
 .../rawcoder/ByteBufferEncodingState.java       |   98 ++
 .../io/erasurecode/rawcoder/CoderOption.java    |   43 -
 .../io/erasurecode/rawcoder/CoderUtil.java      |  199 ++++
 .../io/erasurecode/rawcoder/DecodingState.java  |   55 +
 .../erasurecode/rawcoder/DummyRawDecoder.java   |   16 +-
 .../erasurecode/rawcoder/DummyRawEncoder.java   |   15 +-
 .../rawcoder/DummyRawErasureCoderFactory.java   |   10 +-
 .../io/erasurecode/rawcoder/EncodingState.java  |   44 +
 .../io/erasurecode/rawcoder/RSRawDecoder.java   |   48 +-
 .../rawcoder/RSRawDecoderLegacy.java            |   66 +-
 .../io/erasurecode/rawcoder/RSRawEncoder.java   |   45 +-
 .../rawcoder/RSRawEncoderLegacy.java            |   82 +-
 .../rawcoder/RSRawErasureCoderFactory.java      |    9 +-
 .../RSRawErasureCoderFactoryLegacy.java         |    9 +-
 .../erasurecode/rawcoder/RawErasureCoder.java   |   73 --
 .../rawcoder/RawErasureCoderFactory.java        |   11 +-
 .../erasurecode/rawcoder/RawErasureDecoder.java |  137 ++-
 .../erasurecode/rawcoder/RawErasureEncoder.java |  135 ++-
 .../io/erasurecode/rawcoder/XORRawDecoder.java  |   51 +-
 .../io/erasurecode/rawcoder/XORRawEncoder.java  |   57 +-
 .../rawcoder/XORRawErasureCoderFactory.java     |    9 +-
 .../io/erasurecode/rawcoder/package-info.java   |   38 +
 .../io/erasurecode/rawcoder/util/CoderUtil.java |   83 --
 .../erasurecode/rawcoder/util/GaloisField.java  |    4 +-
 .../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 |  124 +-
 .../apache/hadoop/ipc/DecayRpcScheduler.java    |  130 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |   65 +-
 .../java/org/apache/hadoop/log/LogLevel.java    |  285 ++++-
 .../hadoop/metrics2/impl/MetricsSystemImpl.java |    1 +
 .../metrics2/lib/DefaultMetricsSystem.java      |    9 +
 .../metrics2/sink/RollingFileSystemSink.java    |  353 +++++-
 .../java/org/apache/hadoop/security/KDiag.java  |   34 +-
 .../hadoop/security/LdapGroupsMapping.java      |   12 +
 .../hadoop/security/ShellBasedIdMapping.java    |    2 +-
 .../hadoop/security/alias/CredentialShell.java  |  203 ++--
 .../hadoop/security/token/DtFileOperations.java |   23 +
 .../hadoop/security/token/DtUtilShell.java      |   47 +-
 .../DelegationTokenAuthenticationHandler.java   |    7 +
 .../web/DelegationTokenAuthenticator.java       |   19 +
 .../org/apache/hadoop/tools/CommandShell.java   |    4 +-
 .../org/apache/hadoop/tracing/TraceAdmin.java   |   16 +-
 .../org/apache/hadoop/util/HostsFileReader.java |  159 ++-
 .../java/org/apache/hadoop/util/KMSUtil.java    |   76 ++
 .../org/apache/hadoop/util/NativeCrc32.java     |    6 +-
 .../apache/hadoop/util/concurrent/AsyncGet.java |   65 +
 .../hadoop/util/concurrent/AsyncGetFuture.java  |   73 ++
 ...rg.apache.hadoop.security.token.TokenRenewer |   14 +
 .../src/main/resources/core-default.xml         |  109 +-
 .../src/site/markdown/CommandsManual.md         |   32 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |    1 +
 .../src/site/markdown/SingleCluster.md.vm       |   15 +-
 .../hadoop-common/src/site/markdown/Tracing.md  |    9 +
 .../conf/TestCommonConfigurationFields.java     |    6 +
 .../kms/TestLoadBalancingKMSClientProvider.java |   94 ++
 .../fs/FileContextCreateMkdirBaseTest.java      |   73 +-
 .../hadoop/fs/contract/ContractTestUtils.java   |  420 +++++++
 .../erasurecode/TestCodecRawCoderMapping.java   |   29 +-
 .../hadoop/io/erasurecode/TestCoderBase.java    |   14 +-
 .../erasurecode/rawcoder/TestDummyRawCoder.java |    2 +-
 .../erasurecode/rawcoder/TestRawCoderBase.java  |   50 +-
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     |  130 +-
 .../java/org/apache/hadoop/ipc/TestRPC.java     |   31 +-
 .../org/apache/hadoop/log/TestLogLevel.java     |  479 ++++++--
 .../sink/RollingFileSystemSinkTestBase.java     |   24 +-
 .../sink/TestRollingFileSystemSink.java         |  285 +++--
 .../TestRollingFileSystemSinkWithLocal.java     |  157 +++
 .../org/apache/hadoop/net/ServerSocketUtil.java |   39 +
 .../hadoop/security/TestLdapGroupsMapping.java  |  140 +++
 .../security/TestShellBasedIdMapping.java       |   16 +-
 .../hadoop/security/token/TestDtUtilShell.java  |   24 +
 .../delegation/web/TestWebDelegationToken.java  |  114 +-
 .../apache/hadoop/util/TestHostsFileReader.java |   26 +
 .../hadoop-common/src/test/resources/krb5.conf  |   23 +-
 .../key/kms/server/KMSAuthenticationFilter.java |    2 +-
 .../hadoop/crypto/key/kms/server/TestKMS.java   |  134 ++-
 .../dev-support/findbugsExcludeFile.xml         |   28 +
 hadoop-common-project/hadoop-minikdc/pom.xml    |  119 +-
 .../java/org/apache/hadoop/minikdc/MiniKdc.java |  381 ++----
 .../src/main/resources/minikdc-krb5.conf        |   25 -
 .../src/main/resources/minikdc.ldiff            |   47 -
 .../org/apache/hadoop/minikdc/TestMiniKdc.java  |   19 +-
 .../hadoop/hdfs/AsyncDistributedFileSystem.java |   86 +-
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |   39 +-
 .../hadoop/hdfs/DFSStripedInputStream.java      |    7 +-
 .../hadoop/hdfs/DFSStripedOutputStream.java     |    7 +-
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   38 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |    3 +
 .../apache/hadoop/hdfs/FileChecksumHelper.java  |    3 +-
 .../hadoop/hdfs/protocol/StripedBlockInfo.java  |   10 +-
 .../hdfs/protocol/datatransfer/Sender.java      |    2 +
 .../ClientNamenodeProtocolTranslatorPB.java     |   73 +-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |   16 +
 .../hdfs/web/resources/NoRedirectParam.java     |   49 +
 .../src/main/proto/datatransfer.proto           |    1 +
 .../hadoop/fs/http/server/HttpFSServer.java     |  218 ++--
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   10 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   21 +-
 .../apache/hadoop/hdfs/HDFSPolicyProvider.java  |    6 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |    1 +
 .../qjournal/server/JournalNodeRpcServer.java   |    6 +-
 .../hadoop/hdfs/server/balancer/Balancer.java   |    2 +-
 .../server/blockmanagement/BlockManager.java    |   72 +-
 .../blockmanagement/DecommissionManager.java    |    8 +-
 .../PendingReconstructionBlocks.java            |   16 +-
 .../hdfs/server/common/HdfsServerConstants.java |    1 -
 .../server/datanode/BlockChecksumHelper.java    |  172 ++-
 .../hadoop/hdfs/server/datanode/DataNode.java   |    3 +-
 .../server/datanode/ShortCircuitRegistry.java   |    6 +-
 .../erasurecode/ErasureCodingWorker.java        |   15 +-
 .../StripedBlockChecksumReconstructor.java      |  129 ++
 .../erasurecode/StripedBlockReconstructor.java  |  119 ++
 .../datanode/erasurecode/StripedReader.java     |   22 +-
 .../erasurecode/StripedReconstructionInfo.java  |   99 ++
 .../erasurecode/StripedReconstructor.java       |  176 +--
 .../datanode/erasurecode/StripedWriter.java     |   29 +-
 .../datanode/web/webhdfs/ParameterParser.java   |    5 +
 .../datanode/web/webhdfs/WebHdfsHandler.java    |   26 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |    2 +-
 .../server/namenode/EncryptionZoneManager.java  |   35 +-
 .../server/namenode/FSDirEncryptionZoneOp.java  |    2 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   11 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   49 +-
 .../server/namenode/FSPermissionChecker.java    |   24 +-
 .../hdfs/server/namenode/LeaseManager.java      |   21 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |    5 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |    2 +-
 .../web/resources/NamenodeWebHdfsMethods.java   |   82 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |   42 +-
 .../hadoop/hdfs/tools/NNHAServiceTarget.java    |    2 +-
 .../src/main/resources/hdfs-default.xml         |   18 +
 .../src/main/webapps/hdfs/dfshealth.html        |    2 +-
 .../src/main/webapps/hdfs/explorer.html         |   25 +-
 .../src/main/webapps/hdfs/explorer.js           |   53 +-
 .../src/site/markdown/HdfsPermissionsGuide.md   |    2 +-
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |   57 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |    5 +
 .../apache/hadoop/hdfs/StripedFileTestUtil.java |    8 +-
 .../org/apache/hadoop/hdfs/TestAsyncDFS.java    |  454 +++++++
 .../apache/hadoop/hdfs/TestAsyncDFSRename.java  |  572 ++-------
 .../apache/hadoop/hdfs/TestAsyncHDFSWithHA.java |  181 +++
 .../apache/hadoop/hdfs/TestDFSPermission.java   |   13 +
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |    4 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |   23 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |   16 +-
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     |   47 +-
 .../hdfs/TestDecommissionWithStriped.java       |   47 +
 .../hadoop/hdfs/TestDistributedFileSystem.java  |    1 -
 .../apache/hadoop/hdfs/TestFileChecksum.java    |   41 +-
 .../hadoop/hdfs/TestHDFSPolicyProvider.java     |  126 ++
 .../apache/hadoop/hdfs/TestMiniDFSCluster.java  |   31 -
 .../java/org/apache/hadoop/hdfs/TestRead.java   |   87 ++
 .../hdfs/server/balancer/TestBalancer.java      |   17 +-
 .../balancer/TestBalancerWithHANameNodes.java   |    2 +-
 .../TestBalancerWithMultipleNameNodes.java      |    2 +-
 .../balancer/TestBalancerWithNodeGroup.java     |    4 +-
 .../TestPendingReconstruction.java              |   20 +-
 .../server/datanode/SimulatedFSDataset.java     |    4 +-
 .../TestDataNodeErasureCodingMetrics.java       |   16 +
 .../hadoop/hdfs/server/mover/TestMover.java     |   12 +-
 .../hdfs/server/mover/TestStorageMover.java     |    2 +-
 .../hdfs/server/namenode/FSAclBaseTest.java     |   12 +-
 .../hdfs/server/namenode/TestLeaseManager.java  |   24 +-
 .../hdfs/server/namenode/ha/HATestUtil.java     |    9 +-
 .../server/namenode/ha/TestNNHealthCheck.java   |   12 +-
 .../shortcircuit/TestShortCircuitCache.java     |   34 +-
 .../shortcircuit/TestShortCircuitLocalRead.java |    6 +-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |   84 +-
 .../sink/TestRollingFileSystemSinkWithHdfs.java |  106 +-
 ...TestRollingFileSystemSinkWithSecureHdfs.java |  201 ++--
 .../apache/hadoop/tracing/TestTraceAdmin.java   |   69 +-
 .../hadoop-hdfs/src/test/resources/krb5.conf    |   19 +-
 .../src/test/resources/testHDFSConf.xml         |    2 +-
 .../hadoop/mapred/LocalContainerLauncher.java   |   28 +
 .../jobhistory/JobHistoryEventHandler.java      |    4 +-
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |   32 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |    7 +-
 .../v2/app/launcher/ContainerLauncherEvent.java |   21 +-
 .../v2/app/launcher/ContainerLauncherImpl.java  |   19 +-
 .../v2/app/rm/RMContainerAllocator.java         |  192 +--
 .../v2/app/rm/RMContainerRequestor.java         |   19 +-
 .../v2/app/rm/ResourceCalculatorUtils.java      |   17 +-
 .../CheckpointAMPreemptionPolicy.java           |    8 +-
 .../v2/app/TestCheckpointPreemptionPolicy.java  |   16 +-
 .../hadoop/mapreduce/v2/app/TestRecovery.java   |    2 +-
 .../mapreduce/v2/app/TestStagingCleanup.java    |  120 ++
 .../v2/app/job/impl/TestTaskAttempt.java        |   29 +-
 .../v2/app/launcher/TestContainerLauncher.java  |   10 +-
 .../app/launcher/TestContainerLauncherImpl.java |    8 +
 .../app/local/TestLocalContainerAllocator.java  |    2 +-
 .../v2/app/rm/TestRMContainerAllocator.java     |    2 +-
 .../v2/app/rm/TestResourceCalculatorUtils.java  |    4 +-
 .../apache/hadoop/mapreduce/TypeConverter.java  |    6 +-
 .../mapreduce/v2/jobhistory/JHAdminConfig.java  |    8 +
 .../java/org/apache/hadoop/mapred/JobConf.java  |    6 -
 .../org/apache/hadoop/mapreduce/Cluster.java    |   15 +-
 .../org/apache/hadoop/mapreduce/JobStatus.java  |   18 +-
 .../apache/hadoop/mapreduce/JobSubmitter.java   |    7 +
 .../apache/hadoop/mapreduce/MRJobConfig.java    |   18 +
 .../jobhistory/NormalizedResourceEvent.java     |    6 +-
 .../org/apache/hadoop/mapreduce/tools/CLI.java  |    6 +-
 .../src/main/resources/mapred-default.xml       |   16 +
 .../mapreduce/v2/hs/HistoryClientService.java   |    1 +
 .../hadoop/mapred/ResourceMgrDelegate.java      |   12 +-
 .../apache/hadoop/mapred/JobClientUnitTest.java |    6 +-
 .../hadoop/mapred/TestClientRedirect.java       |   10 +-
 .../hadoop/mapred/TestLocalJobSubmission.java   |   24 +-
 .../hadoop/mapred/TestMROpportunisticMaps.java  |  281 +++++
 .../hadoop/mapred/TestMiniMRChildTask.java      |  233 +---
 .../TestClientProtocolProviderImpls.java        |   26 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |  119 ++
 hadoop-project-dist/pom.xml                     |    2 +
 hadoop-project/pom.xml                          |   48 +-
 hadoop-project/src/site/site.xml                |    2 +
 hadoop-tools/hadoop-ant/pom.xml                 |   56 -
 .../java/org/apache/hadoop/ant/DfsTask.java     |  220 ----
 .../ant/condition/DfsBaseConditional.java       |   68 --
 .../apache/hadoop/ant/condition/DfsExists.java  |   24 -
 .../apache/hadoop/ant/condition/DfsIsDir.java   |   24 -
 .../apache/hadoop/ant/condition/DfsZeroLen.java |   24 -
 .../resources/org/apache/hadoop/ant/antlib.xml  |   29 -
 hadoop-tools/hadoop-aws/pom.xml                 |   16 +
 .../hadoop/fs/s3a/AWSClientIOException.java     |   50 +
 .../apache/hadoop/fs/s3a/AWSS3IOException.java  |   61 +
 .../hadoop/fs/s3a/AWSServiceIOException.java    |   72 ++
 .../fs/s3a/AnonymousAWSCredentialsProvider.java |   11 +
 .../fs/s3a/BasicAWSCredentialsProvider.java     |   11 +-
 .../org/apache/hadoop/fs/s3a/Constants.java     |   18 +-
 .../s3a/CredentialInitializationException.java  |   46 +
 .../fs/s3a/ProgressableProgressListener.java    |   94 ++
 .../hadoop/fs/s3a/S3AFastOutputStream.java      |  152 ++-
 .../org/apache/hadoop/fs/s3a/S3AFileStatus.java |    7 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 1032 +++++++++++-----
 .../apache/hadoop/fs/s3a/S3AInputStream.java    |   33 +-
 .../hadoop/fs/s3a/S3AInstrumentation.java       |  218 ++--
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |  116 +-
 .../hadoop/fs/s3a/S3AStorageStatistics.java     |  104 ++
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  237 ++++
 .../org/apache/hadoop/fs/s3a/Statistic.java     |  143 +++
 .../fs/s3a/TemporaryAWSCredentialsProvider.java |   70 ++
 .../org/apache/hadoop/fs/s3a/package-info.java  |   28 +
 .../src/site/markdown/tools/hadoop-aws/index.md |  164 ++-
 .../fs/contract/s3a/TestS3AContractDistCp.java  |   46 +
 .../hadoop/fs/s3a/AbstractS3ATestBase.java      |  103 ++
 .../apache/hadoop/fs/s3a/S3ATestConstants.java  |   76 ++
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  |  293 ++++-
 .../fs/s3a/TestS3AAWSCredentialsProvider.java   |  171 +++
 .../hadoop/fs/s3a/TestS3AConfiguration.java     |   23 +-
 .../apache/hadoop/fs/s3a/TestS3AEncryption.java |  104 ++
 .../TestS3AEncryptionAlgorithmPropagation.java  |   82 ++
 .../s3a/TestS3AEncryptionFastOutputStream.java  |   35 +
 .../hadoop/fs/s3a/TestS3AFailureHandling.java   |  194 +++
 .../hadoop/fs/s3a/TestS3AFileOperationCost.java |  191 +++
 .../fs/s3a/TestS3AFileSystemContract.java       |    1 -
 .../fs/s3a/TestS3ATemporaryCredentials.java     |  150 +++
 .../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 +-
 .../dev-support/findbugs-exclude.xml            |   24 +
 hadoop-tools/hadoop-azure-datalake/pom.xml      |  180 +++
 .../main/java/org/apache/hadoop/fs/adl/Adl.java |   52 +
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |   41 +
 ...hedRefreshTokenBasedAccessTokenProvider.java |  135 +++
 .../hadoop/fs/adl/oauth2/package-info.java      |   23 +
 .../org/apache/hadoop/fs/adl/package-info.java  |   23 +
 .../org/apache/hadoop/hdfs/web/ADLConfKeys.java |   61 +
 .../apache/hadoop/hdfs/web/BufferManager.java   |  180 +++
 .../web/PrivateAzureDataLakeFileSystem.java     | 1108 ++++++++++++++++++
 ...hedRefreshTokenBasedAccessTokenProvider.java |   37 +
 .../hadoop/hdfs/web/oauth2/package-info.java    |   24 +
 .../apache/hadoop/hdfs/web/package-info.java    |   25 +
 .../hadoop/hdfs/web/resources/ADLFlush.java     |   49 +
 .../hdfs/web/resources/ADLGetOpParam.java       |   96 ++
 .../hdfs/web/resources/ADLPostOpParam.java      |   97 ++
 .../hdfs/web/resources/ADLPutOpParam.java       |   94 ++
 .../hdfs/web/resources/ADLVersionInfo.java      |   51 +
 .../web/resources/AppendADLNoRedirectParam.java |   45 +
 .../web/resources/CreateADLNoRedirectParam.java |   44 +
 .../hadoop/hdfs/web/resources/LeaseParam.java   |   53 +
 .../web/resources/ReadADLNoRedirectParam.java   |   44 +
 .../hadoop/hdfs/web/resources/package-info.java |   27 +
 .../src/site/markdown/index.md                  |  219 ++++
 ...hedRefreshTokenBasedAccessTokenProvider.java |  147 +++
 hadoop-tools/hadoop-azure/pom.xml               |   19 +
 .../contract/TestAzureNativeContractDistCp.java |   33 +
 hadoop-tools/hadoop-distcp/pom.xml              |   16 +
 .../apache/hadoop/tools/util/DistCpUtils.java   |   10 +-
 .../src/site/markdown/DistCp.md.vm              |    1 +
 .../contract/AbstractContractDistCpTest.java    |  204 ++++
 .../sls/scheduler/FairSchedulerMetrics.java     |   60 +-
 .../sls/scheduler/ResourceSchedulerWrapper.java |   36 +-
 .../sls/scheduler/SLSCapacityScheduler.java     |   36 +-
 hadoop-tools/hadoop-tools-dist/pom.xml          |    4 +-
 hadoop-tools/pom.xml                            |    2 +-
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |    2 +
 .../dev-support/findbugs-exclude.xml            |    5 +
 .../yarn/api/ApplicationClientProtocol.java     |   27 +-
 .../yarn/api/ContainerManagementProtocol.java   |    5 +
 .../GetNewReservationRequest.java               |   41 +
 .../GetNewReservationResponse.java              |   71 ++
 .../ReservationSubmissionRequest.java           |   25 +-
 .../ReservationSubmissionResponse.java          |   46 +-
 .../SignalContainerResponse.java                |    2 +-
 .../yarn/api/records/AMBlackListingRequest.java |   67 --
 .../records/ApplicationSubmissionContext.java   |   23 -
 .../hadoop/yarn/api/records/Container.java      |   26 +
 .../yarn/api/records/ExecutionTypeRequest.java  |  124 ++
 .../hadoop/yarn/api/records/Resource.java       |   34 +-
 .../yarn/api/records/ResourceRequest.java       |   34 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |   67 +-
 .../main/proto/applicationclient_protocol.proto |    3 +-
 .../proto/containermanagement_protocol.proto    |    1 +
 .../src/main/proto/yarn_protos.proto            |   17 +-
 .../src/main/proto/yarn_service_protos.proto    |    9 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |    7 +
 .../distributedshell/ApplicationMaster.java     |   10 +-
 .../applications/distributedshell/Client.java   |    4 +-
 .../hadoop/yarn/client/api/YarnClient.java      |   25 +-
 .../yarn/client/api/impl/AMRMClientImpl.java    |   16 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |   14 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |    6 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |  376 +++---
 .../apache/hadoop/yarn/client/cli/NodeCLI.java  |    4 +-
 .../apache/hadoop/yarn/client/cli/TopCLI.java   |    4 +-
 .../api/impl/TestDistributedScheduling.java     |    9 +-
 .../yarn/client/api/impl/TestYarnClient.java    |  374 ++++--
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |  179 ++-
 .../yarn/api/ContainerManagementProtocolPB.java |    7 +
 .../ApplicationClientProtocolPBClientImpl.java  |   23 +-
 ...ContainerManagementProtocolPBClientImpl.java |   19 +
 .../ApplicationClientProtocolPBServiceImpl.java |   26 +-
 ...ontainerManagementProtocolPBServiceImpl.java |   20 +
 .../impl/pb/GetNewReservationRequestPBImpl.java |   77 ++
 .../pb/GetNewReservationResponsePBImpl.java     |  144 +++
 .../pb/ReservationSubmissionRequestPBImpl.java  |   30 +
 .../pb/ReservationSubmissionResponsePBImpl.java |   59 -
 .../impl/pb/AMBlackListingRequestPBImpl.java    |  104 --
 .../pb/ApplicationSubmissionContextPBImpl.java  |   40 -
 .../api/records/impl/pb/ContainerPBImpl.java    |   26 +-
 .../impl/pb/ExecutionTypeRequestPBImpl.java     |   93 ++
 .../yarn/api/records/impl/pb/ProtoUtils.java    |   15 +
 .../api/records/impl/pb/ResourcePBImpl.java     |   30 +-
 .../records/impl/pb/ResourceRequestPBImpl.java  |   52 +-
 .../logaggregation/AggregatedLogFormat.java     |  175 ++-
 .../logaggregation/ContainerLogsRequest.java    |  135 +++
 .../yarn/logaggregation/LogCLIHelpers.java      |  160 ++-
 .../nodelabels/CommonNodeLabelsManager.java     |   18 +-
 .../yarn/security/SchedulerSecurityInfo.java    |    2 +-
 .../apache/hadoop/yarn/util/AdHocLogDumper.java |    9 +-
 .../yarn/util/ProcfsBasedProcessTree.java       |   19 +-
 .../util/ResourceCalculatorProcessTree.java     |    4 +-
 .../yarn/util/WindowsBasedProcessTree.java      |    8 +
 .../resource/DefaultResourceCalculator.java     |   36 +-
 .../resource/DominantResourceCalculator.java    |   46 +-
 .../yarn/util/resource/ResourceCalculator.java  |   10 +-
 .../hadoop/yarn/util/resource/Resources.java    |   62 +-
 .../org/apache/hadoop/yarn/webapp/WebApps.java  |   53 +-
 .../hadoop/yarn/webapp/util/WebAppUtils.java    |   18 +
 .../src/main/resources/yarn-default.xml         |   73 +-
 .../hadoop/yarn/TestContainerLaunchRPC.java     |   10 +
 .../yarn/TestContainerResourceIncreaseRPC.java  |    8 +
 .../java/org/apache/hadoop/yarn/TestRPC.java    |   10 +
 .../hadoop/yarn/api/TestPBImplRecords.java      |   12 +-
 .../yarn/util/TestProcfsBasedProcessTree.java   |   66 +-
 .../yarn/util/resource/TestResources.java       |    8 +-
 .../client/impl/zk/RegistrySecurity.java        |    2 +-
 .../ApplicationHistoryServer.java               |    1 +
 .../webapp/AHSWebServices.java                  |  187 +--
 .../webapp/TestAHSWebServices.java              |   66 ++
 .../api/DistributedSchedulerProtocolPB.java     |   10 +-
 .../pb/DistSchedRegisterResponsePBImpl.java     |    6 +-
 .../hadoop/yarn/server/utils/BuilderUtils.java  |   12 +-
 .../hadoop/yarn/server/webapp/dao/AppInfo.java  |   10 +-
 .../yarn/server/webapp/dao/ContainerInfo.java   |   10 +-
 .../proto/distributed_scheduler_protocol.proto  |    1 +
 .../hadoop/yarn/TestYarnServerApiClasses.java   |    6 +-
 .../server/nodemanager/ContainerExecutor.java   |    2 +-
 .../nodemanager/DefaultContainerExecutor.java   |   22 +-
 .../containermanager/ContainerManagerImpl.java  |   40 +-
 .../application/ApplicationImpl.java            |   80 +-
 .../container/ContainerImpl.java                |   20 +-
 .../launcher/RecoveredContainerLaunch.java      |    7 +-
 .../CGroupsMemoryResourceHandlerImpl.java       |    4 +-
 .../runtime/DockerLinuxContainerRuntime.java    |   46 +-
 .../localizer/ContainerLocalizer.java           |   40 +-
 .../logaggregation/AppLogAggregatorImpl.java    |  101 +-
 .../logaggregation/LogAggregationService.java   |   60 +-
 .../event/LogHandlerAppStartedEvent.java        |   21 +-
 .../monitor/ContainerMetrics.java               |   18 +-
 .../monitor/ContainersMonitorImpl.java          |   41 +-
 .../queuing/QueuingContainerManagerImpl.java    |   41 +-
 .../nodemanager/metrics/NodeManagerMetrics.java |   12 +-
 .../nodemanager/scheduler/LocalScheduler.java   |    3 +-
 .../OpportunisticContainerAllocator.java        |    3 +-
 .../nodemanager/webapp/NMWebServices.java       |   89 +-
 .../server/nodemanager/webapp/WebServer.java    |    5 +-
 .../nodemanager/webapp/dao/ContainerInfo.java   |    2 +-
 .../impl/container-executor.c                   |   19 +-
 .../test/test-container-executor.c              |   49 -
 .../yarn_server_nodemanager_recovery.proto      |    1 +
 .../TestDefaultContainerExecutor.java           |   89 +-
 .../yarn/server/nodemanager/TestEventFlow.java  |    3 +
 .../nodemanager/TestNodeStatusUpdater.java      |    4 +-
 .../TestNodeStatusUpdaterForLabels.java         |    7 +
 .../amrmproxy/MockResourceManagerFacade.java    |   12 +-
 .../BaseContainerManagerTest.java               |   67 ++
 .../containermanager/TestContainerManager.java  |   76 +-
 .../application/TestApplication.java            |   45 +-
 .../container/TestContainer.java                |   34 +
 .../runtime/TestDockerContainerRuntime.java     |  298 ++++-
 .../MockLocalizerHeartbeatResponse.java         |    7 +-
 .../localizer/TestContainerLocalizer.java       |   41 +-
 .../TestAppLogAggregatorImpl.java               |  436 +++++++
 .../TestLogAggregationService.java              |    7 +-
 .../MockCPUResourceCalculatorProcessTree.java   |   70 ++
 .../MockResourceCalculatorProcessTree.java      |    5 +
 .../monitor/TestContainerMetrics.java           |    4 +-
 .../monitor/TestContainersMonitor.java          |    2 +-
 .../TestContainersMonitorResourceChange.java    |   62 +-
 .../queuing/TestQueuingContainerManager.java    |  551 +++++++--
 .../scheduler/TestLocalScheduler.java           |   11 +-
 .../nodemanager/webapp/TestNMWebServices.java   |   63 +-
 .../server/resourcemanager/ClientRMService.java |   38 +-
 .../resourcemanager/NodesListManager.java       |   69 +-
 .../server/resourcemanager/RMAuditLogger.java   |    2 +
 .../yarn/server/resourcemanager/RMNMInfo.java   |    4 +-
 .../server/resourcemanager/RMServerUtils.java   |   14 +-
 .../server/resourcemanager/ResourceManager.java |    1 +
 .../resourcemanager/ResourceTrackerService.java |    2 +-
 .../blacklist/BlacklistManager.java             |    9 +-
 .../blacklist/BlacklistUpdates.java             |   47 -
 .../blacklist/DisabledBlacklistManager.java     |   12 +-
 .../blacklist/SimpleBlacklistManager.java       |   17 +-
 .../metrics/SystemMetricsPublisher.java         |    2 +-
 .../monitor/capacity/TempQueuePerPartition.java |   12 +-
 .../nodelabels/RMNodeLabelsManager.java         |    9 +-
 .../AbstractSchedulerPlanFollower.java          |    2 +-
 .../reservation/CapacityOverTimePolicy.java     |    6 +-
 .../reservation/NoOverCommitPolicy.java         |   38 +-
 .../RLESparseResourceAllocation.java            |    6 +-
 .../reservation/ReservationInputValidator.java  |    7 +
 .../reservation/ReservationSystemUtil.java      |    2 +-
 .../planning/StageAllocatorGreedyRLE.java       |   16 +-
 .../planning/StageEarliestStartByDemand.java    |    2 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |   79 +-
 .../rmapp/attempt/RMAppAttempt.java             |    2 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |  113 +-
 .../rmcontainer/RMContainerImpl.java            |    2 +-
 .../scheduler/AbstractYarnScheduler.java        |   10 +-
 .../scheduler/AppSchedulingInfo.java            |   74 +-
 .../scheduler/ClusterNodeTracker.java           |   10 +-
 .../resourcemanager/scheduler/QueueMetrics.java |   61 +-
 .../scheduler/SchedulerAppUtils.java            |   16 +-
 .../scheduler/SchedulerApplicationAttempt.java  |   39 +-
 .../scheduler/SchedulerUtils.java               |    8 +-
 .../scheduler/capacity/CSQueueMetrics.java      |   24 +-
 .../capacity/CapacityHeadroomProvider.java      |    2 +-
 .../scheduler/capacity/CapacityScheduler.java   |   11 +-
 .../CapacitySchedulerConfiguration.java         |    6 +-
 .../scheduler/capacity/LeafQueue.java           |    6 +-
 .../allocator/RegularContainerAllocator.java    |    4 +-
 .../scheduler/fair/FSAppAttempt.java            |    5 +-
 .../scheduler/fair/FSLeafQueue.java             |   25 +-
 .../scheduler/fair/FSParentQueue.java           |    2 +-
 .../resourcemanager/scheduler/fair/FSQueue.java |   35 +-
 .../scheduler/fair/FSQueueMetrics.java          |   66 +-
 .../scheduler/fair/FairScheduler.java           |   47 +-
 .../fair/FairSchedulerConfiguration.java        |   12 +
 .../scheduler/fair/QueueManager.java            |    5 +-
 .../fair/policies/ComputeFairShares.java        |   18 +-
 .../DominantResourceFairnessPolicy.java         |   10 +-
 .../fair/policies/FairSharePolicy.java          |   20 +-
 .../scheduler/fair/policies/FifoPolicy.java     |    8 +-
 .../scheduler/fifo/FifoScheduler.java           |   27 +-
 .../scheduler/policy/FairOrderingPolicy.java    |    5 +-
 .../security/DelegationTokenRenewer.java        |   27 +-
 .../webapp/DefaultSchedulerPage.java            |    4 +-
 .../webapp/FairSchedulerAppsBlock.java          |    2 +-
 .../webapp/RMAppAttemptBlock.java               |    9 +-
 .../resourcemanager/webapp/RMAppBlock.java      |   13 +-
 .../resourcemanager/webapp/RMWebServices.java   |  128 +-
 .../webapp/dao/AMBlackListingRequestInfo.java   |   61 -
 .../webapp/dao/AppAttemptInfo.java              |    8 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |   20 +-
 .../dao/ApplicationSubmissionContextInfo.java   |   13 -
 .../webapp/dao/FairSchedulerInfo.java           |    4 +-
 .../webapp/dao/FairSchedulerQueueInfo.java      |   14 +-
 .../webapp/dao/FifoSchedulerInfo.java           |   18 +-
 .../webapp/dao/NewReservation.java              |   55 +
 .../resourcemanager/webapp/dao/NodeInfo.java    |    4 +-
 .../dao/ReservationSubmissionRequestInfo.java   |   11 +
 .../dao/ReservationSubmissionResponseInfo.java  |   54 -
 .../webapp/dao/ResourceInfo.java                |   10 +-
 .../server/resourcemanager/Application.java     |    2 +-
 .../yarn/server/resourcemanager/MockNM.java     |    8 +-
 .../yarn/server/resourcemanager/MockNodes.java  |    4 +-
 .../yarn/server/resourcemanager/MockRM.java     |   20 +-
 .../server/resourcemanager/NodeManager.java     |   17 +-
 .../ReservationACLsTestBase.java                |   63 +-
 .../resourcemanager/TestAMAuthorization.java    |    8 +
 .../resourcemanager/TestApplicationACLs.java    |    6 +-
 .../resourcemanager/TestApplicationCleanup.java |    2 +-
 .../TestApplicationMasterLauncher.java          |    8 +
 .../resourcemanager/TestClientRMService.java    |  287 +++--
 .../TestContainerResourceUsage.java             |   28 +-
 .../TestDistributedSchedulingService.java       |  166 ++-
 .../TestNodeBlacklistingOnAMFailures.java       |  251 ++++
 .../yarn/server/resourcemanager/TestRM.java     |    8 +-
 .../yarn/server/resourcemanager/TestRMHA.java   |    4 +-
 .../resourcemanager/TestRMNodeTransitions.java  |   22 +-
 .../TestReservationSystemWithRMHA.java          |   94 +-
 .../TestResourceTrackerService.java             |    2 +-
 .../resourcemanager/TestSignalContainer.java    |    2 +-
 .../TestWorkPreservingRMRestart.java            |  121 +-
 .../applicationsmanager/TestAMRestart.java      |  177 +--
 .../blacklist/TestBlacklistManager.java         |   29 +-
 .../metrics/TestSystemMetricsPublisher.java     |   10 +-
 ...alCapacityPreemptionPolicyMockFramework.java |    4 +-
 ...estProportionalCapacityPreemptionPolicy.java |   39 +-
 ...alCapacityPreemptionPolicyMockFramework.java |    6 +-
 .../nodelabels/TestRMNodeLabelsManager.java     |   69 +-
 .../reservation/ReservationSystemTestUtil.java  |    7 +-
 .../TestRLESparseResourceAllocation.java        |   40 +-
 .../planning/TestSimpleCapacityReplanner.java   |    6 +-
 .../rmapp/TestRMAppTransitions.java             |   58 -
 .../attempt/TestRMAppAttemptTransitions.java    |    3 +-
 .../scheduler/TestAbstractYarnScheduler.java    |   16 +-
 .../scheduler/TestAppSchedulingInfo.java        |   12 +-
 .../scheduler/TestQueueMetrics.java             |    8 +-
 .../scheduler/TestResourceUsage.java            |    2 +-
 .../scheduler/TestSchedulerUtils.java           |   20 +-
 .../capacity/TestApplicationLimits.java         |   15 +-
 .../capacity/TestApplicationPriority.java       |   52 +-
 .../capacity/TestCapacityScheduler.java         |  129 +-
 .../TestCapacitySchedulerLazyPreemption.java    |   30 +-
 .../TestCapacitySchedulerNodeLabelUpdate.java   |   52 +-
 ...TestCapacitySchedulerSurgicalPreemption.java |    4 +-
 .../scheduler/capacity/TestChildQueueOrder.java |    4 +-
 .../capacity/TestContainerAllocation.java       |   53 +-
 .../capacity/TestContainerResizing.java         |  150 +--
 .../capacity/TestIncreaseAllocationExpirer.java |   36 +-
 .../scheduler/capacity/TestLeafQueue.java       |  492 ++++----
 .../TestNodeLabelContainerAllocation.java       |   15 +-
 .../scheduler/capacity/TestParentQueue.java     |    4 +-
 .../scheduler/capacity/TestQueueCapacities.java |    2 +-
 .../scheduler/capacity/TestReservations.java    |  406 +++----
 ...TestWorkPreservingRMRestartForNodeLabel.java |    4 +-
 .../scheduler/fair/FairSchedulerTestBase.java   |    4 +-
 .../scheduler/fair/TestComputeFairShares.java   |    2 +-
 .../fair/TestContinuousScheduling.java          |    2 +-
 .../scheduler/fair/TestFSAppAttempt.java        |   44 +-
 .../scheduler/fair/TestFSLeafQueue.java         |   27 +-
 .../scheduler/fair/TestFairScheduler.java       |  403 ++++---
 .../fair/TestFairSchedulerFairShare.java        |   32 +-
 .../fair/TestFairSchedulerPreemption.java       |   62 +-
 .../scheduler/fifo/TestFifoScheduler.java       |   63 +-
 .../security/TestDelegationTokenRenewer.java    |   98 ++
 .../webapp/TestRMWebServices.java               |   17 +-
 .../webapp/TestRMWebServicesApps.java           |   46 +-
 .../TestRMWebServicesAppsModification.java      |   39 +-
 .../webapp/TestRMWebServicesCapacitySched.java  |    5 +-
 .../webapp/TestRMWebServicesNodes.java          |    4 +-
 .../webapp/TestRMWebServicesReservation.java    |  351 ++++--
 .../hadoop/yarn/webapp/TestRMWithXFSFilter.java |  158 +++
 .../src/test/resources/krb5.conf                |   23 +-
 .../src/test/resources/submit-reservation.json  |    1 +
 .../yarn/server/timeline/EntityCacheItem.java   |   66 +-
 .../timeline/EntityGroupFSTimelineStore.java    |   82 +-
 .../timeline/EntityGroupPlugInForTest.java      |   19 +-
 .../TestEntityGroupFSTimelineStore.java         |  248 +++-
 .../src/site/markdown/CapacityScheduler.md      |    2 +-
 .../src/site/markdown/FairScheduler.md          |    7 +-
 .../src/site/markdown/ReservationSystem.md      |    4 +-
 .../src/site/markdown/ResourceManagerRest.md    |   90 +-
 hadoop-yarn-project/hadoop-yarn/pom.xml         |    7 +
 pom.xml                                         |   44 +
 620 files changed, 27022 insertions(+), 9621 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/23923086/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23923086/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/23923086/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------


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


[40/50] [abbrv] hadoop git commit: YARN-4308. ContainersAggregated CPU resource utilization reports negative usage in first few heartbeats. Contributed by Sunil G

Posted by ae...@apache.org.
YARN-4308. ContainersAggregated CPU resource utilization reports negative usage in first few heartbeats. Contributed by Sunil G


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

Branch: refs/heads/HDFS-7240
Commit: 1500a0a3009e453c9f05a93df7a78b4e185eef30
Parents: ae04765
Author: Naganarasimha <na...@apache.org>
Authored: Thu Jun 9 05:41:09 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Thu Jun 9 05:41:09 2016 +0530

----------------------------------------------------------------------
 .../yarn/util/ProcfsBasedProcessTree.java       |  8 +++
 .../util/ResourceCalculatorProcessTree.java     |  4 +-
 .../yarn/util/WindowsBasedProcessTree.java      |  8 +++
 .../monitor/ContainersMonitorImpl.java          |  9 +++
 .../MockCPUResourceCalculatorProcessTree.java   | 70 ++++++++++++++++++++
 .../MockResourceCalculatorProcessTree.java      |  5 ++
 .../TestContainersMonitorResourceChange.java    | 62 ++++++++++++++++-
 7 files changed, 163 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
index bb9c183..80d49c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
@@ -467,6 +467,14 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     return totalStime.add(BigInteger.valueOf(totalUtime));
   }
 
+  /**
+   * Get the CPU usage by all the processes in the process-tree in Unix.
+   * Note: UNAVAILABLE will be returned in case when CPU usage is not
+   * available. It is NOT advised to return any other error code.
+   *
+   * @return percentage CPU usage since the process-tree was created,
+   * {@link #UNAVAILABLE} if CPU usage cannot be calculated or not available.
+   */
   @Override
   public float getCpuUsagePercent() {
     BigInteger processTotalJiffies = getTotalProcessJiffies();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
index 7214c75..771ec86 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
@@ -187,9 +187,11 @@ public abstract class ResourceCalculatorProcessTree extends Configured {
    * Get the CPU usage by all the processes in the process-tree based on
    * average between samples as a ratio of overall CPU cycles similar to top.
    * Thus, if 2 out of 4 cores are used this should return 200.0.
+   * Note: UNAVAILABLE will be returned in case when CPU usage is not
+   * available. It is NOT advised to return any other error code.
    *
    * @return percentage CPU usage since the process-tree was created,
-   * {@link #UNAVAILABLE} if it cannot be calculated.
+   * {@link #UNAVAILABLE} if CPU usage cannot be calculated or not available.
    */
   public float getCpuUsagePercent() {
     return UNAVAILABLE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
index 7858292..1c7eaf7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
@@ -268,6 +268,14 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree {
     return BigInteger.valueOf(totalMs);
   }
 
+  /**
+   * Get the CPU usage by all the processes in the process-tree in Windows.
+   * Note: UNAVAILABLE will be returned in case when CPU usage is not
+   * available. It is NOT advised to return any other error code.
+   *
+   * @return percentage CPU usage since the process-tree was created,
+   * {@link #UNAVAILABLE} if CPU usage cannot be calculated or not available.
+   */
   @Override
   public float getCpuUsagePercent() {
     BigInteger processTotalMs = getTotalProcessMs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/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 cfe6f801..b5c2747 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
@@ -455,6 +455,15 @@ public class ContainersMonitorImpl extends AbstractService implements
             // cpuUsagePercentPerCore should be 300% and
             // cpuUsageTotalCoresPercentage should be 50%
             float cpuUsagePercentPerCore = pTree.getCpuUsagePercent();
+            if (cpuUsagePercentPerCore < 0) {
+              // CPU usage is not available likely because the container just
+              // started. Let us skip this turn and consider this container
+              // in the next iteration.
+              LOG.info("Skipping monitoring container " + containerId
+                  + " since CPU usage is not yet available.");
+              continue;
+            }
+
             float cpuUsageTotalCoresPercentage = cpuUsagePercentPerCore /
                 resourceCalculatorPlugin.getNumProcessors();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockCPUResourceCalculatorProcessTree.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/MockCPUResourceCalculatorProcessTree.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/MockCPUResourceCalculatorProcessTree.java
new file mode 100644
index 0000000..eb35c91
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockCPUResourceCalculatorProcessTree.java
@@ -0,0 +1,70 @@
+/**
+ * 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.server.nodemanager.containermanager.monitor;
+
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+
+/**
+ * Mock class to obtain resource usage (CPU).
+ */
+public class MockCPUResourceCalculatorProcessTree
+    extends ResourceCalculatorProcessTree {
+
+  private long cpuPercentage = ResourceCalculatorProcessTree.UNAVAILABLE;
+
+  /**
+   * Constructor for MockCPUResourceCalculatorProcessTree with specified root
+   * process.
+   * @param root
+   */
+  public MockCPUResourceCalculatorProcessTree(String root) {
+    super(root);
+  }
+
+  @Override
+  public void updateProcessTree() {
+  }
+
+  @Override
+  public String getProcessTreeDump() {
+    return "";
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    return 0;
+  }
+
+  @Override
+  public boolean checkPidPgrpidForMatch() {
+    return true;
+  }
+
+  @Override
+  public float getCpuUsagePercent() {
+    long cpu = this.cpuPercentage;
+    // First getter call will be returned with -1, and other calls will
+    // return non-zero value as defined below.
+    if (cpu == ResourceCalculatorProcessTree.UNAVAILABLE) {
+      // Set a default value other than 0 for test.
+      this.cpuPercentage = 50;
+    }
+    return cpu;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.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/MockResourceCalculatorProcessTree.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/MockResourceCalculatorProcessTree.java
index c5aaa77..ff2a570 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/MockResourceCalculatorProcessTree.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/MockResourceCalculatorProcessTree.java
@@ -54,4 +54,9 @@ public class MockResourceCalculatorProcessTree extends ResourceCalculatorProcess
   public long getRssMemorySize() {
     return this.rssMemorySize;
   }
+
+  @Override
+  public float getCpuUsagePercent() {
+    return 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.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/TestContainersMonitorResourceChange.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/TestContainersMonitorResourceChange.java
index 1a0c690..2df0c98 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/TestContainersMonitorResourceChange.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/TestContainersMonitorResourceChange.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -43,19 +43,21 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
+import org.apache.log4j.Logger;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 
 public class TestContainersMonitorResourceChange {
 
+  static final Logger LOG = Logger
+      .getLogger(TestContainersMonitorResourceChange.class);
   private ContainersMonitorImpl containersMonitor;
   private MockExecutor executor;
   private Configuration conf;
@@ -63,6 +65,8 @@ public class TestContainersMonitorResourceChange {
   private Context context;
   private MockContainerEventHandler containerEventHandler;
 
+  static final int WAIT_MS_PER_LOOP = 20; // 20 milli seconds
+
   private static class MockExecutor extends ContainerExecutor {
     @Override
     public void init() throws IOException {
@@ -232,6 +236,60 @@ public class TestContainersMonitorResourceChange {
     containersMonitor.stop();
   }
 
+  @Test
+  public void testContainersCPUResourceForDefaultValue() throws Exception {
+    Configuration newConf = new Configuration(conf);
+    // set container monitor interval to be 20s
+    newConf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20L);
+    containersMonitor = createContainersMonitor(executor, dispatcher, context);
+    newConf.set(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
+        MockCPUResourceCalculatorProcessTree.class.getCanonicalName());
+    // set container monitor interval to be 20ms
+    containersMonitor.init(newConf);
+    containersMonitor.start();
+
+    // create container 1
+    containersMonitor.handle(new ContainerStartMonitoringEvent(
+        getContainerId(1), 2100L, 1000L, 1, 0, 0));
+
+    // Verify the container utilization value.
+    // Since MockCPUResourceCalculatorProcessTree will return a -1 as CPU
+    // utilization, containersUtilization will not be calculated and hence it
+    // will be 0.
+    assertEquals(
+        "Resource utilization must be default with MonitorThread's first run",
+        0, containersMonitor.getContainersUtilization()
+            .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f)));
+
+    // Verify the container utilization value. Since atleast one round is done,
+    // we can expect a non-zero value for container utilization as
+    // MockCPUResourceCalculatorProcessTree#getCpuUsagePercent will return 50.
+    waitForContainerResourceUtilizationChange(containersMonitor, 100);
+
+    containersMonitor.stop();
+  }
+
+  public static void waitForContainerResourceUtilizationChange(
+      ContainersMonitorImpl containersMonitor, int timeoutMsecs)
+      throws InterruptedException {
+    int timeWaiting = 0;
+    while (0 == containersMonitor.getContainersUtilization()
+        .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f))) {
+      if (timeWaiting >= timeoutMsecs) {
+        break;
+      }
+
+      LOG.info(
+          "Monitor thread is waiting for resource utlization change.");
+      Thread.sleep(WAIT_MS_PER_LOOP);
+      timeWaiting += WAIT_MS_PER_LOOP;
+    }
+
+    assertTrue("Resource utilization is not changed from second run onwards",
+        0 != containersMonitor.getContainersUtilization()
+            .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f)));
+  }
+
   private ContainersMonitorImpl createContainersMonitor(
       ContainerExecutor containerExecutor, AsyncDispatcher dispatcher,
       Context context) {


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


[28/50] [abbrv] hadoop git commit: YARN-4837. User facing aspects of 'AM blacklisting' feature need fixing. (vinodkv via wangda)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.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/webapp/dao/AppAttemptInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
index 60b728e..e8c8bca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
@@ -42,7 +42,7 @@ public class AppAttemptInfo {
   protected String nodeId;
   protected String logsLink;
   protected String blacklistedNodes;
-  protected String rmBlacklistedNodesForAMLaunches;
+  private String nodesBlacklistedBySystem;
   protected String appAttemptId;
 
   public AppAttemptInfo() {
@@ -69,9 +69,9 @@ public class AppAttemptInfo {
             + masterContainer.getNodeHttpAddress(),
             ConverterUtils.toString(masterContainer.getId()), user);
 
-        rmBlacklistedNodesForAMLaunches = StringUtils.join(
-            attempt.getAMBlacklist().getBlacklistUpdates().getAdditions(),
-            ", ");
+        nodesBlacklistedBySystem =
+            StringUtils.join(attempt.getAMBlacklistManager()
+              .getBlacklistUpdates().getBlacklistAdditions(), ", ");
         if (rm.getResourceScheduler() instanceof AbstractYarnScheduler) {
           AbstractYarnScheduler ayScheduler =
               (AbstractYarnScheduler) rm.getResourceScheduler();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.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/webapp/dao/ApplicationSubmissionContextInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
index 4cbe7a8..3d95ca1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.java
@@ -87,9 +87,6 @@ public class ApplicationSubmissionContextInfo {
   @XmlElement(name = "reservation-id")
   String reservationId;
 
-  @XmlElement(name = "am-black-listing-requests")
-  AMBlackListingRequestInfo amBlackListingRequestInfo;
-
   public ApplicationSubmissionContextInfo() {
     applicationId = "";
     applicationName = "";
@@ -106,7 +103,6 @@ public class ApplicationSubmissionContextInfo {
     logAggregationContextInfo = null;
     attemptFailuresValidityInterval = -1;
     reservationId = "";
-    amBlackListingRequestInfo = null;
   }
 
   public String getApplicationId() {
@@ -173,10 +169,6 @@ public class ApplicationSubmissionContextInfo {
     return attemptFailuresValidityInterval;
   }
 
-  public AMBlackListingRequestInfo getAMBlackListingRequestInfo() {
-    return amBlackListingRequestInfo;
-  }
-
   public String getReservationId() {
     return reservationId;
   }
@@ -252,9 +244,4 @@ public class ApplicationSubmissionContextInfo {
   public void setReservationId(String reservationId) {
     this.reservationId = reservationId;
   }
-
-  public void setAMBlackListingRequestInfo(
-      AMBlackListingRequestInfo amBlackListingRequestInfo) {
-    this.amBlackListingRequestInfo = amBlackListingRequestInfo;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.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/TestNodeBlacklistingOnAMFailures.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
new file mode 100644
index 0000000..ef6d43b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
@@ -0,0 +1,251 @@
+/**
+ * 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.server.resourcemanager;
+
+import java.util.List;
+
+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.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
+import org.apache.hadoop.yarn.event.EventDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.TestAMRestart;
+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.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Validate system behavior when the am-scheduling logic 'blacklists' a node for
+ * an application because of AM failures.
+ */
+public class TestNodeBlacklistingOnAMFailures {
+
+  @Test(timeout = 100000)
+  public void testNodeBlacklistingOnAMFailure() throws Exception {
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+        true);
+
+    DrainDispatcher dispatcher = new DrainDispatcher();
+    MockRM rm = startRM(conf, dispatcher);
+    CapacityScheduler scheduler = (CapacityScheduler) rm.getResourceScheduler();
+
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 8000, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    MockNM nm2 =
+        new MockNM("127.0.0.2:2345", 8000, rm.getResourceTrackerService());
+    nm2.registerNode();
+
+    RMApp app = rm.submitApp(200);
+
+    MockAM am1 = MockRM.launchAndRegisterAM(app, rm, nm1);
+    ContainerId amContainerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    RMContainer rmContainer = scheduler.getRMContainer(amContainerId);
+    NodeId nodeWhereAMRan = rmContainer.getAllocatedNode();
+
+    MockNM currentNode, otherNode;
+    if (nodeWhereAMRan.equals(nm1.getNodeId())) {
+      currentNode = nm1;
+      otherNode = nm2;
+    } else {
+      currentNode = nm2;
+      otherNode = nm1;
+    }
+
+    // Set the exist status to INVALID so that we can verify that the system
+    // automatically blacklisting the node
+    makeAMContainerExit(rm, amContainerId, currentNode,
+        ContainerExitStatus.INVALID);
+
+    // restart the am
+    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app, rm);
+    System.out.println("New AppAttempt launched " + attempt.getAppAttemptId());
+
+    // Try the current node a few times
+    for (int i = 0; i <= 2; i++) {
+      currentNode.nodeHeartbeat(true);
+      dispatcher.await();
+
+      Assert.assertEquals(
+          "AppAttemptState should still be SCHEDULED if currentNode is "
+              + "blacklisted correctly", RMAppAttemptState.SCHEDULED,
+          attempt.getAppAttemptState());
+    }
+
+    // Now try the other node
+    otherNode.nodeHeartbeat(true);
+    dispatcher.await();
+
+    // Now the AM container should be allocated
+    MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+
+    MockAM am2 = rm.sendAMLaunched(attempt.getAppAttemptId());
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+    amContainerId =
+        ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
+    rmContainer = scheduler.getRMContainer(amContainerId);
+    nodeWhereAMRan = rmContainer.getAllocatedNode();
+
+    // The other node should now receive the assignment
+    Assert.assertEquals(
+        "After blacklisting, AM should have run on the other node",
+        otherNode.getNodeId(), nodeWhereAMRan);
+
+    am2.registerAppAttempt();
+    rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
+
+    List<Container> allocatedContainers =
+        TestAMRestart.allocateContainers(currentNode, am2, 1);
+    Assert.assertEquals(
+        "Even though AM is blacklisted from the node, application can "
+            + "still allocate non-AM containers there",
+        currentNode.getNodeId(), allocatedContainers.get(0).getNodeId());
+  }
+
+  @Test(timeout = 100000)
+  public void testNoBlacklistingForNonSystemErrors() throws Exception {
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+        true);
+    // disable the float so it is possible to blacklist the entire cluster
+    conf.setFloat(
+        YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+        1.5f);
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 100);
+
+    DrainDispatcher dispatcher = new DrainDispatcher();
+    MockRM rm = startRM(conf, dispatcher);
+
+    MockNM node =
+        new MockNM("127.0.0.1:1234", 8000, rm.getResourceTrackerService());
+    node.registerNode();
+
+    RMApp app = rm.submitApp(200);
+    ApplicationId appId = app.getApplicationId();
+
+    int numAppAttempts = 1;
+
+    // Now the AM container should be allocated
+    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app, rm);
+    node.nodeHeartbeat(true);
+    dispatcher.await();
+    MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+    rm.sendAMLaunched(attempt.getAppAttemptId());
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, numAppAttempts);
+    ContainerId amContainerId = ContainerId.newContainerId(appAttemptId, 1);
+
+    for (int containerExitStatus : new int[] {
+        ContainerExitStatus.PREEMPTED,
+        ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
+        // ContainerExitStatus.KILLED_BY_APPMASTER,
+        ContainerExitStatus.KILLED_AFTER_APP_COMPLETION,
+        ContainerExitStatus.ABORTED, ContainerExitStatus.DISKS_FAILED,
+        ContainerExitStatus.KILLED_EXCEEDED_VMEM,
+        ContainerExitStatus.KILLED_EXCEEDED_PMEM }) {
+
+      // Set the exist status to be containerExitStatus so that we can verify
+      // that the system automatically blacklisting the node
+      makeAMContainerExit(rm, amContainerId, node, containerExitStatus);
+
+      // restart the am
+      attempt = MockRM.waitForAttemptScheduled(app, rm);
+      System.out
+          .println("New AppAttempt launched " + attempt.getAppAttemptId());
+
+      node.nodeHeartbeat(true);
+      dispatcher.await();
+
+      MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+      rm.sendAMLaunched(attempt.getAppAttemptId());
+      rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+
+      numAppAttempts++;
+      appAttemptId = ApplicationAttemptId.newInstance(appId, numAppAttempts);
+      amContainerId = ContainerId.newContainerId(appAttemptId, 1);
+      rm.waitForState(node, amContainerId, RMContainerState.ACQUIRED);
+    }
+  }
+
+  private void makeAMContainerExit(MockRM rm, ContainerId amContainer,
+      MockNM node, int exitStatus) throws Exception, InterruptedException {
+    ContainerStatus containerStatus =
+        BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE,
+            "", exitStatus, Resources.createResource(200));
+    node.containerStatus(containerStatus);
+    ApplicationAttemptId amAttemptID = amContainer.getApplicationAttemptId();
+    rm.waitForState(amAttemptID, RMAppAttemptState.FAILED);
+    rm.waitForState(amAttemptID.getApplicationId(), RMAppState.ACCEPTED);
+  }
+
+  private MockRM startRM(YarnConfiguration conf,
+      final DrainDispatcher dispatcher) {
+
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+
+    MockRM rm1 = new MockRM(conf, memStore) {
+      @Override
+      protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
+        return new EventDispatcher<SchedulerEvent>(this.scheduler,
+            this.scheduler.getClass().getName()) {
+          @Override
+          public void handle(SchedulerEvent event) {
+            super.handle(event);
+          }
+        };
+      }
+
+      @Override
+      protected Dispatcher createDispatcher() {
+        return dispatcher;
+      }
+    };
+
+    rm1.start();
+    return rm1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.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/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 66780a0..76a17c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -35,13 +35,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.DrainDispatcher;
-import org.apache.hadoop.yarn.event.EventDispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -54,18 +49,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -238,7 +229,7 @@ public class TestAMRestart {
     rm1.stop();
   }
 
-  private List<Container> allocateContainers(MockNM nm1, MockAM am1,
+  public static List<Container> allocateContainers(MockNM nm1, MockAM am1,
       int NUM_CONTAINERS) throws Exception {
     // allocate NUM_CONTAINERS containers
     am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
@@ -277,7 +268,9 @@ public class TestAMRestart {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3);
     // To prevent test from blacklisting nm1 for AM, we sit threshold to half
     // of 2 nodes which is 1
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD, 0.5f);
+    conf.setFloat(
+        YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+        0.5f);
 
     MockRM rm1 = new MockRM(conf);
     rm1.start();
@@ -379,168 +372,6 @@ public class TestAMRestart {
     rm1.stop();
   }
 
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreventsRestartOnSameNode() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    testAMBlacklistPreventRestartOnSameNode(false, conf);
-  }
-
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreventsRestartOnSameNodeForMinicluster()
-      throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME,
-        true);
-    testAMBlacklistPreventRestartOnSameNode(false, conf);
-  }
-
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreemption() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    // disable the float so it is possible to blacklist the entire cluster
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD, 1.5f);
-    // since the exit status is PREEMPTED, it should not lead to the node being
-    // blacklisted
-    testAMBlacklistPreventRestartOnSameNode(true, conf);
-  }
-
-  /**
-   * Tests AM blacklisting. In the multi-node mode (i.e. singleNode = false),
-   * it tests the blacklisting behavior so that the AM container gets allocated
-   * on the node that is not blacklisted. In the single-node mode, it tests the
-   * PREEMPTED status to see if the AM container can continue to be scheduled.
-   */
-  private void testAMBlacklistPreventRestartOnSameNode(boolean singleNode,
-      YarnConfiguration conf) throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    final DrainDispatcher dispatcher = new DrainDispatcher();
-    MockRM rm1 = new MockRM(conf, memStore) {
-      @Override
-      protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
-        return new EventDispatcher<SchedulerEvent>(this.scheduler,
-            this.scheduler.getClass().getName()) {
-          @Override
-          public void handle(SchedulerEvent event) {
-            super.handle(event);
-          }
-        };
-      }
-
-      @Override
-      protected Dispatcher createDispatcher() {
-        return dispatcher;
-      }
-    };
-
-    rm1.start();
-
-    MockNM nm1 =
-        new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
-    nm1.registerNode();
-
-    MockNM nm2 = null;
-    if (!singleNode) {
-      nm2 =
-          new MockNM("127.0.0.2:2345", 8000, rm1.getResourceTrackerService());
-      nm2.registerNode();
-    }
-
-    RMApp app1 = rm1.submitApp(200);
-
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-    CapacityScheduler scheduler =
-        (CapacityScheduler) rm1.getResourceScheduler();
-    ContainerId amContainer =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
-    // Preempt the first attempt;
-    RMContainer rmContainer = scheduler.getRMContainer(amContainer);
-    NodeId nodeWhereAMRan = rmContainer.getAllocatedNode();
-
-    MockNM currentNode, otherNode;
-    if (singleNode) {
-      Assert.assertEquals(nm1.getNodeId(), nodeWhereAMRan);
-      currentNode = nm1;
-      otherNode = null; // not applicable
-    } else {
-      if (nodeWhereAMRan == nm1.getNodeId()) {
-        currentNode = nm1;
-        otherNode = nm2;
-      } else {
-        currentNode = nm2;
-        otherNode = nm1;
-      }
-    }
-
-    // set the exist status to test
-    // any status other than SUCCESS and PREEMPTED should cause the node to be
-    // blacklisted
-    int exitStatus = singleNode ?
-            ContainerExitStatus.PREEMPTED :
-            ContainerExitStatus.INVALID;
-    ContainerStatus containerStatus =
-        BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE,
-            "", exitStatus, Resources.createResource(200));
-    currentNode.containerStatus(containerStatus);
-    rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED);
-    rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
-
-    // restart the am
-    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app1, rm1);
-    System.out.println("Launch AM " + attempt.getAppAttemptId());
-
-
-
-    currentNode.nodeHeartbeat(true);
-    dispatcher.await();
-
-    if (!singleNode) {
-      Assert.assertEquals(
-          "AppAttemptState should still be SCHEDULED if currentNode is " +
-          "blacklisted correctly",
-          RMAppAttemptState.SCHEDULED,
-          attempt.getAppAttemptState());
-
-      otherNode.nodeHeartbeat(true);
-      dispatcher.await();
-    }
-
-    MockAM am2 = rm1.sendAMLaunched(attempt.getAppAttemptId());
-    rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
-    amContainer =
-        ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
-    rmContainer = scheduler.getRMContainer(amContainer);
-    nodeWhereAMRan = rmContainer.getAllocatedNode();
-    if (singleNode) {
-      // with preemption, the node should not be blacklisted and should get the
-      // assignment (with a single node)
-      Assert.assertEquals(
-          "AM should still have been able to run on the same node",
-          currentNode.getNodeId(), nodeWhereAMRan);
-    } else {
-      // with a failed status, the other node should receive the assignment
-      Assert.assertEquals(
-          "After blacklisting AM should have run on the other node",
-          otherNode.getNodeId(), nodeWhereAMRan);
-
-      am2.registerAppAttempt();
-      rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
-
-      List<Container> allocatedContainers =
-          allocateContainers(currentNode, am2, 1);
-      Assert.assertEquals(
-          "Even though AM is blacklisted from the node, application can " +
-          "still allocate containers there",
-          currentNode.getNodeId(), allocatedContainers.get(0).getNodeId());
-    }
-  }
-
-
   // AM container preempted, nm disk failure
   // should not be counted towards AM max retry count.
   @Test(timeout = 100000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.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/blacklist/TestBlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
index 96b373f..3102c6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
@@ -19,12 +19,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
 
-import org.junit.Assert;
-import org.junit.Test;
-
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.junit.Assert;
+import org.junit.Test;
+
 public class TestBlacklistManager {
 
   @Test
@@ -37,12 +38,12 @@ public class TestBlacklistManager {
     String anyNode2 = "bar";
     manager.addNode(anyNode);
     manager.addNode(anyNode2);
-    BlacklistUpdates blacklist = manager
+    ResourceBlacklistRequest blacklist = manager
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     String[] expectedBlacklistAdditions = new String[]{anyNode2, anyNode};
     Assert.assertArrayEquals(
         "Blacklist additions was not as expected",
@@ -61,12 +62,12 @@ public class TestBlacklistManager {
     String anyNode = "foo";
     String anyNode2 = "bar";
     manager.addNode(anyNode);
-    BlacklistUpdates blacklist = manager
+    ResourceBlacklistRequest blacklist = manager
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     String[] expectedBlacklistAdditions = new String[]{anyNode};
     Assert.assertArrayEquals(
         "Blacklist additions was not as expected",
@@ -81,9 +82,9 @@ public class TestBlacklistManager {
 
     blacklist = manager
         .getBlacklistUpdates();
-    blacklistAdditions = blacklist.getAdditions();
+    blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    blacklistRemovals = blacklist.getRemovals();
+    blacklistRemovals = blacklist.getBlacklistRemovals();
     Collections.sort(blacklistRemovals);
     String[] expectedBlacklistRemovals = new String[] {anyNode2, anyNode};
     Assert.assertTrue(
@@ -101,11 +102,11 @@ public class TestBlacklistManager {
     BlacklistManager disabled = new DisabledBlacklistManager();
     String anyNode = "foo";
     disabled.addNode(anyNode);
-    BlacklistUpdates blacklist = disabled
+    ResourceBlacklistRequest blacklist = disabled
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     Assert.assertTrue(
         "Blacklist additions should be empty but was " +
             blacklistAdditions,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.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/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index 293c0b6..bb5c25c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.yarn.MockApps;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -1029,63 +1028,6 @@ public class TestRMAppTransitions {
             + "/"));
   }
 
-  @Test
-  public void testAMBlackListConfigFromApp() {
-    // Scenario 1: Application enables AM blacklisting
-    float disableThreshold = 0.9f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, false);
-    ApplicationSubmissionContext submissionContext =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        true, disableThreshold));
-    RMAppImpl application = (RMAppImpl) createNewTestApp(submissionContext);
-
-    Assert.assertTrue(application.isAmBlacklistingEnabled());
-    Assert.assertEquals(disableThreshold,
-        application.getAmBlacklistingDisableThreshold(), 1e-8);
-
-    // Scenario 2: Application disables AM blacklisting
-    float globalThreshold = 0.9f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext2 =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext2.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        false, disableThreshold));
-    RMAppImpl application2 = (RMAppImpl) createNewTestApp(submissionContext2);
-
-    // Am blacklisting will be disabled eventhough its enabled in RM.
-    Assert.assertFalse(application2.isAmBlacklistingEnabled());
-
-    // Scenario 3: Application updates invalid AM threshold
-    float invalidDisableThreshold = -0.5f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext3 =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext3.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        true, invalidDisableThreshold));
-    RMAppImpl application3 = (RMAppImpl) createNewTestApp(submissionContext3);
-
-    Assert.assertTrue(application3.isAmBlacklistingEnabled());
-    Assert.assertEquals(globalThreshold,
-        application3.getAmBlacklistingDisableThreshold(), 1e-8);
-
-    // Scenario 4: Empty AMBlackListingRequest in Submission Context
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext4 =
-        new ApplicationSubmissionContextPBImpl();
-    RMAppImpl application4 = (RMAppImpl) createNewTestApp(submissionContext4);
-
-    Assert.assertTrue(application4.isAmBlacklistingEnabled());
-    Assert.assertEquals(globalThreshold,
-        application4.getAmBlacklistingDisableThreshold(), 1e-8);
-  }
-
   private void verifyApplicationFinished(RMAppState state) {
     ArgumentCaptor<RMAppState> finalState =
         ArgumentCaptor.forClass(RMAppState.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.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/scheduler/TestAppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
index 4141a53..a1c6294 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
@@ -41,32 +41,32 @@ public class TestAppSchedulingInfo {
     AppSchedulingInfo  appSchedulingInfo = new AppSchedulingInfo(
         appAttemptId, "test", queue, null, 0, new ResourceUsage());
 
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         new ArrayList<String>());
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
 
     ArrayList<String> blacklistAdditions = new ArrayList<String>();
     blacklistAdditions.add("node1");
     blacklistAdditions.add("node2");
-    appSchedulingInfo.updateBlacklist(blacklistAdditions,
+    appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
         new ArrayList<String>());
     Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
 
     blacklistAdditions.clear();
     blacklistAdditions.add("node1");
-    appSchedulingInfo.updateBlacklist(blacklistAdditions,
+    appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
         new ArrayList<String>());
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
 
     ArrayList<String> blacklistRemovals = new ArrayList<String>();
     blacklistRemovals.add("node1");
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
     Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
 
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 72d2f85..7c34292 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -684,14 +684,16 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
+        .isPlaceBlacklisted(host));
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
+        .isPlaceBlacklisted(host));
     rm.stop();
   }
-  
+
   @Test
   public void testAllocateReorder() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.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/scheduler/fair/TestFSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
index 1901fa9..af1dc62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
@@ -314,24 +314,24 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
     FSAppAttempt spyApp = spy(app);
     doReturn(false)
         .when(spyApp).isWaitingForAMContainer();
-    assertTrue(spyApp.isBlacklisted(n1.getNodeName()));
-    assertFalse(spyApp.isBlacklisted(n2.getNodeName()));
+    assertTrue(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(n2.getUnallocatedResource(), spyApp.getHeadroom());
 
     blacklistAdditions.clear();
     blacklistAdditions.add(n2.getNodeName());
     blacklistRemovals.add(n1.getNodeName());
     app.updateBlacklist(blacklistAdditions, blacklistRemovals);
-    assertFalse(spyApp.isBlacklisted(n1.getNodeName()));
-    assertTrue(spyApp.isBlacklisted(n2.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertTrue(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(n1.getUnallocatedResource(), spyApp.getHeadroom());
 
     blacklistAdditions.clear();
     blacklistRemovals.clear();
     blacklistRemovals.add(n2.getNodeName());
     app.updateBlacklist(blacklistAdditions, blacklistRemovals);
-    assertFalse(spyApp.isBlacklisted(n1.getNodeName()));
-    assertFalse(spyApp.isBlacklisted(n2.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(clusterResource, spyApp.getHeadroom());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index abf87d7..3e5a40f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -3995,11 +3995,12 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    assertTrue(app.isBlacklisted(host));
+    assertTrue(app.isPlaceBlacklisted(host));
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host));
+    assertFalse(scheduler.getSchedulerApp(appAttemptId)
+        .isPlaceBlacklisted(host));
 
     List<ResourceRequest> update = Arrays.asList(
         createResourceRequest(GB, node.getHostName(), 1, 0, true));
@@ -4008,7 +4009,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    assertTrue(app.isBlacklisted(host));
+    assertTrue(app.isPlaceBlacklisted(host));
     scheduler.update();
     scheduler.handle(updateEvent);
     assertEquals("Incorrect number of containers allocated", 0, app
@@ -4018,7 +4019,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    assertFalse(app.isBlacklisted(host));
+    assertFalse(app.isPlaceBlacklisted(host));
     createSchedulingRequest(GB, "root.default", "user", 1);
     scheduler.update();
     scheduler.handle(updateEvent);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.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/webapp/TestRMWebServicesAppsModification.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
index 38b32e9..c7ef8fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
@@ -22,7 +22,14 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 
-import java.io.*;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.StringWriter;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -43,15 +50,12 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -72,7 +76,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.*;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CredentialsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LogAggregationContextInfo;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
@@ -101,6 +111,7 @@ import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.api.client.filter.LoggingFilter;
 import com.sun.jersey.api.json.JSONJAXBContext;
 import com.sun.jersey.api.json.JSONMarshaller;
@@ -827,17 +838,6 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
         System.currentTimeMillis(), 1).toString();
     appInfo.setReservationId(reservationId);
 
-    // Set AMBlackListingRequestInfo
-    boolean isAMBlackListingEnabled = true;
-    float disableFailureThreshold = 0.01f;
-    AMBlackListingRequestInfo amBlackListingRequestInfo
-        = new AMBlackListingRequestInfo();
-    amBlackListingRequestInfo.setAMBlackListingEnabled(
-        isAMBlackListingEnabled);
-    amBlackListingRequestInfo.setBlackListingDisableFailureThreshold(
-        disableFailureThreshold);
-    appInfo.setAMBlackListingRequestInfo(amBlackListingRequestInfo);
-
     ClientResponse response =
         this.constructWebResource(urlPath).accept(acceptMedia)
           .entity(appInfo, contentMedia).post(ClientResponse.class);
@@ -912,13 +912,6 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     // Check ReservationId
     assertEquals(reservationId, app.getReservationId().toString());
 
-    // Check AMBlackListingRequestInfo
-    AMBlackListingRequest amBlackListingRequest = asc.getAMBlackListRequest();
-    assertEquals(isAMBlackListingEnabled,
-        amBlackListingRequest.isAMBlackListingEnabled());
-    assertTrue(disableFailureThreshold == amBlackListingRequest
-        .getBlackListingDisableFailureThreshold());
-
     response =
         this.constructWebResource("apps", appId).accept(acceptMedia)
           .get(ClientResponse.class);


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


[48/50] [abbrv] hadoop git commit: HADOOP-12666. Support Microsoft Azure Data Lake - as a file system in Hadoop. Contributed by Vishwajeet Dusane.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
new file mode 100644
index 0000000..7f7e749
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
@@ -0,0 +1,97 @@
+/*
+ * 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.web.resources;
+
+import java.net.HttpURLConnection;
+
+/**
+ * Extended Webhdfs PostOpParam to avoid redirect during append operation for
+ * azure data lake storage.
+ */
+
+public class ADLPostOpParam extends HttpOpParam<ADLPostOpParam.Op> {
+  private static final Domain<Op> DOMAIN = new Domain<ADLPostOpParam.Op>(NAME,
+      Op.class);
+
+  /**
+   * Constructor.
+   *
+   * @param str a string representation of the parameter value.
+   */
+  public ADLPostOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+
+  /**
+   * Post operations.
+   */
+  public static enum Op implements HttpOpParam.Op {
+    APPEND(true, false, HttpURLConnection.HTTP_OK);
+
+    private final boolean redirect;
+    private final boolean doOutput;
+    private final int expectedHttpResponseCode;
+
+    Op(final boolean doOut, final boolean doRedirect,
+        final int expectHttpResponseCode) {
+      this.doOutput = doOut;
+      this.redirect = doRedirect;
+      this.expectedHttpResponseCode = expectHttpResponseCode;
+    }
+
+    @Override
+    public Type getType() {
+      return Type.POST;
+    }
+
+    @Override
+    public boolean getRequireAuth() {
+      return false;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return doOutput;
+    }
+
+    @Override
+    public boolean getRedirect() {
+      return redirect;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    /**
+     * @return a URI query string.
+     */
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
new file mode 100644
index 0000000..d300a1c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.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.hdfs.web.resources;
+
+import java.net.HttpURLConnection;
+
+/**
+ * Extended Webhdfs PutOpParam to avoid redirect during Create operation for
+ * azure data lake storage.
+ */
+public class ADLPutOpParam extends HttpOpParam<ADLPutOpParam.Op> {
+  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   *
+   * @param str a string representation of the parameter value.
+   */
+  public ADLPutOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+
+  /**
+   * Put operations.
+   */
+  public static enum Op implements HttpOpParam.Op {
+    CREATE(true, false, HttpURLConnection.HTTP_CREATED);
+
+    private final boolean redirect;
+    private final boolean doOutput;
+    private final int expectedHttpResponseCode;
+    private final boolean requireAuth;
+
+    Op(final boolean doOut, final boolean doRedirect,
+        final int expectHttpResponseCode) {
+      this.doOutput = doOut;
+      this.redirect = doRedirect;
+      this.expectedHttpResponseCode = expectHttpResponseCode;
+      this.requireAuth = false;
+    }
+
+    @Override
+    public HttpOpParam.Type getType() {
+      return HttpOpParam.Type.PUT;
+    }
+
+    @Override
+    public boolean getRequireAuth() {
+      return requireAuth;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return doOutput;
+    }
+
+    @Override
+    public boolean getRedirect() {
+      return redirect;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
new file mode 100644
index 0000000..0bfe521
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
@@ -0,0 +1,51 @@
+/*
+ * 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.web.resources;
+
+import org.apache.hadoop.hdfs.web.ADLConfKeys;
+
+import java.util.regex.Pattern;
+
+/**
+ * Capture ADL Jar version information. Require for debugging and analysis
+ * purpose in the backend.
+ */
+public class ADLVersionInfo extends StringParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = ADLConfKeys.ADL_WEBSDK_VERSION_KEY;
+
+  private static final StringParam.Domain DOMAIN = new StringParam.Domain(NAME,
+      Pattern.compile(".+"));
+
+  /**
+   * Constructor.
+   * @param featureSetVersion Enabled featured information
+   */
+  public ADLVersionInfo(String featureSetVersion) {
+    super(DOMAIN, featureSetVersion);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
new file mode 100644
index 0000000..b9ea79e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
@@ -0,0 +1,45 @@
+/*
+ * 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.web.resources;
+
+/**
+ * Overwrite parameter.
+ */
+public class AppendADLNoRedirectParam extends BooleanParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = "append";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   *
+   * @param value the parameter value.
+   */
+  public AppendADLNoRedirectParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
new file mode 100644
index 0000000..83f3970
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
@@ -0,0 +1,44 @@
+/**
+ * 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.web.resources;
+
+/**
+ * Overwrite parameter.
+ */
+public class CreateADLNoRedirectParam extends BooleanParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = "write";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   *
+   * @param value the parameter value.
+   */
+  public CreateADLNoRedirectParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
new file mode 100644
index 0000000..6801235
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
@@ -0,0 +1,53 @@
+/*
+ * 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.web.resources;
+
+/**
+ * To support single writer semantics. Notify to ADL backend if the stream
+ * needs to locked in order to protect
+ * concurrent write operation on the same stream.
+ *
+ * Used in append operation.
+ */
+public class LeaseParam extends StringParam {
+
+  public static final String NAME = "leaseId";
+  /**
+   * Default parameter value.
+   */
+  public static final String DEFAULT = NULL;
+
+  private static final StringParam.Domain DOMAIN = new StringParam.Domain(NAME,
+      null);
+
+  /**
+   * Constructor.
+   *
+   * @param str a string representation of the parameter value.
+   */
+  public LeaseParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
new file mode 100644
index 0000000..a600161
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
@@ -0,0 +1,44 @@
+/**
+ * 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.web.resources;
+
+/**
+ * Overwrite parameter.
+ */
+public class ReadADLNoRedirectParam extends BooleanParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = "read";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   *
+   * @param value the parameter value.
+   */
+  public ReadADLNoRedirectParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
new file mode 100644
index 0000000..2231cc2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * A distributed implementation of {@link
+ * org.apache.hadoop.hdfs.web.resources} for reading or extending query
+ * parameter for webhdfs specification. ADL
+ * specific
+ * query parameter also goes in the same package.
+ */
+package org.apache.hadoop.hdfs.web.resources;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
new file mode 100644
index 0000000..4158c88
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
@@ -0,0 +1,219 @@
+<!---
+  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. See accompanying LICENSE file.
+-->
+
+# Hadoop Azure Data Lake Support
+
+* [Introduction](#Introduction)
+* [Features](#Features)
+* [Limitations](#Limitations)
+* [Usage](#Usage)
+    * [Concepts](#Concepts)
+        * [Webhdfs Compliance](#Webhdfs_Specification_Compliance)
+        * [OAuth2 Support](#OAuth2_Support)
+        * [Read Ahead Buffer Management](Read_Ahead_Buffer_Management)
+    * [Configuring Credentials & FileSystem](#Configuring_Credentials)
+    * [Accessing adl URLs](#Accessing_adl_URLs)
+* [Testing the hadoop-azure Module](#Testing_the_hadoop-azure_Module)
+
+## <a name="Introduction" />Introduction
+
+The hadoop-azure-datalake module provides support for integration with
+[Azure Data Lake Store]( https://azure.microsoft.com/en-in/documentation/services/data-lake-store/).
+The jar file is named azure-datalake-store.jar.
+
+## <a name="Features" />Features
+
+* Read and write data stored in an Azure Data Lake Storage account.
+* Partial support for [Webhdfs Specification 2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html)
+* Reference file system paths using URLs using the `adl` scheme for Secure Webhdfs i.e. SSL
+  encrypted access.
+* Can act as a source of data in a MapReduce job, or a sink.
+* Tested on both Linux and Windows.
+* Tested for scale.
+
+## <a name="Limitations" />Limitations
+Partial or no support for the following operations in [Webhdfs Specification 2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html):
+
+* Operation on Symbolic Link
+* Proxy Users
+* File Truncate
+* File Checksum
+* File replication factor
+* Home Directory Partial supported based on OAuth2 token information and not the active user on Hadoop cluster.
+* Extended Attributes(XAttrs) Operations
+* Snapshot Operations
+* Delegation Token Operations
+* User and group information returned as ListStatus and GetFileStatus is in form of GUID associated in Azure Active Directory.
+
+## <a name="Usage" />Usage
+
+### <a name="Concepts" />Concepts
+Azure Data Lake Storage access path syntax is
+
+    adl://<Account Name>.azuredatalakestore.net/
+
+Get started with azure data lake account with [https://azure.microsoft.com/en-in/documentation/articles/data-lake-store-get-started-portal/](https://azure.microsoft.com/en-in/documentation/articles/data-lake-store-get-started-portal/)
+
+#### <a name="Webhdfs_Specification_Compliance" />Webhdfs Compliance
+Azure Data Lake Storage exposes a public REST endpoint as per [Webhdfs Specification 2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html) to access storage file system.
+
+Syntax to access Azure data lake storage account over [Webhdfs Specification 2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html) is
+
+    https://<Account Name>.azuredatalakestore.net/webhdfs/v1/<File System Path>?<Query paramaters>
+
+
+#### <a name="#OAuth2_Support" />OAuth2 Support
+Usage of Azure Data Lake Storage requires OAuth2 bearer token to be present as part of the HTTPS header as per OAuth2 specification. Valid OAuth2 bearer token should be obtained from Azure Active Directory for valid users who have  access to Azure Data Lake Storage Account.
+
+Azure Active Directory (Azure AD) is Microsoft\ufffds multi-tenant cloud based directory and identity management service. See [https://azure.microsoft.com/en-in/documentation/articles/active-directory-whatis/](https://azure.microsoft.com/en-in/documentation/articles/active-directory-whatis/)
+
+Following sections describes on OAuth2 configuration in core-site.xml.
+
+#### <a name="#Read_Ahead_Buffer_Management" />Read Ahead Buffer Management
+Azure Data Lake Storage offers high throughput. To maximize throughput, applications can  use this feature to buffer data concurrently, in memory during read operation. This data is cached in memory per process per stream.
+
+
+To Enable/Disable read ahead feature.
+
+    <property>
+        <name>adl.feature.override.readahead</name>
+        <value>true</value>
+        <description>
+            Enables read aheads in the ADL client, the feature is used to improve read throughput.
+            This works in conjunction with the value set in adl.feature.override.readahead.max.buffersize.
+            When set to false the read ahead feature is turned off.
+            Default : True if not configured.
+        </description>
+    </property>
+
+To configure read ahead buffer size.
+
+    <property>
+        <name>adl.feature.override.readahead.max.buffersize</name>
+        <value>8388608</value>
+        <description>
+            Define maximum buffer size to cache read ahead data, this is allocated per process to
+            cache read ahead data. Applicable only when adl.feature.override.readahead is set to true.
+            Default : 8388608 Byte i.e. 8MB if not configured.
+        </description>
+    </property>
+
+To configure number of concurrent connection to Azure Data Lake Storage Account.
+
+    <property>
+        <name>adl.feature.override.readahead.max.concurrent.connection</name>
+        <value>2</value>
+        <description>
+            Define maximum concurrent connection can be established to
+            read ahead. If the data size is<4MB then only 1 read n/w connection
+            is set. If the data size is >4MB but<8MB then 2 read n/w
+            connection
+            is set. Data >8MB then value set under the property would
+             take
+            effect. Applicable only when adl.feature.override.readahead is set
+            to true and buffer size is >8MB.
+            It is recommended to reset this property if the adl.feature.override.readahead.max.buffersize
+            is < 8MB to gain performance. Application has to consider
+             throttling
+            limit for the account as well before configuring large buffer size.
+        </description>
+    </property>
+
+## <a name="Configuring_Credentials" />Configuring Credentials & FileSystem
+
+Update core-site.xml for OAuth2 configuration
+
+         <property>
+            <name>dfs.webhdfs.oauth2.refresh.token.expires.ms.since.epoch</name>
+            <value>0</value>
+         </property>
+
+         <property>
+            <name>dfs.webhdfs.oauth2.credential</name>
+            <value>bearer.and.refresh.token</value>
+         </property>
+
+        <property>
+            <name>dfs.webhdfs.oauth2.access.token</name>
+            <value>NOT_SET</value>
+        </property>
+
+        <property>
+            <name>dfs.webhdfs.oauth2.refresh.url</name>
+            <value>https://login.windows.net/common/oauth2/token/</value>
+        </property>
+
+        <property>
+            <name>dfs.webhdfs.oauth2.access.token.provider</name>
+            <value>org.apache.hadoop.fs.adl.oauth2.CachedRefreshTokenBasedAccessTokenProvider</value>
+        </property>
+
+Application require to set Client id and OAuth2 refresh token from Azure Active Directory associated with client id. See [https://github.com/AzureAD/azure-activedirectory-library-for-java](https://github.com/AzureAD/azure-activedirectory-library-for-java).
+
+**Do not share client id and refresh token, it must be kept secret.**
+
+        <property>
+            <name>dfs.webhdfs.oauth2.client.id</name>
+            <value></value>
+        </property>
+
+        <property>
+            <name>dfs.webhdfs.oauth2.refresh.token</name>
+            <value></value>
+        </property>
+
+For ADL FileSystem to take effect. Update core-site.xml with
+
+        <property>
+            <name>fs.adl.impl</name>
+            <value>org.apache.hadoop.fs.adl.AdlFileSystem</value>
+        </property>
+
+        <property>
+            <name>fs.AbstractFileSystem.adl.impl</name>
+            <value>org.apache.hadoop.fs.adl.Adl</value>
+        </property>
+
+
+### <a name="Accessing_adl_URLs" />Accessing adl URLs
+
+After credentials are configured in core-site.xml, any Hadoop component may
+reference files in that Azure Data Lake Storage account by using URLs of the following
+format:
+
+    adl://<Account Name>.azuredatalakestore.net/<path>
+
+The schemes `adl` identify a URL on a file system backed by Azure
+Data Lake Storage.  `adl` utilizes encrypted HTTPS access for all interaction with
+the Azure Data Lake Storage API.
+
+For example, the following
+[FileSystem Shell](../hadoop-project-dist/hadoop-common/FileSystemShell.html)
+commands demonstrate access to a storage account named `youraccount`.
+
+    > hadoop fs -mkdir adl://yourcontainer.azuredatalakestore.net/testDir
+
+    > hadoop fs -put testFile adl://yourcontainer.azuredatalakestore.net/testDir/testFile
+
+    > hadoop fs -cat adl://yourcontainer.azuredatalakestore.net/testDir/testFile
+    test file content
+## <a name="Testing_the_hadoop-azure_Module" />Testing the azure-datalake-store Module
+The hadoop-azure module includes a full suite of unit tests. Most of the tests will run without additional configuration by running mvn test. This includes tests against mocked storage, which is an in-memory emulation of Azure Data Lake Storage.
+
+A selection of tests can run against the Azure Data Lake Storage. To run tests against Adl storage. Please configure contract-test-options.xml with Adl account information mentioned in the above sections. Also turn on contract test execution flag to trigger tests against Azure Data Lake Storage.
+
+    <property>
+      <name>dfs.adl.test.contract.enable</name>
+      <value>true</value>
+    </property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
new file mode 100644
index 0000000..e57d3a9
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
@@ -0,0 +1,147 @@
+/*
+ * 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.adl.oauth2;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.oauth2.AccessTokenProvider;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_KEY;
+
+/**
+ * Verify cache behavior of ConfRefreshTokenBasedAccessTokenProvider instances.
+ */
+public class TestCachedRefreshTokenBasedAccessTokenProvider {
+
+  private Configuration conf;
+
+  @Rule public TestName name = new TestName();
+  String clientId(int id) {
+    return name.getMethodName() + "_clientID" + id;
+  }
+
+  @Before
+  public void initConfig() {
+    conf = new Configuration(false);
+    conf.set(OAUTH_CLIENT_ID_KEY, clientId(0));
+    conf.set(OAUTH_REFRESH_TOKEN_KEY, "01234567890abcdef");
+    conf.set(OAUTH_REFRESH_URL_KEY, "http://dingo.invalid:80");
+  }
+
+  @Test
+  public void testCacheInstance() throws Exception {
+    final AccessTokenProvider inst0 = mock(AccessTokenProvider.class);
+    when(inst0.getConf()).thenReturn(conf);
+
+    // verify config
+    CachedRefreshTokenBasedAccessTokenProvider t1 = new MockProvider(inst0);
+    t1.setConf(conf);
+    verify(inst0).setConf(any(Configuration.class)); // cloned, not exact match
+
+    // verify cache hit
+    CachedRefreshTokenBasedAccessTokenProvider t2 =
+        new CachedRefreshTokenBasedAccessTokenProvider() {
+          @Override
+          AccessTokenProvider newInstance() {
+            fail("Failed to return cached instance");
+            return null;
+          }
+        };
+    t2.setConf(conf);
+
+    // verify force refresh
+    conf.setBoolean(
+        CachedRefreshTokenBasedAccessTokenProvider.FORCE_REFRESH, true);
+    final AccessTokenProvider inst1 = mock(AccessTokenProvider.class);
+    when(inst1.getConf()).thenReturn(conf);
+    CachedRefreshTokenBasedAccessTokenProvider t3 = new MockProvider(inst1);
+    t3.setConf(conf);
+    verify(inst1).setConf(any(Configuration.class));
+
+    // verify cache miss
+    conf.set(OAUTH_REFRESH_URL_KEY, "http://yak.invalid:80");
+    final AccessTokenProvider inst2 = mock(AccessTokenProvider.class);
+    when(inst2.getConf()).thenReturn(conf);
+    CachedRefreshTokenBasedAccessTokenProvider t4 = new MockProvider(inst2);
+    t4.setConf(conf);
+    verify(inst2).setConf(any(Configuration.class));
+  }
+
+  @Test
+  public void testCacheLimit() throws Exception {
+    final int iter = CachedRefreshTokenBasedAccessTokenProvider.MAX_PROVIDERS;
+    for (int i = 0; i < iter; ++i) {
+      conf.set(OAUTH_CLIENT_ID_KEY, clientId(i));
+      AccessTokenProvider inst = mock(AccessTokenProvider.class);
+      when(inst.getConf()).thenReturn(conf);
+      CachedRefreshTokenBasedAccessTokenProvider t = new MockProvider(inst);
+      t.setConf(conf);
+      verify(inst).setConf(any(Configuration.class));
+    }
+    // verify cache hit
+    for (int i = 0; i < iter; ++i) {
+      conf.set(OAUTH_CLIENT_ID_KEY, clientId(i));
+      CachedRefreshTokenBasedAccessTokenProvider t =
+          new CachedRefreshTokenBasedAccessTokenProvider() {
+            @Override
+            AccessTokenProvider newInstance() {
+              fail("Failed to return cached instance");
+              return null;
+            }
+          };
+      t.setConf(conf);
+    }
+
+    // verify miss, evict 0
+    conf.set(OAUTH_CLIENT_ID_KEY, clientId(iter));
+    final AccessTokenProvider inst = mock(AccessTokenProvider.class);
+    when(inst.getConf()).thenReturn(conf);
+    CachedRefreshTokenBasedAccessTokenProvider t = new MockProvider(inst);
+    t.setConf(conf);
+    verify(inst).setConf(any(Configuration.class));
+
+    // verify miss
+    conf.set(OAUTH_CLIENT_ID_KEY, clientId(0));
+    final AccessTokenProvider inst0 = mock(AccessTokenProvider.class);
+    when(inst0.getConf()).thenReturn(conf);
+    CachedRefreshTokenBasedAccessTokenProvider t0 = new MockProvider(inst0);
+    t0.setConf(conf);
+    verify(inst0).setConf(any(Configuration.class));
+  }
+
+  static class MockProvider extends CachedRefreshTokenBasedAccessTokenProvider {
+    private final AccessTokenProvider inst;
+    MockProvider(AccessTokenProvider inst) {
+      this.inst = inst;
+    }
+    @Override
+    AccessTokenProvider newInstance() {
+      return inst;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-tools-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml
index e1fbef1..9485741 100644
--- a/hadoop-tools/hadoop-tools-dist/pom.xml
+++ b/hadoop-tools/hadoop-tools-dist/pom.xml
@@ -105,6 +105,12 @@
       <artifactId>hadoop-sls</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-azure-datalake</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index bd5f784..a2c441b 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -46,6 +46,7 @@
     <module>hadoop-azure</module>
     <module>hadoop-aws</module>
     <module>hadoop-kafka</module>
+    <module>hadoop-azure-datalake</module>
   </modules>
 
   <build>


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


[31/50] [abbrv] hadoop git commit: Addendum patch for YARN-5180 updating findbugs-exclude.xml

Posted by ae...@apache.org.
Addendum patch for YARN-5180 updating findbugs-exclude.xml


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

Branch: refs/heads/HDFS-7240
Commit: 8554aee1bef5aff9e49e5e9119d6a7a4abf1c432
Parents: 733f3f1
Author: Arun Suresh <as...@apache.org>
Authored: Tue Jun 7 15:59:13 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Tue Jun 7 15:59:43 2016 -0700

----------------------------------------------------------------------
 .../hadoop-yarn/dev-support/findbugs-exclude.xml                | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8554aee1/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 81c7e6a..6998d75 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -517,6 +517,11 @@
   </Match>
 
   <Match>
+    <Class name="org.apache.hadoop.yarn.api.records.ResourceRequest" />
+    <Method name="equals" />
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE" />
+  </Match>
+  <Match>
     <Package name="org.apache.hadoop.yarn.api.records.impl.pb" />
     <Bug pattern="NP_BOOLEAN_RETURN_NULL" />
   </Match>


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


[04/50] [abbrv] 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-7240
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


[39/50] [abbrv] hadoop git commit: HDFS-10220. A large number of expired leases can make namenode unresponsive and cause failover (Nicolas Fraison via raviprak)

Posted by ae...@apache.org.
HDFS-10220. A large number of expired leases can make namenode unresponsive and cause failover (Nicolas Fraison via raviprak)


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

Branch: refs/heads/HDFS-7240
Commit: ae047655f4355288406cd5396fb4e3ea7c307b14
Parents: 0af96a1
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Wed Jun 8 13:44:22 2016 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Wed Jun 8 13:44:22 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   | 10 +++++
 .../hdfs/server/common/HdfsServerConstants.java |  1 -
 .../hdfs/server/namenode/FSNamesystem.java      | 42 ++++++++++++++++----
 .../hdfs/server/namenode/LeaseManager.java      | 21 ++++++++--
 .../src/main/resources/hdfs-default.xml         | 18 +++++++++
 .../hdfs/server/namenode/TestLeaseManager.java  | 24 ++++++-----
 6 files changed, 94 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 19e1791..f18a6c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -397,6 +397,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT = 16384;
   public static final int     DFS_NAMENODE_MAX_XATTR_SIZE_HARD_LIMIT = 32768;
 
+  public static final String  DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY =
+      "dfs.namenode.lease-recheck-interval-ms";
+  public static final long    DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT =
+      2000;
+  public static final String
+      DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY =
+      "dfs.namenode.max-lock-hold-to-release-lease-ms";
+  public static final long
+      DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT = 25;
+
   public static final String  DFS_UPGRADE_DOMAIN_FACTOR = "dfs.namenode.upgrade.domain.factor";
   public static final int DFS_UPGRADE_DOMAIN_FACTOR_DEFAULT = DFS_REPLICATION_DEFAULT;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index b2dda3c..3798394 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -361,7 +361,6 @@ public interface HdfsServerConstants {
   }
   
   String NAMENODE_LEASE_HOLDER = "HDFS_NameNode";
-  long NAMENODE_LEASE_RECHECK_INTERVAL = 2000;
 
   String CRYPTO_XATTR_ENCRYPTION_ZONE =
       "raw.hdfs.crypto.encryption.zone";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/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 c9f2487..915ae97 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
@@ -76,6 +76,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPI
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT;
@@ -385,7 +389,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final UserGroupInformation fsOwner;
   private final String supergroup;
   private final boolean standbyShouldCheckpoint;
-  
+
+  /** Interval between each check of lease to release. */
+  private final long leaseRecheckIntervalMs;
+  /** Maximum time the lock is hold to release lease. */
+  private final long maxLockHoldToReleaseLeaseMs;
+
   // Scan interval is not configurable.
   private static final long DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL =
     TimeUnit.MILLISECONDS.convert(1, TimeUnit.HOURS);
@@ -803,6 +812,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_KEY,
           DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_DEFAULT);
 
+      this.leaseRecheckIntervalMs = conf.getLong(
+          DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY,
+          DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT);
+      this.maxLockHoldToReleaseLeaseMs = conf.getLong(
+          DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY,
+          DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT);
+
       // For testing purposes, allow the DT secret manager to be started regardless
       // of whether security is enabled.
       alwaysUseDelegationTokensForTests = conf.getBoolean(
@@ -847,6 +863,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return retryCache;
   }
 
+  @VisibleForTesting
+  public long getLeaseRecheckIntervalMs() {
+    return leaseRecheckIntervalMs;
+  }
+
+  @VisibleForTesting
+  public long getMaxLockHoldToReleaseLeaseMs() {
+    return maxLockHoldToReleaseLeaseMs;
+  }
+
   void lockRetryCache() {
     if (retryCache != null) {
       retryCache.lock();
@@ -3116,9 +3142,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if(nrCompleteBlocks == nrBlocks) {
       finalizeINodeFileUnderConstruction(src, pendingFile,
           iip.getLatestSnapshotId(), false);
-      NameNode.stateChangeLog.warn("BLOCK*"
-        + " internalReleaseLease: All existing blocks are COMPLETE,"
-        + " lease removed, file closed.");
+      NameNode.stateChangeLog.warn("BLOCK*" +
+          " internalReleaseLease: All existing blocks are COMPLETE," +
+          " lease removed, file " + src + " closed.");
       return true;  // closed!
     }
 
@@ -3155,9 +3181,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           blockManager.hasMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId(), false);
-        NameNode.stateChangeLog.warn("BLOCK*"
-          + " internalReleaseLease: Committed blocks are minimally replicated,"
-          + " lease removed, file closed.");
+        NameNode.stateChangeLog.warn("BLOCK*" +
+            " internalReleaseLease: Committed blocks are minimally" +
+            " replicated, lease removed, file" + src + " closed.");
         return true;  // closed!
       }
       // Cannot close file right now, since some blocks 
@@ -3200,7 +3226,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId(), false);
         NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: "
-            + "Removed empty last block and closed file.");
+            + "Removed empty last block and closed file " + src);
         return true;
       }
       // start recovery of the last block for this file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index e97aa53..06f6586 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -336,7 +336,7 @@ public class LeaseManager {
             }
           }
   
-          Thread.sleep(HdfsServerConstants.NAMENODE_LEASE_RECHECK_INTERVAL);
+          Thread.sleep(fsnamesystem.getLeaseRecheckIntervalMs());
         } catch(InterruptedException ie) {
           if (LOG.isDebugEnabled()) {
             LOG.debug(name + " is interrupted", ie);
@@ -356,8 +356,11 @@ public class LeaseManager {
     boolean needSync = false;
     assert fsnamesystem.hasWriteLock();
 
-    while(!sortedLeases.isEmpty() && sortedLeases.peek().expiredHardLimit()) {
-      Lease leaseToCheck = sortedLeases.poll();
+    long start = monotonicNow();
+
+    while(!sortedLeases.isEmpty() && sortedLeases.peek().expiredHardLimit()
+      && !isMaxLockHoldToReleaseLease(start)) {
+      Lease leaseToCheck = sortedLeases.peek();
       LOG.info(leaseToCheck + " has expired hard limit");
 
       final List<Long> removing = new ArrayList<>();
@@ -397,6 +400,11 @@ public class LeaseManager {
               + leaseToCheck, e);
           removing.add(id);
         }
+        if (isMaxLockHoldToReleaseLease(start)) {
+          LOG.debug("Breaking out of checkLeases after " +
+              fsnamesystem.getMaxLockHoldToReleaseLeaseMs() + "ms.");
+          break;
+        }
       }
 
       for(Long id : removing) {
@@ -407,6 +415,13 @@ public class LeaseManager {
     return needSync;
   }
 
+
+  /** @return true if max lock hold is reached */
+  private boolean isMaxLockHoldToReleaseLease(long start) {
+    return monotonicNow() - start >
+        fsnamesystem.getMaxLockHoldToReleaseLeaseMs();
+  }
+
   @Override
   public synchronized String toString() {
     return getClass().getSimpleName() + "= {"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 79f7911..fc2f942 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2590,6 +2590,24 @@
 </property>
 
 <property>
+  <name>dfs.namenode.lease-recheck-interval-ms</name>
+  <value>2000</value>
+  <description>During the release of lease a lock is hold that make any
+    operations on the namenode stuck. In order to not block them during
+    a too long duration we stop releasing lease after this max lock limit.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.max-lock-hold-to-release-lease-ms</name>
+  <value>25</value>
+  <description>During the release of lease a lock is hold that make any
+    operations on the namenode stuck. In order to not block them during
+    a too long duration we stop releasing lease after this max lock limit.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.startup.delay.block.deletion.sec</name>
   <value>0</value>
   <description>The delay in seconds at which we will pause the blocks deletion

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
index 3bb7bb7..f823745 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
@@ -39,6 +40,8 @@ public class TestLeaseManager {
   @Rule
   public Timeout timeout = new Timeout(300000);
 
+  public static long maxLockHoldToReleaseLeaseMs = 100;
+
   @Test
   public void testRemoveLeases() throws Exception {
     FSNamesystem fsn = mock(FSNamesystem.class);
@@ -57,28 +60,28 @@ public class TestLeaseManager {
     assertEquals(0, lm.getINodeIdWithLeases().size());
   }
 
-  /** Check that even if LeaseManager.checkLease is not able to relinquish
-   * leases, the Namenode does't enter an infinite loop while holding the FSN
-   * write lock and thus become unresponsive
+  /** Check that LeaseManager.checkLease release some leases
    */
   @Test
-  public void testCheckLeaseNotInfiniteLoop() {
+  public void testCheckLease() {
     LeaseManager lm = new LeaseManager(makeMockFsNameSystem());
 
+    long numLease = 100;
+
     //Make sure the leases we are going to add exceed the hard limit
     lm.setLeasePeriod(0, 0);
 
-    //Add some leases to the LeaseManager
-    lm.addLease("holder1", INodeId.ROOT_INODE_ID + 1);
-    lm.addLease("holder2", INodeId.ROOT_INODE_ID + 2);
-    lm.addLease("holder3", INodeId.ROOT_INODE_ID + 3);
-    assertEquals(lm.countLease(), 3);
+    for (long i = 0; i <= numLease - 1; i++) {
+      //Add some leases to the LeaseManager
+      lm.addLease("holder"+i, INodeId.ROOT_INODE_ID + i);
+    }
+    assertEquals(numLease, lm.countLease());
 
     //Initiate a call to checkLease. This should exit within the test timeout
     lm.checkLeases();
+    assertTrue(lm.countLease() < numLease);
   }
 
-
   @Test
   public void testCountPath() {
     LeaseManager lm = new LeaseManager(makeMockFsNameSystem());
@@ -112,6 +115,7 @@ public class TestLeaseManager {
     when(fsn.isRunning()).thenReturn(true);
     when(fsn.hasWriteLock()).thenReturn(true);
     when(fsn.getFSDirectory()).thenReturn(dir);
+    when(fsn.getMaxLockHoldToReleaseLeaseMs()).thenReturn(maxLockHoldToReleaseLeaseMs);
     return fsn;
   }
 


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


[30/50] [abbrv] hadoop git commit: MAPREDUCE-6702. TestMiniMRChildTask.testTaskEnv and TestMiniMRChildTask.testTaskOldEnv are failing (ajisakaa via rkanter)

Posted by ae...@apache.org.
MAPREDUCE-6702. TestMiniMRChildTask.testTaskEnv and TestMiniMRChildTask.testTaskOldEnv are failing (ajisakaa via rkanter)


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

Branch: refs/heads/HDFS-7240
Commit: 733f3f18d5cf38cbae35146fbef8e16e35fdf5e1
Parents: 620325e
Author: Robert Kanter <rk...@apache.org>
Authored: Tue Jun 7 15:46:06 2016 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Tue Jun 7 15:46:06 2016 -0700

----------------------------------------------------------------------
 .../src/site/markdown/SingleCluster.md.vm       |  12 +-
 .../java/org/apache/hadoop/mapred/JobConf.java  |   6 -
 .../hadoop/mapred/TestMiniMRChildTask.java      | 233 +++----------------
 3 files changed, 41 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/733f3f18/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 573ca32..4825e00 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
@@ -181,13 +181,23 @@ You can run a MapReduce job on YARN in a pseudo-distributed mode by setting a fe
 
 The following instructions assume that 1. ~ 4. steps of [the above instructions](#Execution) are already executed.
 
-1.  Configure parameters as follows:`etc/hadoop/mapred-site.xml`:
+1.  Configure parameters as follows:
+
+    `etc/hadoop/mapred-site.xml`:
 
         <configuration>
             <property>
                 <name>mapreduce.framework.name</name>
                 <value>yarn</value>
             </property>
+            <property>
+                <name>mapreduce.admin.user.env</name>
+                <value>HADOOP_MAPRED_HOME=$HADOOP_COMMON_HOME</value>
+            </property>
+            <property>
+                <name>yarn.app.mapreduce.am.env</name>
+                <value>HADOOP_MAPRED_HOME=$HADOOP_COMMON_HOME</value>
+            </property>
         </configuration>
 
     `etc/hadoop/yarn-site.xml`:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/733f3f18/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
index 2cfce1f..f2b0aae 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
@@ -294,8 +294,6 @@ public class JobConf extends Configuration {
    * Example:
    * <ul>
    *   <li> A=foo - This will set the env variable A to foo. </li>
-   *   <li> B=$X:c This is inherit tasktracker's X env variable on Linux. </li>
-   *   <li> B=%X%;c This is inherit tasktracker's X env variable on Windows. </li>
    * </ul>
    * 
    * @deprecated Use {@link #MAPRED_MAP_TASK_ENV} or 
@@ -314,8 +312,6 @@ public class JobConf extends Configuration {
    * Example:
    * <ul>
    *   <li> A=foo - This will set the env variable A to foo. </li>
-   *   <li> B=$X:c This is inherit tasktracker's X env variable on Linux. </li>
-   *   <li> B=%X%;c This is inherit tasktracker's X env variable on Windows. </li>
    * </ul>
    */
   public static final String MAPRED_MAP_TASK_ENV = JobContext.MAP_ENV;
@@ -330,8 +326,6 @@ public class JobConf extends Configuration {
    * Example:
    * <ul>
    *   <li> A=foo - This will set the env variable A to foo. </li>
-   *   <li> B=$X:c This is inherit tasktracker's X env variable on Linux. </li>
-   *   <li> B=%X%;c This is inherit tasktracker's X env variable on Windows. </li>
    * </ul>
    */
   public static final String MAPRED_REDUCE_TASK_ENV = JobContext.REDUCE_ENV;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/733f3f18/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
index cbeeccf..f690118 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
@@ -25,9 +25,7 @@ import static org.junit.Assert.fail;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Iterator;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -41,11 +39,9 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
-import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.util.Shell;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -111,29 +107,6 @@ public class TestMiniMRChildTask {
         }
       }
   }
-  
-  /**
-   * Map class which checks if hadoop lib location 
-   * is in the execution path
-   */
-  public static class ExecutionEnvCheckMapClass extends MapReduceBase
-      implements Mapper<LongWritable, Text, Text, IntWritable> {
-      public void map (LongWritable key, Text value, 
-          OutputCollector<Text, IntWritable> output, 
-          Reporter reporter) throws IOException {
-      }
-      public void configure(JobConf job) {
-        String executionEnvPathVariable = System.getenv(Shell.WINDOWS ? "PATH"
-            : "LD_LIBRARY_PATH");
-        String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
-        if (hadoopHome == null) {
-          hadoopHome = "";
-        }
-        String hadoopLibLocation = hadoopHome 
-            + (Shell.WINDOWS ? "\\bin" : "/lib/native");
-        assertTrue(executionEnvPathVariable.contains(hadoopLibLocation));
-      }
-  }
 
   // configure a job
   private void configure(JobConf conf, Path inDir, Path outDir, String input,
@@ -167,57 +140,31 @@ public class TestMiniMRChildTask {
     conf.set("test.build.data", TEST_ROOT_DIR);
   }
 
-  /**
-   * Launch tests 
-   * @param conf Configuration of the mapreduce job.
-   * @param inDir input path
-   * @param outDir output path
-   * @param input Input text
-   * @throws IOException
-   */
-  public void launchTest(JobConf conf,
-                         Path inDir,
-                         Path outDir,
-                         String input)
-  throws IOException, InterruptedException, ClassNotFoundException {
-
-    FileSystem outFs = outDir.getFileSystem(conf);
-    
-    // Launch job with default option for temp dir. 
-    // i.e. temp dir is ./tmp 
-    Job job = Job.getInstance(conf);
-    job.addFileToClassPath(APP_JAR);
-    job.setJarByClass(TestMiniMRChildTask.class);
-    job.setMaxMapAttempts(1); // speed up failures
-    job.waitForCompletion(true);
-    boolean succeeded = job.waitForCompletion(true);
-    assertTrue(succeeded);
-    outFs.delete(outDir, true);
-  }
-
   private static void checkEnv(String envName, String expValue, String mode) {
-    String envValue = System.getenv(envName).trim();
+    String envValue = System.getenv(envName);
     if ("append".equals(mode)) {
       if (envValue == null || !envValue.contains(File.pathSeparator)) {
         throw new RuntimeException("Missing env variable");
       } else {
-        String parts[] = envValue.split(File.pathSeparator);
+        String[] parts = envValue.trim().split(File.pathSeparator);
         // check if the value is appended
         if (!parts[parts.length - 1].equals(expValue)) {
           throw new RuntimeException("Wrong env variable in append mode");
         }
       }
     } else {
-      if (envValue == null || !envValue.equals(expValue)) {
+      if (envValue == null || !envValue.trim().equals(expValue)) {
         throw new RuntimeException("Wrong env variable in noappend mode");
       }
     }
   }
 
   // Mappers that simply checks if the desired user env are present or not
-  static class EnvCheckMapper extends MapReduceBase implements
+  private static class EnvCheckMapper extends MapReduceBase implements
       Mapper<WritableComparable, Writable, WritableComparable, Writable> {
-    
+
+    @Override
+    @SuppressWarnings("deprecation")
     public void configure(JobConf job) {
       boolean oldConfigs = job.getBoolean(OLD_CONFIGS, false);
       if (oldConfigs) {
@@ -236,38 +183,12 @@ public class TestMiniMRChildTask {
                      mapJavaOpts, MAP_OPTS_VAL);
       }
 
-      String path = job.get("path");
-      
-      // check if the pwd is there in LD_LIBRARY_PATH
-      String pwd = System.getenv("PWD");
-      
-      assertTrue("LD doesnt contain pwd", 
-                 System.getenv("LD_LIBRARY_PATH").contains(pwd));
-      
-      // check if X=$X:/abc works for LD_LIBRARY_PATH
-      checkEnv("LD_LIBRARY_PATH", "/tmp", "append");
       // check if X=y works for an already existing parameter
       checkEnv("LANG", "en_us_8859_1", "noappend");
       // check if X=/tmp for a new env variable
       checkEnv("MY_PATH", "/tmp", "noappend");
       // check if X=$X:/tmp works for a new env var and results into :/tmp
       checkEnv("NEW_PATH", File.pathSeparator + "/tmp", "noappend");
-      // check if X=$(tt's X var):/tmp for an old env variable inherited from 
-      // the tt
-      if (Shell.WINDOWS) {
-        // On Windows, PATH is replaced one more time as part of default config
-        // of "mapreduce.admin.user.env", i.e. on Windows,
-        // "mapreduce.admin.user.env" is set to
-        // "PATH=%PATH%;%HADOOP_COMMON_HOME%\\bin"
-        String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
-        if (hadoopHome == null) {
-          hadoopHome = "";
-        }
-        String hadoopLibLocation = hadoopHome + "\\bin";
-        path += File.pathSeparator + hadoopLibLocation;
-        path += File.pathSeparator + path;
-      }
-      checkEnv("PATH",  path + File.pathSeparator + "/tmp", "noappend");
 
       String jobLocalDir = job.get(MRJobConfig.JOB_LOCAL_DIR);
       assertNotNull(MRJobConfig.JOB_LOCAL_DIR + " is null",
@@ -281,10 +202,12 @@ public class TestMiniMRChildTask {
     }
   }
 
-  static class EnvCheckReducer extends MapReduceBase 
-  implements Reducer<WritableComparable, Writable, WritableComparable, Writable> {
-    
+  private static class EnvCheckReducer extends MapReduceBase
+      implements Reducer<WritableComparable, Writable,
+                         WritableComparable, Writable> {
+
     @Override
+    @SuppressWarnings("deprecation")
     public void configure(JobConf job) {
       boolean oldConfigs = job.getBoolean(OLD_CONFIGS, false);
       if (oldConfigs) {
@@ -303,39 +226,12 @@ public class TestMiniMRChildTask {
                      reduceJavaOpts, REDUCE_OPTS_VAL);
       }
 
-      String path = job.get("path");
-      
-      // check if the pwd is there in LD_LIBRARY_PATH
-      String pwd = System.getenv("PWD");
-      
-      assertTrue("LD doesnt contain pwd", 
-                 System.getenv("LD_LIBRARY_PATH").contains(pwd));
-      
-      // check if X=$X:/abc works for LD_LIBRARY_PATH
-      checkEnv("LD_LIBRARY_PATH", "/tmp", "append");
       // check if X=y works for an already existing parameter
       checkEnv("LANG", "en_us_8859_1", "noappend");
       // check if X=/tmp for a new env variable
       checkEnv("MY_PATH", "/tmp", "noappend");
       // check if X=$X:/tmp works for a new env var and results into :/tmp
       checkEnv("NEW_PATH", File.pathSeparator + "/tmp", "noappend");
-      // check if X=$(tt's X var):/tmp for an old env variable inherited from 
-      // the tt
-      if (Shell.WINDOWS) {
-        // On Windows, PATH is replaced one more time as part of default config
-        // of "mapreduce.admin.user.env", i.e. on Windows,
-        // "mapreduce.admin.user.env"
-        // is set to "PATH=%PATH%;%HADOOP_COMMON_HOME%\\bin"
-        String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
-        if (hadoopHome == null) {
-          hadoopHome = "";
-        }
-        String hadoopLibLocation = hadoopHome + "\\bin";
-        path += File.pathSeparator + hadoopLibLocation;
-        path += File.pathSeparator + path;
-      }
-      checkEnv("PATH",  path + File.pathSeparator + "/tmp", "noappend");
-
     }
 
     @Override
@@ -393,72 +289,9 @@ public class TestMiniMRChildTask {
   }
  
   /**
-   * To test OS dependent setting of default execution path for a MapRed task.
-   * Mainly that we can use MRJobConfig.DEFAULT_MAPRED_ADMIN_USER_ENV to set -
-   * for WINDOWS: %HADOOP_COMMON_HOME%\bin is expected to be included in PATH -
-   * for Linux: $HADOOP_COMMON_HOME/lib/native is expected to be included in
-   * LD_LIBRARY_PATH
-   */
-  @Test
-  public void testMapRedExecutionEnv() {
-    // for windows, test if the env variable can be set
-    // this may be removed as part of MAPREDUCE-6588
-    if (Shell.WINDOWS) {
-      try {
-        // Application environment
-        Map<String, String> environment = new HashMap<String, String>();
-        String setupHadoopHomeCommand =
-          "HADOOP_COMMON_HOME=C:\\fake\\PATH\\to\\hadoop\\common\\home";
-        MRApps.setEnvFromInputString(environment, setupHadoopHomeCommand, conf);
-
-        // Add the env variables passed by the admin
-        MRApps.setEnvFromInputString(environment, conf.get(
-            MRJobConfig.MAPRED_ADMIN_USER_ENV,
-            MRJobConfig.DEFAULT_MAPRED_ADMIN_USER_ENV), conf);
-
-        String executionPaths = environment.get("PATH");
-        String toFind =
-            "C:\\fake\\PATH\\to\\hadoop\\common\\home\\bin";
-
-        // Ensure execution PATH/LD_LIBRARY_PATH set up pointing to hadoop lib
-        assertTrue("execution path does not include the hadoop lib location "
-            + toFind, executionPaths.contains(toFind));
-      } catch (Exception e) {
-        e.printStackTrace();
-        fail("Exception in testing execution environment for MapReduce task");
-        tearDown();
-      }
-    }
-
-    // now launch a mapreduce job to ensure that the child 
-    // also gets the configured setting for hadoop lib
-    try {
-
-      JobConf conf = new JobConf(mr.getConfig());
-      // initialize input, output directories
-      Path rootDir = new Path(System.getProperty("test.build.data",
-          "build/test/data"));
-      Path inDir = new Path(rootDir, "input");
-      Path outDir = new Path(rootDir, "output");
-      String input = "The input";
-
-      // set config to use the ExecutionEnvCheckMapClass map class
-      configure(conf, inDir, outDir, input, 
-          ExecutionEnvCheckMapClass.class, IdentityReducer.class);
-      launchTest(conf, inDir, outDir, input);
-
-    } catch(Exception e) {
-      e.printStackTrace();
-      fail("Exception in testing propagation of env setting to child task");
-      tearDown();
-    }
-  }
-  
-  /**
    * Test to test if the user set env variables reflect in the child
    * processes. Mainly
    *   - x=y (x can be a already existing env variable or a new variable)
-   *   - x=$x:y (replace $x with the current value of x)
    */
   @Test
   public void testTaskEnv(){
@@ -482,10 +315,9 @@ public class TestMiniMRChildTask {
    * Test to test if the user set *old* env variables reflect in the child
    * processes. Mainly
    *   - x=y (x can be a already existing env variable or a new variable)
-   *   - x=$x:y (replace $x with the current value of x)
    */
   @Test
-  public void testTaskOldEnv(){
+  public void testTaskOldEnv() {
     try {
       JobConf conf = new JobConf(mr.getConfig());
       String baseDir = System.getProperty("test.build.data", "build/test/data");
@@ -495,23 +327,23 @@ public class TestMiniMRChildTask {
       FileSystem outFs = outDir.getFileSystem(conf);
       runTestTaskEnv(conf, inDir, outDir, true);
       outFs.delete(outDir, true);
-    } catch(Exception e) {
+    } catch (Exception e) {
       e.printStackTrace();
       fail("Exception in testing child env");
       tearDown();
     }
   }
-  
-  void runTestTaskEnv(JobConf conf, Path inDir, Path outDir, boolean oldConfigs) 
-  throws IOException, InterruptedException, ClassNotFoundException {
+
+  @SuppressWarnings("deprecation")
+  private void runTestTaskEnv(JobConf config, Path inDir, Path outDir,
+                              boolean oldConfigs)
+      throws IOException, InterruptedException, ClassNotFoundException {
     String input = "The input";
-    configure(conf, inDir, outDir, input, 
+    configure(config, inDir, outDir, input,
               EnvCheckMapper.class, EnvCheckReducer.class);
     // test 
     //  - new SET of new var (MY_PATH)
     //  - set of old var (LANG)
-    //  - append to an old var from modified env (LD_LIBRARY_PATH)
-    //  - append to an old var from tt's env (PATH)
     //  - append to a new var (NEW_PATH)
     String mapTaskEnvKey = JobConf.MAPRED_MAP_TASK_ENV;
     String reduceTaskEnvKey = JobConf.MAPRED_MAP_TASK_ENV;
@@ -519,29 +351,24 @@ public class TestMiniMRChildTask {
     String reduceTaskJavaOptsKey = JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS;
     String mapTaskJavaOpts = MAP_OPTS_VAL;
     String reduceTaskJavaOpts = REDUCE_OPTS_VAL;
-    conf.setBoolean(OLD_CONFIGS, oldConfigs);
+    config.setBoolean(OLD_CONFIGS, oldConfigs);
     if (oldConfigs) {
       mapTaskEnvKey = reduceTaskEnvKey = JobConf.MAPRED_TASK_ENV;
       mapTaskJavaOptsKey = reduceTaskJavaOptsKey = JobConf.MAPRED_TASK_JAVA_OPTS;
       mapTaskJavaOpts = reduceTaskJavaOpts = TASK_OPTS_VAL;
     }
-    conf.set(
+    config.set(
         mapTaskEnvKey,
-        Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=%LD_LIBRARY_PATH%;/tmp,"
-            + "PATH=%PATH%;/tmp,NEW_PATH=%NEW_PATH%;/tmp"
-            : "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp,"
-                + "PATH=$PATH:/tmp,NEW_PATH=$NEW_PATH:/tmp");
-    conf.set(
+        Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=%MY_PATH%;/tmp"
+            : "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=$NEW_PATH:/tmp");
+    config.set(
         reduceTaskEnvKey,
-        Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=%LD_LIBRARY_PATH%;/tmp,"
-            + "PATH=%PATH%;/tmp,NEW_PATH=%NEW_PATH%;/tmp"
-            : "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp,"
-                + "PATH=$PATH:/tmp,NEW_PATH=$NEW_PATH:/tmp");
-    conf.set("path", System.getenv("PATH"));
-    conf.set(mapTaskJavaOptsKey, mapTaskJavaOpts);
-    conf.set(reduceTaskJavaOptsKey, reduceTaskJavaOpts);
+        Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=%MY_PATH%;/tmp"
+            : "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=$NEW_PATH:/tmp");
+    config.set(mapTaskJavaOptsKey, mapTaskJavaOpts);
+    config.set(reduceTaskJavaOptsKey, reduceTaskJavaOpts);
 
-    Job job = Job.getInstance(conf);
+    Job job = Job.getInstance(config);
     job.addFileToClassPath(APP_JAR);
     job.setJarByClass(TestMiniMRChildTask.class);
     job.setMaxMapAttempts(1); // speed up failures


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


[18/50] [abbrv] 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-7240
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


[11/50] [abbrv] 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-7240
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