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 zj...@apache.org on 2015/03/18 04:33:53 UTC

[01/50] hadoop git commit: HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt synchronization. (Sean Busbey via yliu)

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2928 fb1b59600 -> 8a637914c


HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt synchronization. (Sean Busbey via yliu)


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

Branch: refs/heads/YARN-2928
Commit: a85291003cf3e3fd79b6addcf59d4f43dc72d356
Parents: 8212877
Author: yliu <yl...@apache.org>
Authored: Fri Mar 13 02:25:02 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Fri Mar 13 02:25:02 2015 +0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt  |  3 +++
 .../apache/hadoop/crypto/CryptoOutputStream.java | 19 ++++++++++++-------
 2 files changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8529100/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 6970bad..55028cb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1097,6 +1097,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11693. Azure Storage FileSystem rename operations are throttled too
     aggressively to complete HBase WAL archiving. (Duo Xu via cnauroth)
 
+    HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt
+    synchronization. (Sean Busbey via yliu)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8529100/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
index f1ea0fc..bc09b8c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
@@ -40,6 +40,9 @@ import com.google.common.base.Preconditions;
  * padding = pos%(algorithm blocksize); 
  * <p/>
  * The underlying stream offset is maintained as state.
+ *
+ * Note that while some of this class' methods are synchronized, this is just to
+ * match the threadsafety behavior of DFSOutputStream. See HADOOP-11710.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -126,7 +129,7 @@ public class CryptoOutputStream extends FilterOutputStream implements
    * @throws IOException
    */
   @Override
-  public void write(byte[] b, int off, int len) throws IOException {
+  public synchronized void write(byte[] b, int off, int len) throws IOException {
     checkStream();
     if (b == null) {
       throw new NullPointerException();
@@ -213,14 +216,16 @@ public class CryptoOutputStream extends FilterOutputStream implements
   }
   
   @Override
-  public void close() throws IOException {
+  public synchronized void close() throws IOException {
     if (closed) {
       return;
     }
-    
-    super.close();
-    freeBuffers();
-    closed = true;
+    try {
+      super.close();
+      freeBuffers();
+    } finally {
+      closed = true;
+    }
   }
   
   /**
@@ -228,7 +233,7 @@ public class CryptoOutputStream extends FilterOutputStream implements
    * underlying stream, then do the flush.
    */
   @Override
-  public void flush() throws IOException {
+  public synchronized void flush() throws IOException {
     checkStream();
     encrypt();
     super.flush();


[50/50] hadoop git commit: YARN-3039. Implemented the app-level timeline aggregator discovery service. Contributed by Junping Du.

Posted by zj...@apache.org.
YARN-3039. Implemented the app-level timeline aggregator discovery service. Contributed by Junping Du.


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

Branch: refs/heads/YARN-2928
Commit: 8a637914c13baae6749b481551901cfac94694f4
Parents: 5de4026
Author: Zhijie Shen <zj...@apache.org>
Authored: Tue Mar 17 20:23:49 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Tue Mar 17 20:23:49 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../api/protocolrecords/AllocateResponse.java   |  33 ++
 .../hadoop/yarn/conf/YarnConfiguration.java     |  12 +
 .../src/main/proto/yarn_service_protos.proto    |   1 +
 .../distributedshell/ApplicationMaster.java     |  82 ++++-
 .../hadoop/yarn/client/api/AMRMClient.java      |  18 +
 .../yarn/client/api/async/AMRMClientAsync.java  |  17 +
 .../api/async/impl/AMRMClientAsyncImpl.java     |  15 +-
 .../impl/pb/AllocateResponsePBImpl.java         |  17 +
 .../hadoop/yarn/client/api/TimelineClient.java  |   6 +-
 .../client/api/impl/TimelineClientImpl.java     | 133 ++++++-
 .../hadoop/yarn/webapp/util/WebAppUtils.java    |   2 +-
 .../src/main/resources/yarn-default.xml         |  13 +
 .../hadoop/yarn/TestContainerLaunchRPC.java     |  16 +-
 .../java/org/apache/hadoop/yarn/TestRPC.java    | 247 -------------
 .../hadoop/yarn/api/TestAllocateResponse.java   |  17 +
 .../hadoop-yarn-server-common/pom.xml           |   1 +
 .../api/AggregatorNodemanagerProtocol.java      |  56 +++
 .../api/AggregatorNodemanagerProtocolPB.java    |  33 ++
 ...gregatorNodemanagerProtocolPBClientImpl.java |  94 +++++
 ...regatorNodemanagerProtocolPBServiceImpl.java |  61 ++++
 .../protocolrecords/NodeHeartbeatRequest.java   |  23 ++
 .../protocolrecords/NodeHeartbeatResponse.java  |   4 +
 .../ReportNewAggregatorsInfoRequest.java        |  53 +++
 .../ReportNewAggregatorsInfoResponse.java       |  32 ++
 .../impl/pb/NodeHeartbeatRequestPBImpl.java     |  61 ++++
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |  47 +++
 .../ReportNewAggregatorsInfoRequestPBImpl.java  | 142 ++++++++
 .../ReportNewAggregatorsInfoResponsePBImpl.java |  74 ++++
 .../server/api/records/AppAggregatorsMap.java   |  33 ++
 .../impl/pb/AppAggregatorsMapPBImpl.java        | 151 ++++++++
 .../proto/aggregatornodemanager_protocol.proto  |  29 ++
 .../yarn_server_common_service_protos.proto     |  21 ++
 .../java/org/apache/hadoop/yarn/TestRPC.java    | 345 +++++++++++++++++++
 .../hadoop/yarn/TestYarnServerApiClasses.java   |  17 +
 .../hadoop/yarn/server/nodemanager/Context.java |  13 +
 .../yarn/server/nodemanager/NodeManager.java    |  46 ++-
 .../nodemanager/NodeStatusUpdaterImpl.java      |   7 +-
 .../aggregatormanager/NMAggregatorService.java  | 113 ++++++
 .../application/ApplicationImpl.java            |   4 +
 .../ApplicationMasterService.java               |   6 +
 .../resourcemanager/ResourceTrackerService.java |  68 +++-
 .../server/resourcemanager/rmapp/RMApp.java     |  17 +
 .../rmapp/RMAppAggregatorUpdateEvent.java       |  36 ++
 .../resourcemanager/rmapp/RMAppEventType.java   |   3 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |  51 ++-
 .../applicationsmanager/MockAsm.java            |  12 +
 .../server/resourcemanager/rmapp/MockRMApp.java |  15 +
 .../PerNodeTimelineAggregatorsAuxService.java   |   5 +-
 .../TimelineAggregatorsCollection.java          |  78 ++++-
 .../TestTimelineAggregatorsCollection.java      |  11 +-
 51 files changed, 2103 insertions(+), 291 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e62bcf9..47351c6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -29,6 +29,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3264. Created backing storage write interface and a POC only FS based
     storage implementation. (Vrushali C via zjshen)
 
+    YARN-3039. Implemented the app-level timeline aggregator discovery service.
+    (Junping Du via zjshen)
+
   IMPROVEMENTS
 
   OPTIMIZATIONS

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
index c4fdb79..421c2a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
@@ -120,6 +120,25 @@ public abstract class AllocateResponse {
     response.setAMRMToken(amRMToken);
     return response;
   }
+  
+  @Public
+  @Unstable
+  public static AllocateResponse newInstance(int responseId,
+      List<ContainerStatus> completedContainers,
+      List<Container> allocatedContainers, List<NodeReport> updatedNodes,
+      Resource availResources, AMCommand command, int numClusterNodes,
+      PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
+      List<ContainerResourceIncrease> increasedContainers,
+      List<ContainerResourceDecrease> decreasedContainers,
+      String aggregatorAddr) {
+    AllocateResponse response =
+        newInstance(responseId, completedContainers, allocatedContainers,
+          updatedNodes, availResources, command, numClusterNodes, preempt,
+          nmTokens, increasedContainers, decreasedContainers);
+    response.setAMRMToken(amRMToken);
+    response.setAggregatorAddr(aggregatorAddr);
+    return response;
+  }
 
   /**
    * If the <code>ResourceManager</code> needs the
@@ -304,4 +323,18 @@ public abstract class AllocateResponse {
   @Private
   @Unstable
   public abstract void setAMRMToken(Token amRMToken);
+  
+  /**
+   * The address of aggregator that belong to this app
+   *
+   * @return The address of aggregator that belong to this attempt
+   */
+  @Public
+  @Unstable
+  public abstract String getAggregatorAddr();
+  
+  @Private
+  @Unstable
+  public abstract void setAggregatorAddr(String aggregatorAddr);
+  
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/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 4512086..a987044 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
@@ -643,6 +643,11 @@ public class YarnConfiguration extends Configuration {
     NM_PREFIX + "container-manager.thread-count";
   public static final int DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT = 20;
   
+  /** Number of threads container manager uses.*/
+  public static final String NM_AGGREGATOR_SERVICE_THREAD_COUNT =
+    NM_PREFIX + "aggregator-service.thread-count";
+  public static final int DEFAULT_NM_AGGREGATOR_SERVICE_THREAD_COUNT = 5;
+  
   /** Number of threads used in cleanup.*/
   public static final String NM_DELETE_THREAD_COUNT = 
     NM_PREFIX +  "delete.thread-count";
@@ -670,6 +675,13 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_NM_LOCALIZER_ADDRESS = "0.0.0.0:" +
     DEFAULT_NM_LOCALIZER_PORT;
   
+  /** Address where the aggregator service IPC is.*/
+  public static final String NM_AGGREGATOR_SERVICE_ADDRESS =
+    NM_PREFIX + "aggregator-service.address";
+  public static final int DEFAULT_NM_AGGREGATOR_SERVICE_PORT = 8048;
+  public static final String DEFAULT_NM_AGGREGATOR_SERVICE_ADDRESS = 
+      "0.0.0.0:" + DEFAULT_NM_LOCALIZER_PORT;
+  
   /** Interval in between cache cleanups.*/
   public static final String NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS =
     NM_PREFIX + "localizer.cache.cleanup.interval-ms";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 33d1207..4ae4806 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -87,6 +87,7 @@ message AllocateResponseProto {
   repeated ContainerResourceIncreaseProto increased_containers = 10;
   repeated ContainerResourceDecreaseProto decreased_containers = 11;
   optional hadoop.common.TokenProto am_rm_token = 12;
+  optional string aggregator_addr = 13;
 }
 
 enum SchedulerResourceTypes {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index db49166..3a19ac2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -38,6 +38,9 @@ import java.util.Vector;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
@@ -100,6 +103,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.LogManager;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * An ApplicationMaster for executing shell commands on a set of launched
@@ -209,6 +213,13 @@ public class ApplicationMaster {
   private String appMasterTrackingUrl = "";
   
   private boolean newTimelineService = false;
+  
+  // For posting entities in new timeline service in a non-blocking way
+  // TODO replace with event loop in TimelineClient.
+  private static ExecutorService threadPool = 
+      Executors.newCachedThreadPool(
+          new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
+          .build());
 
   // App Master configuration
   // No. of containers to run shell command on
@@ -291,6 +302,19 @@ public class ApplicationMaster {
       }
       appMaster.run();
       result = appMaster.finish();
+      
+      threadPool.shutdown();
+      
+      while (!threadPool.isTerminated()) { // wait for all posting thread to finish
+        try {
+          if (!threadPool.awaitTermination(30, TimeUnit.SECONDS)) {
+            threadPool.shutdownNow(); // send interrupt to hurry them along
+          }
+        } catch (InterruptedException e) {
+          LOG.warn("Timeline client service stop interrupted!");
+          break;
+        }
+      }
     } catch (Throwable t) {
       LOG.fatal("Error running ApplicationMaster", t);
       LogManager.shutdown();
@@ -515,8 +539,12 @@ public class ApplicationMaster {
         }
       }
       // Creating the Timeline Client
-      timelineClient = TimelineClient.createTimelineClient(
-          appAttemptID.getApplicationId());
+      if (newTimelineService) {
+        timelineClient = TimelineClient.createTimelineClient(
+            appAttemptID.getApplicationId());
+      } else {
+        timelineClient = TimelineClient.createTimelineClient();
+      }
       timelineClient.init(conf);
       timelineClient.start();
     } else {
@@ -589,8 +617,10 @@ public class ApplicationMaster {
     AMRMClientAsync.CallbackHandler allocListener = new RMCallbackHandler();
     amRMClient = AMRMClientAsync.createAMRMClientAsync(1000, allocListener);
     amRMClient.init(conf);
+    // need to bind timelineClient before start.
+    amRMClient.registerTimelineClient(timelineClient);
     amRMClient.start();
-
+   
     containerListener = createNMCallbackHandler();
     nmClientAsync = new NMClientAsyncImpl(containerListener);
     nmClientAsync.init(conf);
@@ -727,7 +757,7 @@ public class ApplicationMaster {
     if(timelineClient != null) {
       timelineClient.stop();
     }
-
+    
     return success;
   }
   
@@ -1192,6 +1222,18 @@ public class ApplicationMaster {
   }
   
   private static void publishContainerStartEventOnNewTimelineService(
+      final TimelineClient timelineClient, final Container container, 
+      final String domainId, final UserGroupInformation ugi) {
+    Runnable publishWrapper = new Runnable() {
+      public void run() {
+        publishContainerStartEventOnNewTimelineServiceBase(timelineClient, 
+            container, domainId, ugi);
+      }
+    };
+    threadPool.execute(publishWrapper);
+  }
+
+  private static void publishContainerStartEventOnNewTimelineServiceBase(
       final TimelineClient timelineClient, Container container, String domainId,
       UserGroupInformation ugi) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = 
@@ -1223,10 +1265,22 @@ public class ApplicationMaster {
           e instanceof UndeclaredThrowableException ? e.getCause() : e);
     }
   }
-
+  
   private static void publishContainerEndEventOnNewTimelineService(
-      final TimelineClient timelineClient, ContainerStatus container,
-      String domainId, UserGroupInformation ugi) {
+      final TimelineClient timelineClient, final ContainerStatus container,
+      final String domainId, final UserGroupInformation ugi) {
+    Runnable publishWrapper = new Runnable() {
+      public void run() {
+          publishContainerEndEventOnNewTimelineServiceBase(timelineClient, 
+              container, domainId, ugi);
+      }
+    };
+    threadPool.execute(publishWrapper);
+  }
+  
+  private static void publishContainerEndEventOnNewTimelineServiceBase(
+      final TimelineClient timelineClient, final ContainerStatus container,
+      final String domainId, final UserGroupInformation ugi) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = 
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
     entity.setId(container.getContainerId().toString());
@@ -1257,6 +1311,20 @@ public class ApplicationMaster {
   }
 
   private static void publishApplicationAttemptEventOnNewTimelineService(
+      final TimelineClient timelineClient, final String appAttemptId,
+      final DSEvent appEvent, final String domainId, 
+      final UserGroupInformation ugi) {
+  
+    Runnable publishWrapper = new Runnable() {
+      public void run() {
+        publishApplicationAttemptEventOnNewTimelineServiceBase(timelineClient, 
+            appAttemptId, appEvent, domainId, ugi);
+      }
+    };
+    threadPool.execute(publishWrapper);
+  }
+  
+  private static void publishApplicationAttemptEventOnNewTimelineServiceBase(
       final TimelineClient timelineClient, String appAttemptId,
       DSEvent appEvent, String domainId, UserGroupInformation ugi) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index bfe10d6..56f2b10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -47,6 +47,8 @@ import com.google.common.collect.ImmutableList;
 public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     AbstractService {
   private static final Log LOG = LogFactory.getLog(AMRMClient.class);
+  
+  private TimelineClient timelineClient;
 
   /**
    * Create a new instance of AMRMClient.
@@ -374,6 +376,22 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
   }
 
   /**
+   * Register TimelineClient to AMRMClient.
+   * @param timelineClient
+   */
+  public void registerTimelineClient(TimelineClient timelineClient) {
+    this.timelineClient = timelineClient;
+  }
+  
+  /**
+   * Get registered timeline client.
+   * @return
+   */
+  public TimelineClient getRegisteredTimeineClient() {
+    return this.timelineClient;
+  }
+  
+  /**
    * Wait for <code>check</code> to return true for each 1000 ms.
    * See also {@link #waitFor(com.google.common.base.Supplier, int)}
    * and {@link #waitFor(com.google.common.base.Supplier, int, int)}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
index f62e71b..be5610e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -193,6 +194,22 @@ extends AbstractService {
    * @return Current number of nodes in the cluster
    */
   public abstract int getClusterNodeCount();
+  
+  /**
+   * Register TimelineClient to AMRMClient.
+   * @param timelineClient
+   */
+  public void registerTimelineClient(TimelineClient timelineClient) {
+    client.registerTimelineClient(timelineClient);
+  }
+  
+  /**
+   * Get registered timeline client.
+   * @return
+   */
+  public TimelineClient getRegisteredTimeineClient() {
+    return client.getRegisteredTimeineClient();
+  }
 
   /**
    * Update application's blacklist with addition or removal resources.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
index addc3b6..f0f0bc9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -65,6 +66,8 @@ extends AMRMClientAsync<T> {
   private volatile boolean keepRunning;
   private volatile float progress;
   
+  private volatile String aggregatorAddr;
+  
   private volatile Throwable savedException;
   
   public AMRMClientAsyncImpl(int intervalMs, CallbackHandler callbackHandler) {
@@ -304,7 +307,17 @@ extends AMRMClientAsync<T> {
           if (!allocated.isEmpty()) {
             handler.onContainersAllocated(allocated);
           }
-
+          
+          String aggregatorAddress = response.getAggregatorAddr();
+          TimelineClient timelineClient = client.getRegisteredTimeineClient();
+          if (timelineClient != null && aggregatorAddress != null 
+              && !aggregatorAddress.isEmpty()) {
+            if (aggregatorAddr == null || 
+                !aggregatorAddr.equals(aggregatorAddress)) {
+              aggregatorAddr = aggregatorAddress;
+              timelineClient.setTimelineServiceAddress(aggregatorAddress);
+            }
+          }
           progress = handler.getProgress();
         } catch (Throwable ex) {
           handler.onError(ex);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
index f2796fd..605c29a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
@@ -384,6 +384,23 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     }
     this.amrmToken = amRMToken;
   }
+  
+
+  @Override
+  public String getAggregatorAddr() {
+    AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getAggregatorAddr();
+  }
+  
+  @Override
+  public void setAggregatorAddr(String aggregatorAddr) {
+    maybeInitBuilder();
+    if (aggregatorAddr == null) {
+      builder.clearAggregatorAddr();
+      return;
+    }
+    builder.setAggregatorAddr(aggregatorAddr);
+  }
 
   private synchronized void initLocalIncreasedContainerList() {
     if (this.increasedContainers != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
index d40ad7c..5db347e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 public abstract class TimelineClient extends AbstractService {
 
   protected ApplicationId contextAppId;
-  protected String timelineServiceAddress;
 
   @Public
   public static TimelineClient createTimelineClient() {
@@ -185,7 +184,6 @@ public abstract class TimelineClient extends AbstractService {
    * @param address
    *          the timeline service address
    */
-  public void setTimelineServiceAddress(String address) {
-    timelineServiceAddress = address;
-  }
+  public abstract void setTimelineServiceAddress(String address);
+  
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
index 22dcc00..407682d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
@@ -115,6 +115,15 @@ public class TimelineClientImpl extends TimelineClient {
   private DelegationTokenAuthenticatedURL.Token token;
   private UserGroupInformation authUgi;
   private String doAsUser;
+  
+  private volatile String timelineServiceAddress;
+  
+  // Retry parameters for identifying new timeline service
+  // TODO consider to merge with connection retry
+  private int maxServiceRetries;
+  private long serviceRetryInterval;
+  
+  private boolean newTimelineService = false;
 
   @Private
   @VisibleForTesting
@@ -260,6 +269,7 @@ public class TimelineClientImpl extends TimelineClient {
 
   public TimelineClientImpl(ApplicationId applicationId) {
     super(TimelineClientImpl.class.getName(), applicationId);
+    this.newTimelineService = true;
   }
 
   protected void serviceInit(Configuration conf) throws Exception {
@@ -287,18 +297,32 @@ public class TimelineClientImpl extends TimelineClient {
     client = new Client(new URLConnectionClientHandler(
         new TimelineURLConnectionFactory()), cc);
     TimelineJerseyRetryFilter retryFilter = new TimelineJerseyRetryFilter();
-    client.addFilter(retryFilter);
+    // TODO need to cleanup filter retry later.
+    if (!newTimelineService) {
+      client.addFilter(retryFilter);
+    }
 
-    if (YarnConfiguration.useHttps(conf)) {
-      timelineServiceAddress = conf.get(
-          YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
+    // old version timeline service need to get address from configuration
+    // while new version need to auto discovery (with retry).
+    if (newTimelineService) {
+      maxServiceRetries = conf.getInt(
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
+      serviceRetryInterval = conf.getLong(
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
     } else {
-      timelineServiceAddress = conf.get(
-          YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
-    }
-    LOG.info("Timeline service address: " + timelineServiceAddress);
+      if (YarnConfiguration.useHttps(conf)) {
+        setTimelineServiceAddress(conf.get(
+            YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS));
+      } else {
+        setTimelineServiceAddress(conf.get(
+            YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS));
+      }
+      LOG.info("Timeline service address: " + getTimelineServiceAddress());
+    } 
     super.serviceInit(conf);
   }
 
@@ -341,8 +365,7 @@ public class TimelineClientImpl extends TimelineClient {
     if (async) {
       params.add("async", Boolean.TRUE.toString());
     }
-    putObjects(constructResURI(getConfig(), timelineServiceAddress, true),
-        "entities", params, entitiesContainer);
+    putObjects("entities", params, entitiesContainer);
   }
 
   @Override
@@ -350,6 +373,60 @@ public class TimelineClientImpl extends TimelineClient {
       YarnException {
     doPosting(domain, "domain");
   }
+  
+  // Used for new timeline service only
+  @Private
+  public void putObjects(String path, MultivaluedMap<String, String> params, 
+      Object obj) throws IOException, YarnException {
+    
+    // timelineServiceAddress could haven't be initialized yet 
+    // or stale (only for new timeline service)
+    int retries = pollTimelineServiceAddress(this.maxServiceRetries);
+    
+    // timelineServiceAddress could be stale, add retry logic here.
+    boolean needRetry = true;
+    while (needRetry) {
+      try {
+        URI uri = constructResURI(getConfig(), timelineServiceAddress, true);
+        putObjects(uri, path, params, obj);
+        needRetry = false;
+      }
+      catch (Exception e) {
+        // TODO only handle exception for timelineServiceAddress being updated.
+        // skip retry for other exceptions.
+        checkRetryWithSleep(retries, e);
+        retries--;
+      }
+    }
+  }
+  
+  /**
+   * Check if reaching to maximum of retries.
+   * @param retries
+   * @param e
+   */
+  private void checkRetryWithSleep(int retries, Exception e) throws 
+      YarnException, IOException {
+    if (retries > 0) {
+      try {
+        Thread.sleep(this.serviceRetryInterval);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+      }
+    } else {
+      LOG.error(
+        "TimelineClient has reached to max retry times :" + 
+        this.maxServiceRetries + " for service address: " + 
+        timelineServiceAddress);
+      if (e instanceof YarnException) {
+        throw (YarnException)e;
+      } else if (e instanceof IOException) {
+        throw (IOException)e;
+      } else {
+        throw new YarnException(e);
+      }
+    }
+  }
 
   private void putObjects(
       URI base, String path, MultivaluedMap<String, String> params, Object obj)
@@ -410,6 +487,15 @@ public class TimelineClientImpl extends TimelineClient {
     }
     return resp;
   }
+  
+  @Override
+  public void setTimelineServiceAddress(String address) {
+    this.timelineServiceAddress = address;
+  }
+  
+  private String getTimelineServiceAddress() {
+    return this.timelineServiceAddress;
+  }
 
   @SuppressWarnings("unchecked")
   @Override
@@ -424,8 +510,10 @@ public class TimelineClientImpl extends TimelineClient {
             DelegationTokenAuthenticatedURL authUrl =
                 new DelegationTokenAuthenticatedURL(authenticator,
                     connConfigurator);
+            // TODO we should add retry logic here if timelineServiceAddress is
+            // not available immediately.
             return (Token) authUrl.getDelegationToken(
-                constructResURI(getConfig(), timelineServiceAddress, false).toURL(),
+                constructResURI(getConfig(), getTimelineServiceAddress(), false).toURL(),
                 token, renewer, doAsUser);
           }
         };
@@ -523,6 +611,7 @@ public class TimelineClientImpl extends TimelineClient {
     return connectionRetry.retryOn(tokenRetryOp);
   }
 
+  // Old timeline service, no external retry logic.
   @Private
   @VisibleForTesting
   public ClientResponse doPostingObject(Object object, String path) {
@@ -540,6 +629,24 @@ public class TimelineClientImpl extends TimelineClient {
       throw new YarnRuntimeException("Unknown resource type");
     }
   }
+  
+  /**
+   * Poll TimelineServiceAddress for maximum of retries times if it is null
+   * @param retries
+   * @return the left retry times
+   */
+  private int pollTimelineServiceAddress(int retries) {
+    while (timelineServiceAddress == null && retries > 0) {
+      try {
+        Thread.sleep(this.serviceRetryInterval);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+      timelineServiceAddress = getTimelineServiceAddress();
+      retries--;
+    }
+    return retries;
+  }
 
   private class TimelineURLConnectionFactory
       implements HttpURLConnectionFactory {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/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 3aeb33e..e9a0a88 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
@@ -172,7 +172,7 @@ public class WebAppUtils {
     return getResolvedAddress(address);
   }
 
-  private static String getResolvedAddress(InetSocketAddress address) {
+  public static String getResolvedAddress(InetSocketAddress address) {
     address = NetUtils.getConnectAddress(address);
     StringBuilder sb = new StringBuilder();
     InetAddress resolved = address.getAddress();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/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 66400c8..226d8ce 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
@@ -714,6 +714,12 @@
     <name>yarn.nodemanager.container-manager.thread-count</name>
     <value>20</value>
   </property>
+  
+    <property>
+    <description>Number of threads aggregator service uses.</description>
+    <name>yarn.nodemanager.aggregator-service.thread-count</name>
+    <value>5</value>
+  </property>
 
   <property>
     <description>Number of threads used in cleanup.</description>
@@ -782,6 +788,13 @@
     <name>yarn.nodemanager.localizer.address</name>
     <value>${yarn.nodemanager.hostname}:8040</value>
   </property>
+  
+  
+  <property>
+    <description>Address where the aggregator service IPC is.</description>
+    <name>yarn.nodemanager.aggregator-service.address</name>
+    <value>${yarn.nodemanager.hostname}:8048</value>
+  </property>
 
   <property>
     <description>Interval in between cache cleanups.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/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 e2071dd..26d6d04 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
@@ -29,6 +29,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
@@ -105,7 +106,7 @@ public class TestContainerLaunchRPC {
             resource, System.currentTimeMillis() + 10000, 42, 42,
             Priority.newInstance(0), 0);
       Token containerToken =
-          TestRPC.newContainerToken(nodeId, "password".getBytes(),
+          newContainerToken(nodeId, "password".getBytes(),
             containerTokenIdentifier);
 
       StartContainerRequest scRequest =
@@ -130,6 +131,19 @@ public class TestContainerLaunchRPC {
 
     Assert.fail("timeout exception should have occurred!");
   }
+  
+  public static Token newContainerToken(NodeId nodeId, byte[] password,
+      ContainerTokenIdentifier tokenIdentifier) {
+    // RPC layer client expects ip:port as service for tokens
+    InetSocketAddress addr =
+        NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort());
+    // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token
+    Token containerToken =
+        Token.newInstance(tokenIdentifier.getBytes(),
+          ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil
+            .buildTokenService(addr).toString());
+    return containerToken;
+  }
 
   public class DummyContainerManager implements ContainerManagementProtocol {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/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
deleted file mode 100644
index 39e6162..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
+++ /dev/null
@@ -1,247 +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;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
-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.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.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.ContainerLaunchContext;
-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.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.exceptions.YarnException;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
-import org.apache.hadoop.yarn.ipc.RPCUtil;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.util.Records;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestRPC {
-
-  private static final String EXCEPTION_MSG = "test error";
-  private static final String EXCEPTION_CAUSE = "exception cause";
-  private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-  
-  @Test
-  public void testUnknownCall() {
-    Configuration conf = new Configuration();
-    conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class
-        .getName());
-    YarnRPC rpc = YarnRPC.create(conf);
-    String bindAddr = "localhost:0";
-    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
-    Server server = rpc.getServer(ContainerManagementProtocol.class,
-        new DummyContainerManager(), addr, conf, null, 1);
-    server.start();
-
-    // Any unrelated protocol would do
-    ApplicationClientProtocol proxy = (ApplicationClientProtocol) rpc.getProxy(
-        ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), conf);
-
-    try {
-      proxy.getNewApplication(Records
-          .newRecord(GetNewApplicationRequest.class));
-      Assert.fail("Excepted RPC call to fail with unknown method.");
-    } catch (YarnException e) {
-      Assert.assertTrue(e.getMessage().matches(
-          "Unknown method getNewApplication called on.*"
-              + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol"
-              + "\\$ApplicationClientProtocolService\\$BlockingInterface protocol."));
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void testHadoopProtoRPC() throws Exception {
-    test(HadoopYarnProtoRPC.class.getName());
-  }
-  
-  private void test(String rpcClass) throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass);
-    YarnRPC rpc = YarnRPC.create(conf);
-    String bindAddr = "localhost:0";
-    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
-    Server server = rpc.getServer(ContainerManagementProtocol.class, 
-            new DummyContainerManager(), addr, conf, null, 1);
-    server.start();
-    RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, ProtobufRpcEngine.class);
-    ContainerManagementProtocol proxy = (ContainerManagementProtocol) 
-        rpc.getProxy(ContainerManagementProtocol.class, 
-            NetUtils.getConnectAddress(server), conf);
-    ContainerLaunchContext containerLaunchContext = 
-        recordFactory.newRecordInstance(ContainerLaunchContext.class);
-
-    ApplicationId applicationId = ApplicationId.newInstance(0, 0);
-    ApplicationAttemptId applicationAttemptId =
-        ApplicationAttemptId.newInstance(applicationId, 0);
-    ContainerId containerId =
-        ContainerId.newContainerId(applicationAttemptId, 100);
-    NodeId nodeId = NodeId.newInstance("localhost", 1234);
-    Resource resource = Resource.newInstance(1234, 2);
-    ContainerTokenIdentifier containerTokenIdentifier =
-        new ContainerTokenIdentifier(containerId, "localhost", "user",
-          resource, System.currentTimeMillis() + 10000, 42, 42,
-          Priority.newInstance(0), 0);
-    Token containerToken = newContainerToken(nodeId, "password".getBytes(),
-          containerTokenIdentifier);
-
-    StartContainerRequest scRequest =
-        StartContainerRequest.newInstance(containerLaunchContext,
-          containerToken);
-    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
-    list.add(scRequest);
-    StartContainersRequest allRequests =
-        StartContainersRequest.newInstance(list);
-    proxy.startContainers(allRequests);
-
-    List<ContainerId> containerIds = new ArrayList<ContainerId>();
-    containerIds.add(containerId);
-    GetContainerStatusesRequest gcsRequest =
-        GetContainerStatusesRequest.newInstance(containerIds);
-    GetContainerStatusesResponse response =
-        proxy.getContainerStatuses(gcsRequest);
-    List<ContainerStatus> statuses = response.getContainerStatuses();
-
-    //test remote exception
-    boolean exception = false;
-    try {
-      StopContainersRequest stopRequest =
-          recordFactory.newRecordInstance(StopContainersRequest.class);
-      stopRequest.setContainerIds(containerIds);
-      proxy.stopContainers(stopRequest);
-      } catch (YarnException e) {
-      exception = true;
-      Assert.assertTrue(e.getMessage().contains(EXCEPTION_MSG));
-      Assert.assertTrue(e.getMessage().contains(EXCEPTION_CAUSE));
-      System.out.println("Test Exception is " + e.getMessage());
-    } catch (Exception ex) {
-      ex.printStackTrace();
-    }
-    Assert.assertTrue(exception);
-    
-    server.stop();
-    Assert.assertNotNull(statuses.get(0));
-    Assert.assertEquals(ContainerState.RUNNING, statuses.get(0).getState());
-  }
-
-  public class DummyContainerManager implements ContainerManagementProtocol {
-
-    private List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
-
-    @Override
-    public GetContainerStatusesResponse getContainerStatuses(
-        GetContainerStatusesRequest request)
-    throws YarnException {
-      GetContainerStatusesResponse response = 
-          recordFactory.newRecordInstance(GetContainerStatusesResponse.class);
-      response.setContainerStatuses(statuses);
-      return response;
-    }
-
-    @Override
-    public StartContainersResponse startContainers(
-        StartContainersRequest requests) throws YarnException {
-      StartContainersResponse response =
-          recordFactory.newRecordInstance(StartContainersResponse.class);
-      for (StartContainerRequest request : requests.getStartContainerRequests()) {
-        Token containerToken = request.getContainerToken();
-        ContainerTokenIdentifier tokenId = null;
-
-        try {
-          tokenId = newContainerTokenIdentifier(containerToken);
-        } catch (IOException e) {
-          throw RPCUtil.getRemoteException(e);
-        }
-        ContainerStatus status =
-            recordFactory.newRecordInstance(ContainerStatus.class);
-        status.setState(ContainerState.RUNNING);
-        status.setContainerId(tokenId.getContainerID());
-        status.setExitStatus(0);
-        statuses.add(status);
-
-      }
-      return response;
-    }
-
-    @Override
-    public StopContainersResponse stopContainers(StopContainersRequest request) 
-    throws YarnException {
-      Exception e = new Exception(EXCEPTION_MSG, 
-          new Exception(EXCEPTION_CAUSE));
-      throw new YarnException(e);
-    }
-  }
-
-  public static ContainerTokenIdentifier newContainerTokenIdentifier(
-      Token containerToken) throws IOException {
-    org.apache.hadoop.security.token.Token<ContainerTokenIdentifier> token =
-        new org.apache.hadoop.security.token.Token<ContainerTokenIdentifier>(
-            containerToken.getIdentifier()
-                .array(), containerToken.getPassword().array(), new Text(
-                containerToken.getKind()),
-            new Text(containerToken.getService()));
-    return token.decodeIdentifier();
-  }
-
-  public static Token newContainerToken(NodeId nodeId, byte[] password,
-      ContainerTokenIdentifier tokenIdentifier) {
-    // RPC layer client expects ip:port as service for tokens
-    InetSocketAddress addr =
-        NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort());
-    // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token
-    Token containerToken =
-        Token.newInstance(tokenIdentifier.getBytes(),
-          ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil
-            .buildTokenService(addr).toString());
-    return containerToken;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
index fbe9af9..ef0bdcc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
@@ -111,4 +111,21 @@ public class TestAllocateResponse {
     Assert.assertEquals(0, r.getIncreasedContainers().size());
     Assert.assertEquals(0, r.getDecreasedContainers().size());
   }
+  
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testAllocateResponseWithAggregatorAddress() {
+    final String aggregatorAddr = "localhost:0";
+    AllocateResponse r =
+        AllocateResponse.newInstance(3, new ArrayList<ContainerStatus>(),
+            new ArrayList<Container>(), new ArrayList<NodeReport>(), null,
+            AMCommand.AM_RESYNC, 3, null, new ArrayList<NMToken>(), null, 
+            null, null, aggregatorAddr);
+
+    AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto();
+    r = new AllocateResponsePBImpl(p);
+
+    // check value
+    Assert.assertEquals(aggregatorAddr, r.getAggregatorAddr());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index 1a4dab8..d1e4acb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -142,6 +142,7 @@
                   <include>yarn_server_common_service_protos.proto</include>
                   <include>ResourceTracker.proto</include>
                   <include>SCMUploader.proto</include>
+                  <include>aggregatornodemanager_protocol.proto</include>
                 </includes>
               </source>
               <output>${project.build.directory}/generated-sources/java</output>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/AggregatorNodemanagerProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/AggregatorNodemanagerProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/AggregatorNodemanagerProtocol.java
new file mode 100644
index 0000000..53bdb4e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/AggregatorNodemanagerProtocol.java
@@ -0,0 +1,56 @@
+/**
+ * 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.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
+
+/**
+ * <p>The protocol between an <code>TimelineAggregatorsCollection</code> and a 
+ * <code>NodeManager</code> to report a new application aggregator get launched.
+ * </p>
+ * 
+ */
+@Private
+public interface AggregatorNodemanagerProtocol {
+
+  /**
+   * 
+   * <p>
+   * The <code>TimelineAggregatorsCollection</code> provides a list of mapping
+   * between application and aggregator's address in 
+   * {@link ReportNewAggregatorsInfoRequest} to a <code>NodeManager</code> to
+   * <em>register</em> aggregator's info, include: applicationId and REST URI to 
+   * access aggregator. NodeManager will add them into registered aggregators 
+   * and register them into <code>ResourceManager</code> afterwards.
+   * </p>
+   * 
+   * @param request the request of registering a new aggregator or a list of aggregators
+   * @return 
+   * @throws YarnException
+   * @throws IOException
+   */
+  ReportNewAggregatorsInfoResponse reportNewAggregatorInfo(
+      ReportNewAggregatorsInfoRequest request)
+      throws YarnException, IOException;
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/AggregatorNodemanagerProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/AggregatorNodemanagerProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/AggregatorNodemanagerProtocolPB.java
new file mode 100644
index 0000000..4df80a5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/AggregatorNodemanagerProtocolPB.java
@@ -0,0 +1,33 @@
+/**
+ * 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.api;
+
+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.AggregatorNodemanagerProtocol.AggregatorNodemanagerProtocolService;
+
+@Private
+@Unstable
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocolPB",
+    protocolVersion = 1)
+public interface AggregatorNodemanagerProtocolPB extends 
+    AggregatorNodemanagerProtocolService.BlockingInterface {
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/AggregatorNodemanagerProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/AggregatorNodemanagerProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/AggregatorNodemanagerProtocolPBClientImpl.java
new file mode 100644
index 0000000..6e777e7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/AggregatorNodemanagerProtocolPBClientImpl.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.yarn.server.api.impl.pb.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+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.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoRequestProto;
+import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
+import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocolPB;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewAggregatorsInfoRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewAggregatorsInfoResponsePBImpl;
+
+import com.google.protobuf.ServiceException;
+
+public class AggregatorNodemanagerProtocolPBClientImpl implements
+    AggregatorNodemanagerProtocol, Closeable {
+
+  // Not a documented config. Only used for tests internally
+  static final String NM_COMMAND_TIMEOUT = YarnConfiguration.YARN_PREFIX
+      + "rpc.nm-command-timeout";
+
+  /**
+   * Maximum of 1 minute timeout for a Node to react to the command
+   */
+  static final int DEFAULT_COMMAND_TIMEOUT = 60000;
+  
+  private AggregatorNodemanagerProtocolPB proxy;
+  
+  @Private
+  public AggregatorNodemanagerProtocolPBClientImpl(long clientVersion,
+      InetSocketAddress addr, Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, AggregatorNodemanagerProtocolPB.class,
+      ProtobufRpcEngine.class);
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
+    int expireIntvl = conf.getInt(NM_COMMAND_TIMEOUT, DEFAULT_COMMAND_TIMEOUT);
+    proxy =
+        (AggregatorNodemanagerProtocolPB) RPC.getProxy(
+            AggregatorNodemanagerProtocolPB.class,
+            clientVersion, addr, ugi, conf,
+            NetUtils.getDefaultSocketFactory(conf), expireIntvl);
+  }
+  
+  @Override
+  public ReportNewAggregatorsInfoResponse reportNewAggregatorInfo(
+      ReportNewAggregatorsInfoRequest request) throws YarnException, IOException {
+  
+    ReportNewAggregatorsInfoRequestProto requestProto =
+        ((ReportNewAggregatorsInfoRequestPBImpl) request).getProto();
+    try {
+      return new ReportNewAggregatorsInfoResponsePBImpl(
+          proxy.reportNewAggregatorInfo(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+  
+  @Override
+  public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/AggregatorNodemanagerProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/AggregatorNodemanagerProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/AggregatorNodemanagerProtocolPBServiceImpl.java
new file mode 100644
index 0000000..87bce16
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/AggregatorNodemanagerProtocolPBServiceImpl.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.yarn.server.api.impl.pb.service;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoResponseProto;
+import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
+import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocolPB;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewAggregatorsInfoRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReportNewAggregatorsInfoResponsePBImpl;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class AggregatorNodemanagerProtocolPBServiceImpl implements
+    AggregatorNodemanagerProtocolPB {
+
+  private AggregatorNodemanagerProtocol real;
+  
+  public AggregatorNodemanagerProtocolPBServiceImpl(AggregatorNodemanagerProtocol impl) {
+    this.real = impl;
+  }
+
+  @Override
+  public ReportNewAggregatorsInfoResponseProto reportNewAggregatorInfo(
+      RpcController arg0, ReportNewAggregatorsInfoRequestProto proto) 
+      throws ServiceException {
+    ReportNewAggregatorsInfoRequestPBImpl request = 
+        new ReportNewAggregatorsInfoRequestPBImpl(proto);
+    try {
+      ReportNewAggregatorsInfoResponse response = real.reportNewAggregatorInfo(request);
+      return ((ReportNewAggregatorsInfoResponsePBImpl)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/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
index addd3fe..0b020b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.yarn.server.api.protocolrecords;
 
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.util.Records;
@@ -36,6 +40,21 @@ public abstract class NodeHeartbeatRequest {
         .setLastKnownNMTokenMasterKey(lastKnownNMTokenMasterKey);
     return nodeHeartbeatRequest;
   }
+  
+  public static NodeHeartbeatRequest newInstance(NodeStatus nodeStatus,
+      MasterKey lastKnownContainerTokenMasterKey,
+      MasterKey lastKnownNMTokenMasterKey,
+      Map<ApplicationId, String> registeredAggregators) {
+    NodeHeartbeatRequest nodeHeartbeatRequest =
+        Records.newRecord(NodeHeartbeatRequest.class);
+    nodeHeartbeatRequest.setNodeStatus(nodeStatus);
+    nodeHeartbeatRequest
+        .setLastKnownContainerTokenMasterKey(lastKnownContainerTokenMasterKey);
+    nodeHeartbeatRequest
+        .setLastKnownNMTokenMasterKey(lastKnownNMTokenMasterKey);
+    nodeHeartbeatRequest.setRegisteredAggregators(registeredAggregators);
+    return nodeHeartbeatRequest;
+  }
 
   public abstract NodeStatus getNodeStatus();
   public abstract void setNodeStatus(NodeStatus status);
@@ -45,4 +64,8 @@ public abstract class NodeHeartbeatRequest {
   
   public abstract MasterKey getLastKnownNMTokenMasterKey();
   public abstract void setLastKnownNMTokenMasterKey(MasterKey secretKey);
+  
+  // This tells RM registered aggregators' address info on this node
+  public abstract Map<ApplicationId, String> getRegisteredAggregators();
+  public abstract void setRegisteredAggregators(Map<ApplicationId, String> appAggregatorsMap);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
index 9fb44ca..262ca07 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
@@ -35,6 +35,10 @@ public interface NodeHeartbeatResponse {
   List<ContainerId> getContainersToBeRemovedFromNM();
 
   List<ApplicationId> getApplicationsToCleanup();
+  
+  // This tells NM the aggregators' address info of related Apps
+  Map<ApplicationId, String> getAppAggregatorsMap();
+  void setAppAggregatorsMap(Map<ApplicationId, String> appAggregatorsMap);
 
   void setResponseId(int responseId);
   void setNodeAction(NodeAction action);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewAggregatorsInfoRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewAggregatorsInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewAggregatorsInfoRequest.java
new file mode 100644
index 0000000..ae538a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewAggregatorsInfoRequest.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.yarn.server.api.protocolrecords;
+
+import java.util.List;
+import java.util.Arrays;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
+import org.apache.hadoop.yarn.util.Records;
+
+@Private
+public abstract class ReportNewAggregatorsInfoRequest {
+  
+  public static ReportNewAggregatorsInfoRequest newInstance(
+      List<AppAggregatorsMap> appAggregatorsList) {
+    ReportNewAggregatorsInfoRequest request =
+        Records.newRecord(ReportNewAggregatorsInfoRequest.class);
+    request.setAppAggregatorsList(appAggregatorsList);
+    return request;
+  }
+  
+  public static ReportNewAggregatorsInfoRequest newInstance(
+      ApplicationId id, String aggregatorAddr) {
+    ReportNewAggregatorsInfoRequest request =
+        Records.newRecord(ReportNewAggregatorsInfoRequest.class);
+    request.setAppAggregatorsList(
+        Arrays.asList(AppAggregatorsMap.newInstance(id, aggregatorAddr)));
+    return request;
+  }
+  
+  public abstract List<AppAggregatorsMap> getAppAggregatorsList();
+  
+  public abstract void setAppAggregatorsList(
+      List<AppAggregatorsMap> appAggregatorsList);
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewAggregatorsInfoResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewAggregatorsInfoResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewAggregatorsInfoResponse.java
new file mode 100644
index 0000000..3b847d6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReportNewAggregatorsInfoResponse.java
@@ -0,0 +1,32 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.util.Records;
+
+public abstract class ReportNewAggregatorsInfoResponse {
+
+  @Private
+  public static ReportNewAggregatorsInfoResponse newInstance() {
+    ReportNewAggregatorsInfoResponse response =
+        Records.newRecord(ReportNewAggregatorsInfoResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
index 26d1f19..39eaabd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
@@ -18,8 +18,16 @@
 
 package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
@@ -36,6 +44,7 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private NodeStatus nodeStatus = null;
   private MasterKey lastKnownContainerTokenMasterKey = null;
   private MasterKey lastKnownNMTokenMasterKey = null;
+  Map<ApplicationId, String> registeredAggregators = null;
   
   public NodeHeartbeatRequestPBImpl() {
     builder = NodeHeartbeatRequestProto.newBuilder();
@@ -80,6 +89,20 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
       builder.setLastKnownNmTokenMasterKey(
           convertToProtoFormat(this.lastKnownNMTokenMasterKey));
     }
+    
+    if (this.registeredAggregators != null) {
+      addRegisteredAggregatorsToProto();
+    }
+  }
+  
+  private void addRegisteredAggregatorsToProto() {
+    maybeInitBuilder();
+    builder.clearRegisteredAggregators();
+    for (Map.Entry<ApplicationId, String> entry : registeredAggregators.entrySet()) {
+      builder.addRegisteredAggregators(AppAggregatorsMapProto.newBuilder()
+        .setAppId(convertToProtoFormat(entry.getKey()))
+        .setAppAggregatorAddr(entry.getValue()));
+    }
   }
 
   private void mergeLocalToProto() {
@@ -162,6 +185,36 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
       builder.clearLastKnownNmTokenMasterKey();
     this.lastKnownNMTokenMasterKey = masterKey;
   }
+  
+  @Override
+  public Map<ApplicationId, String> getRegisteredAggregators() {
+    if (this.registeredAggregators != null) {
+      return this.registeredAggregators;
+    }
+    initRegisteredAggregators();
+    return registeredAggregators;
+  }
+  
+  private void initRegisteredAggregators() {
+    NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<AppAggregatorsMapProto> list = p.getRegisteredAggregatorsList();
+    this.registeredAggregators = new HashMap<ApplicationId, String> ();
+    for (AppAggregatorsMapProto c : list) {
+      ApplicationId appId = convertFromProtoFormat(c.getAppId());
+      this.registeredAggregators.put(appId, c.getAppAggregatorAddr());
+    }
+  }
+  
+  @Override
+  public void setRegisteredAggregators(
+      Map<ApplicationId, String> registeredAggregators) {
+    if (registeredAggregators == null || registeredAggregators.isEmpty()) {
+      return;
+    }
+    maybeInitBuilder();
+    this.registeredAggregators = new HashMap<ApplicationId, String>();
+    this.registeredAggregators.putAll(registeredAggregators);
+  }
 
   private NodeStatusPBImpl convertFromProtoFormat(NodeStatusProto p) {
     return new NodeStatusPBImpl(p);
@@ -170,6 +223,14 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private NodeStatusProto convertToProtoFormat(NodeStatus t) {
     return ((NodeStatusPBImpl)t).getProto();
   }
+  
+  private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
+    return new ApplicationIdPBImpl(p);
+  }
+  
+  private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
+    return ((ApplicationIdPBImpl) t).getProto();
+  }
 
   private MasterKeyPBImpl convertFromProtoFormat(MasterKeyProto p) {
     return new MasterKeyPBImpl(p);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
index 630a5bf..019b2ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SystemCredentialsForAppsProto;
@@ -55,6 +56,8 @@ public class NodeHeartbeatResponsePBImpl extends
   private List<ContainerId> containersToBeRemovedFromNM = null;
   private List<ApplicationId> applicationsToCleanup = null;
   private Map<ApplicationId, ByteBuffer> systemCredentials = null;
+  
+  Map<ApplicationId, String> appAggregatorsMap = null;
 
   private MasterKey containerTokenMasterKey = null;
   private MasterKey nmTokenMasterKey = null;
@@ -96,6 +99,10 @@ public class NodeHeartbeatResponsePBImpl extends
     if (this.systemCredentials != null) {
       addSystemCredentialsToProto();
     }
+    
+    if (this.appAggregatorsMap != null) {
+      addAppAggregatorsMapToProto();
+    }
   }
 
   private void addSystemCredentialsToProto() {
@@ -108,6 +115,16 @@ public class NodeHeartbeatResponsePBImpl extends
             entry.getValue().duplicate())));
     }
   }
+  
+  private void addAppAggregatorsMapToProto() {
+    maybeInitBuilder();
+    builder.clearAppAggregatorsMap();
+    for (Map.Entry<ApplicationId, String> entry : appAggregatorsMap.entrySet()) {
+      builder.addAppAggregatorsMap(AppAggregatorsMapProto.newBuilder()
+        .setAppId(convertToProtoFormat(entry.getKey()))
+        .setAppAggregatorAddr(entry.getValue()));
+    }
+  }
 
   private void mergeLocalToProto() {
     if (viaProto) 
@@ -417,6 +434,15 @@ public class NodeHeartbeatResponsePBImpl extends
     initSystemCredentials();
     return systemCredentials;
   }
+  
+  @Override
+  public Map<ApplicationId, String> getAppAggregatorsMap() {
+    if (this.appAggregatorsMap != null) {
+      return this.appAggregatorsMap;
+    }
+    initAppAggregatorsMap();
+    return appAggregatorsMap;
+  }
 
   private void initSystemCredentials() {
     NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
@@ -428,6 +454,16 @@ public class NodeHeartbeatResponsePBImpl extends
       this.systemCredentials.put(appId, byteBuffer);
     }
   }
+  
+  private void initAppAggregatorsMap() {
+    NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<AppAggregatorsMapProto> list = p.getAppAggregatorsMapList();
+    this.appAggregatorsMap = new HashMap<ApplicationId, String> ();
+    for (AppAggregatorsMapProto c : list) {
+      ApplicationId appId = convertFromProtoFormat(c.getAppId());
+      this.appAggregatorsMap.put(appId, c.getAppAggregatorAddr());
+    }
+  }
 
   @Override
   public void setSystemCredentialsForApps(
@@ -439,6 +475,17 @@ public class NodeHeartbeatResponsePBImpl extends
     this.systemCredentials = new HashMap<ApplicationId, ByteBuffer>();
     this.systemCredentials.putAll(systemCredentials);
   }
+  
+  @Override
+  public void setAppAggregatorsMap(
+      Map<ApplicationId, String> appAggregatorsMap) {
+    if (appAggregatorsMap == null || appAggregatorsMap.isEmpty()) {
+      return;
+    }
+    maybeInitBuilder();
+    this.appAggregatorsMap = new HashMap<ApplicationId, String>();
+    this.appAggregatorsMap.putAll(appAggregatorsMap);
+  }
 
   @Override
   public long getNextHeartBeatInterval() {


[33/50] hadoop git commit: MAPREDUCE-6100. replace mapreduce.job.credentials.binary with MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY for better readability. Contributed by Zhihai Xu.

Posted by zj...@apache.org.
MAPREDUCE-6100. replace mapreduce.job.credentials.binary with MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY for better readability. Contributed by Zhihai Xu.


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

Branch: refs/heads/YARN-2928
Commit: f222bde273cc10a38945dc31e85206a0c4f06a12
Parents: 046521c
Author: Harsh J <ha...@cloudera.com>
Authored: Tue Mar 17 11:06:35 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Tue Mar 17 11:06:35 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                             | 4 ++++
 .../src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java  | 2 +-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f222bde2/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 52880f6..ee21b70 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -253,6 +253,10 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-6100. replace "mapreduce.job.credentials.binary" with
+    MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY for better readability.
+    (Zhihai Xu via harsh)
+
     MAPREDUCE-6105. Inconsistent configuration in property
     mapreduce.reduce.shuffle.merge.percent. (Ray Chiang via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f222bde2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
index 30a87c7..023bd63 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
@@ -383,7 +383,7 @@ class JobSubmitter {
   throws IOException {
     // add tokens and secrets coming from a token storage file
     String binaryTokenFilename =
-      conf.get("mapreduce.job.credentials.binary");
+      conf.get(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY);
     if (binaryTokenFilename != null) {
       Credentials binary = Credentials.readTokenStorageFile(
           FileSystem.getLocal(conf).makeQualified(


[30/50] hadoop git commit: YARN-3339. TestDockerContainerExecutor should pull a single image and not the entire centos repository. (Ravindra Kumar Naik via raviprak)

Posted by zj...@apache.org.
YARN-3339. TestDockerContainerExecutor should pull a single image and not the entire centos repository. (Ravindra Kumar Naik via raviprak)


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

Branch: refs/heads/YARN-2928
Commit: 56085203c43b8f2561bf3745910e03f8ac176a67
Parents: 7522a64
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Mon Mar 16 16:17:58 2015 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Mon Mar 16 16:17:58 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../yarn/server/nodemanager/TestDockerContainerExecutor.java      | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/56085203/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b8e07a0..cb68480 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -58,6 +58,9 @@ Release 2.8.0 - UNRELEASED
 
   OPTIMIZATIONS
 
+    YARN-3339. TestDockerContainerExecutor should pull a single image and not
+    the entire centos repository. (Ravindra Kumar Naik via raviprak)
+
   BUG FIXES
 
 Release 2.7.0 - UNRELEASED

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56085203/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutor.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/TestDockerContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutor.java
index e43ac2e..ac02542 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutor.java
@@ -78,7 +78,7 @@ public class TestDockerContainerExecutor {
   private int id = 0;
   private String appSubmitter;
   private String dockerUrl;
-  private String testImage = "centos";
+  private String testImage = "centos:latest";
   private String dockerExec;
   private String containerIdStr;
 


[23/50] hadoop git commit: HADOOP-11714. Add more trace log4j messages to SpanReceiverHost (cmccabe)

Posted by zj...@apache.org.
HADOOP-11714. Add more trace log4j messages to SpanReceiverHost (cmccabe)


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

Branch: refs/heads/YARN-2928
Commit: bf3275dbaa99105d49520e25f5a6eadd6fd5b7ed
Parents: ed4e72a
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Mar 16 12:02:10 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Mar 16 12:02:10 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt        |  2 ++
 .../org/apache/hadoop/tracing/SpanReceiverHost.java    | 13 ++++++++++---
 2 files changed, 12 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf3275db/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index a43a153..aa17841 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -692,6 +692,8 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11642. Upgrade azure sdk version from 0.6.0 to 2.0.0.
     (Shashank Khandelwal and Ivan Mitic via cnauroth)
 
+    HADOOP-11714. Add more trace log4j messages to SpanReceiverHost (cmccabe)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf3275db/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
index 01ba76d..f2de0a0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
@@ -134,20 +134,27 @@ public class SpanReceiverHost implements TraceAdminProtocol {
     String[] receiverNames =
         config.getTrimmedStrings(SPAN_RECEIVERS_CONF_KEY);
     if (receiverNames == null || receiverNames.length == 0) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("No span receiver names found in " +
+                  SPAN_RECEIVERS_CONF_KEY + ".");
+      }
       return;
     }
     // It's convenient to have each daemon log to a random trace file when
     // testing.
     if (config.get(LOCAL_FILE_SPAN_RECEIVER_PATH) == null) {
-      config.set(LOCAL_FILE_SPAN_RECEIVER_PATH,
-          getUniqueLocalTraceFileName());
+      String uniqueFile = getUniqueLocalTraceFileName();
+      config.set(LOCAL_FILE_SPAN_RECEIVER_PATH, uniqueFile);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Set " + LOCAL_FILE_SPAN_RECEIVER_PATH + " to " +  uniqueFile);
+      }
     }
     for (String className : receiverNames) {
       try {
         SpanReceiver rcvr = loadInstance(className, EMPTY);
         Trace.addReceiver(rcvr);
         receivers.put(highestId++, rcvr);
-        LOG.info("SpanReceiver " + className + " was loaded successfully.");
+        LOG.info("Loaded SpanReceiver " + className + " successfully.");
       } catch (IOException e) {
         LOG.error("Failed to load SpanReceiver", e);
       }


[27/50] hadoop git commit: HADOOP-11638. OpensslSecureRandom.c pthreads_thread_id should support FreeBSD and Solaris in addition to Linux. Contributed by Kiran Kumar M R.

Posted by zj...@apache.org.
HADOOP-11638. OpensslSecureRandom.c pthreads_thread_id should support FreeBSD and Solaris in addition to Linux. Contributed by Kiran Kumar M R.


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

Branch: refs/heads/YARN-2928
Commit: 72cd4e4a4eb2a9f8695d4c67eb55dd2be36c52dc
Parents: 685dbaf
Author: cnauroth <cn...@apache.org>
Authored: Mon Mar 16 13:26:57 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Mon Mar 16 14:04:40 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt   |  3 +++
 .../hadoop/crypto/random/OpensslSecureRandom.c    | 18 +++++++++++++++++-
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/72cd4e4a/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index aa17841..2a2b916 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1105,6 +1105,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11558. Fix dead links to doc of hadoop-tools. (Jean-Pierre 
     Matsumoto via ozawa)
 
+    HADOOP-11638. OpensslSecureRandom.c pthreads_thread_id should support FreeBSD
+    and Solaris in addition to Linux. (Kiran Kumar M R via cnauroth)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72cd4e4a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/random/OpensslSecureRandom.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/random/OpensslSecureRandom.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/random/OpensslSecureRandom.c
index 6c31d10..8f0c06d 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/random/OpensslSecureRandom.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/random/OpensslSecureRandom.c
@@ -29,6 +29,10 @@
 #include <sys/types.h>
 #endif
 
+#if defined(__FreeBSD__)
+#include <pthread_np.h>
+#endif
+
 #ifdef WINDOWS
 #include <windows.h>
 #endif
@@ -274,7 +278,19 @@ static void pthreads_locking_callback(int mode, int type, char *file, int line)
 
 static unsigned long pthreads_thread_id(void)
 {
-  return (unsigned long)syscall(SYS_gettid);
+  unsigned long thread_id = 0;
+#if defined(__linux__)
+  thread_id = (unsigned long)syscall(SYS_gettid);
+#elif defined(__FreeBSD__)
+  thread_id = (unsigned long)pthread_getthreadid_np();
+#elif defined(__sun)
+  thread_id = (unsigned long)pthread_self();
+#elif defined(__APPLE__)
+  (void)pthread_threadid_np(pthread_self(), &thread_id);
+#else
+#error "Platform not supported"
+#endif
+  return thread_id;
 }
 
 #endif /* UNIX */


[44/50] hadoop git commit: YARN-3305. Normalize AM resource request on app submission. Contributed by Rohith Sharmaks

Posted by zj...@apache.org.
YARN-3305. Normalize AM resource request on app submission. Contributed by Rohith Sharmaks


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

Branch: refs/heads/YARN-2928
Commit: 968425e9f7b850ff9c2ab8ca37a64c3fdbe77dbf
Parents: 32b4330
Author: Jian He <ji...@apache.org>
Authored: Tue Mar 17 13:49:59 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Mar 17 13:49:59 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  7 +++--
 .../server/resourcemanager/RMAppManager.java    |  6 ++++-
 .../server/resourcemanager/TestAppManager.java  |  5 ++++
 .../resourcemanager/TestClientRMService.java    |  5 ++++
 .../capacity/TestCapacityScheduler.java         | 27 ++++++++++++++++++++
 5 files changed, 47 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/968425e9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index fee0ce0..bb752ab 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -66,8 +66,11 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
-     YARN-3197. Confusing log generated by CapacityScheduler. (Varun Saxena 
-     via devaraj)
+    YARN-3197. Confusing log generated by CapacityScheduler. (Varun Saxena 
+    via devaraj)
+
+    YARN-3305. Normalize AM resource request on app submission. (Rohith Sharmaks
+    via jianhe)
 
 Release 2.7.0 - UNRELEASED
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/968425e9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 8dcfe67..9197630 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -390,7 +390,11 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
             + " for application " + submissionContext.getApplicationId(), e);
         throw e;
       }
-      
+      SchedulerUtils.normalizeRequest(amReq, scheduler.getResourceCalculator(),
+          scheduler.getClusterResource(),
+          scheduler.getMinimumResourceCapability(),
+          scheduler.getMaximumResourceCapability(),
+          scheduler.getMinimumResourceCapability());
       return amReq;
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/968425e9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.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/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
index d2ac4ef..5ebc68c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Before;
@@ -604,6 +605,10 @@ public class TestAppManager{
     when(scheduler.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(
             YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
+
+    ResourceCalculator rs = mock(ResourceCalculator.class);
+    when(scheduler.getResourceCalculator()).thenReturn(rs);
+
     return scheduler;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/968425e9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.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/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index dd2b3f8..430763c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -146,6 +146,7 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.UTCClock;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -1289,6 +1290,10 @@ public class TestClientRMService {
         Arrays.asList(getApplicationAttemptId(103)));
     ApplicationAttemptId attemptId = getApplicationAttemptId(1);
     when(yarnScheduler.getAppResourceUsageReport(attemptId)).thenReturn(null);
+
+    ResourceCalculator rs = mock(ResourceCalculator.class);
+    when(yarnScheduler.getResourceCalculator()).thenReturn(rs);
+
     return yarnScheduler;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/968425e9/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 7a265dc..052749f 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
@@ -2543,6 +2543,33 @@ public class TestCapacityScheduler {
         + "queue-a's max capacity will be violated if container allocated");
   }
 
+  // Test verifies AM Used resource for LeafQueue when AM ResourceRequest is
+  // lesser than minimumAllocation
+  @Test(timeout = 30000)
+  public void testAMUsedResource() throws Exception {
+    MockRM rm = setUpMove();
+    Configuration conf = rm.getConfig();
+    int minAllocMb =
+        conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
+    int amMemory = 50;
+    assertTrue("AM memory is greater than or equql to minAllocation",
+        amMemory < minAllocMb);
+    Resource minAllocResource = Resource.newInstance(minAllocMb, 1);
+    String queueName = "a1";
+    RMApp rmApp = rm.submitApp(amMemory, "app-1", "user_0", null, queueName);
+       
+    assertEquals("RMApp does not containes minimum allocation",
+        minAllocResource, rmApp.getAMResourceRequest().getCapability());
+    
+    ResourceScheduler scheduler = rm.getRMContext().getScheduler();
+    LeafQueue queueA =
+        (LeafQueue) ((CapacityScheduler) scheduler).getQueue(queueName);
+    assertEquals("Minimum Resource for AM is incorrect", minAllocResource,
+        queueA.getUser("user_0").getResourceUsage().getAMUsed());
+    rm.stop();
+  }
+
   private void setMaxAllocMb(Configuration conf, int maxAllocMb) {
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
         maxAllocMb);


[13/50] hadoop git commit: HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Masatake Iwasaki.

Posted by zj...@apache.org.
HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Masatake Iwasaki.


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

Branch: refs/heads/YARN-2928
Commit: 7da136ecca4dafc83ef69b5d9980fa5b67ada084
Parents: bd0a9ba
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sun Mar 15 14:17:35 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sun Mar 15 14:17:35 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt       |  3 +++
 .../src/site/markdown/SchedulerLoadSimulator.md       |  2 +-
 .../src/site/markdown/HadoopStreaming.md.vm           | 14 +++++++-------
 3 files changed, 11 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7da136ec/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 55028cb..e386723 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1100,6 +1100,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt
     synchronization. (Sean Busbey via yliu)
 
+    HADOOP-11558. Fix dead links to doc of hadoop-tools. (Masatake Iwasaki
+    via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7da136ec/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
index ca179ee..2cffc86 100644
--- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
+++ b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
@@ -43,7 +43,7 @@ The Yarn Scheduler Load Simulator (SLS) is such a tool, which can simulate large
 o
 The simulator will exercise the real Yarn `ResourceManager` removing the network factor by simulating `NodeManagers` and `ApplicationMasters` via handling and dispatching `NM`/`AMs` heartbeat events from within the same JVM. To keep tracking of scheduler behavior and performance, a scheduler wrapper will wrap the real scheduler.
 
-The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](https://hadoop.apache.org/docs/stable/rumen.html).
+The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](../hadoop-rumen/Rumen.html).
 
 The simulator will produce real time metrics while executing, including:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7da136ec/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
index 0b64586..b4c5e38 100644
--- a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
+++ b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
@@ -201,7 +201,7 @@ To specify additional local temp directories use:
      -D mapred.system.dir=/tmp/system
      -D mapred.temp.dir=/tmp/temp
 
-**Note:** For more details on job configuration parameters see: [mapred-default.xml](./mapred-default.xml)
+**Note:** For more details on job configuration parameters see: [mapred-default.xml](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml)
 
 $H4 Specifying Map-Only Jobs
 
@@ -322,7 +322,7 @@ More Usage Examples
 
 $H3 Hadoop Partitioner Class
 
-Hadoop has a library class, [KeyFieldBasedPartitioner](../../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
+Hadoop has a library class, [KeyFieldBasedPartitioner](../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D stream.map.output.field.separator=. \
@@ -372,7 +372,7 @@ Sorting within each partition for the reducer(all 4 fields used for sorting)
 
 $H3 Hadoop Comparator Class
 
-Hadoop has a library class, [KeyFieldBasedComparator](../../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
+Hadoop has a library class, [KeyFieldBasedComparator](../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.job.output.key.comparator.class=org.apache.hadoop.mapreduce.lib.partition.KeyFieldBasedComparator \
@@ -406,7 +406,7 @@ Sorting output for the reducer (where second field used for sorting)
 
 $H3 Hadoop Aggregate Package
 
-Hadoop has a library package called [Aggregate](../../org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
+Hadoop has a library package called [Aggregate](../api/org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
 
 To use Aggregate, simply specify "-reducer aggregate":
 
@@ -441,7 +441,7 @@ The python program myAggregatorForKeyCount.py looks like:
 
 $H3 Hadoop Field Selection Class
 
-Hadoop has a library class, [FieldSelectionMapReduce](../../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
+Hadoop has a library class, [FieldSelectionMapReduce](../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.map.output.key.field.separator=. \
@@ -480,7 +480,7 @@ As an example, consider the problem of zipping (compressing) a set of files acro
 
 $H3 How many reducers should I use?
 
-See MapReduce Tutorial for details: [Reducer](./MapReduceTutorial.html#Reducer)
+See MapReduce Tutorial for details: [Reducer](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Reducer)
 
 $H3 If I set up an alias in my shell script, will that work after -mapper?
 
@@ -556,4 +556,4 @@ A streaming process can use the stderr to emit status information. To set a stat
 
 $H3 How do I get the Job variables in a streaming job's mapper/reducer?
 
-See [Configured Parameters](./MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.
+See [Configured Parameters](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.


[25/50] hadoop git commit: HDFS-7886. Fix TestFileTruncate falures. Contributed by Plamen Jeliazkov and Konstantin Shvachko.

Posted by zj...@apache.org.
HDFS-7886. Fix TestFileTruncate falures. Contributed by Plamen Jeliazkov and Konstantin Shvachko.

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

Branch: refs/heads/YARN-2928
Commit: ce5de93a5837e115e1f0b7d3c5a67ace25385a63
Parents: 587d8be
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Mon Mar 16 12:54:04 2015 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Mon Mar 16 12:54:04 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  | 44 ++++++++++++++++++--
 .../hdfs/server/namenode/TestFileTruncate.java  | 18 ++++----
 3 files changed, 51 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce5de93a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 93237af..d313b6c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1157,6 +1157,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and
     fail to tell the DFSClient about it because of a network error (cmccabe)
 
+    HDFS-7886. Fix TestFileTruncate falures. (Plamen Jeliazkov and shv)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce5de93a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 834eb32..9208ed2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -77,9 +77,12 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Util;
@@ -1343,7 +1346,6 @@ public class MiniDFSCluster {
     }
 
     int curDatanodesNum = dataNodes.size();
-    final int curDatanodesNumSaved = curDatanodesNum;
     // for mincluster's the default initialDelay for BRs is 0
     if (conf.get(DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
       conf.setLong(DFS_BLOCKREPORT_INITIAL_DELAY_KEY, 0);
@@ -2022,7 +2024,23 @@ public class MiniDFSCluster {
    */
   public synchronized boolean restartDataNode(int i, boolean keepPort)
       throws IOException {
-    DataNodeProperties dnprop = stopDataNode(i);
+    return restartDataNode(i, keepPort, false);
+  }
+
+  /**
+   * Restart a particular DataNode.
+   * @param idn index of the DataNode
+   * @param keepPort true if should restart on the same port
+   * @param expireOnNN true if NameNode should expire the DataNode heartbeat
+   * @return
+   * @throws IOException
+   */
+  public synchronized boolean restartDataNode(
+      int idn, boolean keepPort, boolean expireOnNN) throws IOException {
+    DataNodeProperties dnprop = stopDataNode(idn);
+    if(expireOnNN) {
+      setDataNodeDead(dnprop.datanode.getDatanodeId());
+    }
     if (dnprop == null) {
       return false;
     } else {
@@ -2030,6 +2048,24 @@ public class MiniDFSCluster {
     }
   }
 
+  /**
+   * Expire a DataNode heartbeat on the NameNode
+   * @param dnId
+   * @throws IOException
+   */
+  public void setDataNodeDead(DatanodeID dnId) throws IOException {
+    DatanodeDescriptor dnd =
+        NameNodeAdapter.getDatanode(getNamesystem(), dnId);
+    dnd.setLastUpdate(0L);
+    BlockManagerTestUtil.checkHeartbeat(getNamesystem().getBlockManager());
+  }
+
+  public void setDataNodesDead() throws IOException {
+    for (DataNodeProperties dnp : dataNodes) {
+      setDataNodeDead(dnp.datanode.getDatanodeId());
+    }
+  }
+
   /*
    * Restart all datanodes, on the same ports if keepPort is true
    */
@@ -2255,8 +2291,8 @@ public class MiniDFSCluster {
     // make sure all datanodes have sent first heartbeat to namenode,
     // using (capacity == 0) as proxy.
     for (DatanodeInfo dn : dnInfo) {
-      if (dn.getCapacity() == 0) {
-        LOG.info("dn.getCapacity() == 0");
+      if (dn.getCapacity() == 0 || dn.getLastUpdate() <= 0) {
+        LOG.info("No heartbeat from DataNode: " + dn.toString());
         return true;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce5de93a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 3b6e107..8d447ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -679,10 +679,10 @@ public class TestFileTruncate {
       boolean isReady = fs.truncate(p, newLength);
       assertFalse(isReady);
     } finally {
-      cluster.restartDataNode(dn);
+      cluster.restartDataNode(dn, true, true);
       cluster.waitActive();
-      cluster.triggerBlockReports();
     }
+    checkBlockRecovery(p);
 
     LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
     /*
@@ -699,7 +699,6 @@ public class TestFileTruncate {
     assertEquals(newBlock.getBlock().getGenerationStamp(),
         oldBlock.getBlock().getGenerationStamp() + 1);
 
-    checkBlockRecovery(p);
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // Old replica is disregarded and replaced with the truncated one
@@ -741,10 +740,10 @@ public class TestFileTruncate {
       boolean isReady = fs.truncate(p, newLength);
       assertFalse(isReady);
     } finally {
-      cluster.restartDataNode(dn);
+      cluster.restartDataNode(dn, true, true);
       cluster.waitActive();
-      cluster.triggerBlockReports();
     }
+    checkBlockRecovery(p);
 
     LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
     /*
@@ -757,7 +756,6 @@ public class TestFileTruncate {
     assertEquals(newBlock.getBlock().getGenerationStamp(),
         oldBlock.getBlock().getGenerationStamp() + 1);
 
-    checkBlockRecovery(p);
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // New block is replicated to dn1
@@ -800,9 +798,10 @@ public class TestFileTruncate {
     boolean isReady = fs.truncate(p, newLength);
     assertFalse(isReady);
 
-    cluster.restartDataNode(dn0);
-    cluster.restartDataNode(dn1);
+    cluster.restartDataNode(dn0, true, true);
+    cluster.restartDataNode(dn1, true, true);
     cluster.waitActive();
+    checkBlockRecovery(p);
     cluster.triggerBlockReports();
 
     LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
@@ -815,7 +814,6 @@ public class TestFileTruncate {
     assertEquals(newBlock.getBlock().getGenerationStamp(),
         oldBlock.getBlock().getGenerationStamp() + 1);
 
-    checkBlockRecovery(p);
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // Old replica is disregarded and replaced with the truncated one on dn0
@@ -859,6 +857,7 @@ public class TestFileTruncate {
     assertFalse(isReady);
 
     cluster.shutdownDataNodes();
+    cluster.setDataNodesDead();
     try {
       for(int i = 0; i < SUCCESS_ATTEMPTS && cluster.isDataNodeUp(); i++) {
         Thread.sleep(SLEEP);
@@ -871,6 +870,7 @@ public class TestFileTruncate {
           StartupOption.REGULAR, null);
       cluster.waitActive();
     }
+    checkBlockRecovery(p);
 
     fs.delete(parent, true);
   }


[21/50] hadoop git commit: HADOOP-9477. Amendment to CHANGES.txt.

Posted by zj...@apache.org.
HADOOP-9477. Amendment to CHANGES.txt.


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

Branch: refs/heads/YARN-2928
Commit: d1eebd9c9c1fed5877ef2665959e9bd1485d080c
Parents: 03b77ed
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Mon Mar 16 09:16:57 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Mon Mar 16 09:16:57 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1eebd9c/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e161d7d..a43a153 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -37,9 +37,6 @@ Trunk (Unreleased)
 
     HADOOP-11565. Add --slaves shell option (aw)
 
-    HADOOP-9477. Add posixGroups support for LDAP groups mapping service.
-    (Dapeng Sun via Yongjun Zhang)
-
   IMPROVEMENTS
 
     HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution
@@ -447,6 +444,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11226. Add a configuration to set ipc.Client's traffic class with
     IPTOS_LOWDELAY|IPTOS_RELIABILITY. (Gopal V via ozawa)
 
+    HADOOP-9477. Add posixGroups support for LDAP groups mapping service.
+    (Dapeng Sun via Yongjun Zhang)
+
   IMPROVEMENTS
 
     HADOOP-11692. Improve authentication failure WARN message to avoid user


[29/50] hadoop git commit: YARN-3349. Treat all exceptions as failure in TestFSRMStateStore#testFSRMStateStoreClientRetry. Contributed by Zhihai Xu.

Posted by zj...@apache.org.
YARN-3349. Treat all exceptions as failure in TestFSRMStateStore#testFSRMStateStoreClientRetry. Contributed by Zhihai Xu.


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

Branch: refs/heads/YARN-2928
Commit: 7522a643faeea2d8a8e2c7409ae60e0973e7cf38
Parents: 2681ed9
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Mar 17 08:09:55 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Mar 17 08:09:55 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                          |  3 +++
 .../resourcemanager/recovery/TestFSRMStateStore.java     | 11 +++--------
 2 files changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7522a643/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 26ef7d3..b8e07a0 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -772,6 +772,9 @@ Release 2.7.0 - UNRELEASED
     YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in 
     doc comments. (Akira AJISAKA, Andrew Purtell, and Allen Wittenauer via ozawa)
 
+    YARN-3349. Treat all exceptions as failure in
+    TestFSRMStateStore#testFSRMStateStoreClientRetry. (Zhihai Xu via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7522a643/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.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/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
index 675d73c..d2eddd6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
@@ -100,11 +100,11 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
           workingDirPathURI.toString());
       conf.set(YarnConfiguration.FS_RM_STATE_STORE_RETRY_POLICY_SPEC,
         "100,6000");
-      conf.setInt(YarnConfiguration.FS_RM_STATE_STORE_NUM_RETRIES, 5);
+      conf.setInt(YarnConfiguration.FS_RM_STATE_STORE_NUM_RETRIES, 8);
       conf.setLong(YarnConfiguration.FS_RM_STATE_STORE_RETRY_INTERVAL_MS,
           900L);
       this.store = new TestFileSystemRMStore(conf);
-      Assert.assertEquals(store.getNumRetries(), 5);
+      Assert.assertEquals(store.getNumRetries(), 8);
       Assert.assertEquals(store.getRetryInterval(), 900L);
       return store;
     }
@@ -277,12 +277,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
                 ApplicationStateData.newInstance(111, 111, "user", null,
                     RMAppState.ACCEPTED, "diagnostics", 333));
           } catch (Exception e) {
-            // TODO 0 datanode exception will not be retried by dfs client, fix
-            // that separately.
-            if (!e.getMessage().contains("could only be replicated" +
-                " to 0 nodes instead of minReplication (=1)")) {
-              assertionFailedInThread.set(true);
-            }
+            assertionFailedInThread.set(true);
             e.printStackTrace();
           }
         }


[28/50] hadoop git commit: HADOOP-8059. Update CHANGES.txt to target 2.7.0.

Posted by zj...@apache.org.
HADOOP-8059. Update CHANGES.txt to target 2.7.0.


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

Branch: refs/heads/YARN-2928
Commit: 2681ed96983577d2eae19e749e25bfc5fd0589c3
Parents: 72cd4e4
Author: cnauroth <cn...@apache.org>
Authored: Mon Mar 16 14:17:25 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Mon Mar 16 14:17:25 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2681ed96/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2a2b916..2e04cc1 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -68,9 +68,6 @@ Trunk (Unreleased)
     HADOOP-7659. fs -getmerge isn't guaranteed to work well over non-HDFS
     filesystems (harsh)
 
-    HADOOP-8059. Add javadoc to InterfaceAudience and InterfaceStability.
-    (Brandon Li via suresh)
-
     HADOOP-8434. Add tests for Configuration setter methods.
     (Madhukara Phatak via suresh)
 
@@ -694,6 +691,9 @@ Release 2.7.0 - UNRELEASED
 
     HADOOP-11714. Add more trace log4j messages to SpanReceiverHost (cmccabe)
 
+    HADOOP-8059. Add javadoc to InterfaceAudience and InterfaceStability.
+    (Brandon Li via suresh)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.


[20/50] hadoop git commit: HADOOP-11692. Amendment to CHANGES.txt.

Posted by zj...@apache.org.
HADOOP-11692. Amendment to CHANGES.txt.


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

Branch: refs/heads/YARN-2928
Commit: 03b77ede9249d7d16654257035dfc01a7a0a8c50
Parents: 3da9a97
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Mon Mar 16 09:08:41 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Mon Mar 16 09:08:41 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/03b77ede/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index bb08cfe..e161d7d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -449,6 +449,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-11692. Improve authentication failure WARN message to avoid user
+    confusion. (Yongjun Zhang)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -1088,9 +1091,6 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11686. MiniKDC cannot change ORG_NAME or ORG_DOMAIN.
     (Duo Zhang via wheat9)
 
-    HADOOP-11692. Improve authentication failure WARN message to avoid user
-    confusion. (Yongjun Zhang)
-
     HADOOP-11618. DelegateToFileSystem erroneously uses default FS's port in
     constructor. (Brahma Reddy Battula via gera)
 


[41/50] hadoop git commit: YARN-3243. CapacityScheduler should pass headroom from parent to children to make sure ParentQueue obey its capacity limits. Contributed by Wangda Tan.

Posted by zj...@apache.org.
YARN-3243. CapacityScheduler should pass headroom from parent to children to make sure ParentQueue obey its capacity limits. Contributed by Wangda Tan.


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

Branch: refs/heads/YARN-2928
Commit: 487374b7fe0c92fc7eb1406c568952722b5d5b15
Parents: a89b087
Author: Jian He <ji...@apache.org>
Authored: Tue Mar 17 10:22:15 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Mar 17 10:24:23 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../scheduler/capacity/AbstractCSQueue.java     | 112 ++++++-
 .../scheduler/capacity/CSQueue.java             |   4 +-
 .../scheduler/capacity/CapacityScheduler.java   |  33 ++-
 .../scheduler/capacity/LeafQueue.java           | 292 +++++++------------
 .../scheduler/capacity/ParentQueue.java         | 140 +++------
 .../scheduler/common/fica/FiCaSchedulerApp.java |  16 +-
 .../capacity/TestApplicationLimits.java         |   8 +-
 .../capacity/TestCapacityScheduler.java         |  59 ++++
 .../scheduler/capacity/TestChildQueueOrder.java |  25 +-
 .../scheduler/capacity/TestLeafQueue.java       | 142 ++++-----
 .../scheduler/capacity/TestParentQueue.java     |  97 +++---
 .../scheduler/capacity/TestReservations.java    | 147 +++++-----
 13 files changed, 561 insertions(+), 517 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 82934ad..f5b72d7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -56,6 +56,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    YARN-3243. CapacityScheduler should pass headroom from parent to children
+    to make sure ParentQueue obey its capacity limits. (Wangda Tan via jianhe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.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/AbstractCSQueue.java
index d800709..4e53060 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/AbstractCSQueue.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/AbstractCSQueue.java
@@ -20,10 +20,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
@@ -34,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
@@ -49,6 +53,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import com.google.common.collect.Sets;
 
 public abstract class AbstractCSQueue implements CSQueue {
+  private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);
   
   CSQueue parent;
   final String queueName;
@@ -406,21 +411,102 @@ public abstract class AbstractCSQueue implements CSQueue {
                                         parentQ.getPreemptionDisabled());
   }
   
-  protected Resource getCurrentResourceLimit(Resource clusterResource,
-      ResourceLimits currentResourceLimits) {
+  private Resource getCurrentLimitResource(String nodeLabel,
+      Resource clusterResource, ResourceLimits currentResourceLimits) {
     /*
-     * Queue's max available resource = min(my.max, my.limit)
-     * my.limit is set by my parent, considered used resource of my siblings
+     * Current limit resource: For labeled resource: limit = queue-max-resource
+     * (TODO, this part need update when we support labeled-limit) For
+     * non-labeled resource: limit = min(queue-max-resource,
+     * limit-set-by-parent)
      */
     Resource queueMaxResource =
-        Resources.multiplyAndNormalizeDown(resourceCalculator, clusterResource,
-            queueCapacities.getAbsoluteMaximumCapacity(), minimumAllocation);
-    Resource queueCurrentResourceLimit =
-        Resources.min(resourceCalculator, clusterResource, queueMaxResource,
-            currentResourceLimits.getLimit());
-    queueCurrentResourceLimit =
-        Resources.roundDown(resourceCalculator, queueCurrentResourceLimit,
-            minimumAllocation);
-    return queueCurrentResourceLimit;
+        Resources.multiplyAndNormalizeDown(resourceCalculator,
+            labelManager.getResourceByLabel(nodeLabel, clusterResource),
+            queueCapacities.getAbsoluteMaximumCapacity(nodeLabel), minimumAllocation);
+    if (nodeLabel.equals(RMNodeLabelsManager.NO_LABEL)) {
+      return Resources.min(resourceCalculator, clusterResource,
+          queueMaxResource, currentResourceLimits.getLimit());
+    }
+    return queueMaxResource;
+  }
+  
+  synchronized boolean canAssignToThisQueue(Resource clusterResource,
+      Set<String> nodeLabels, ResourceLimits currentResourceLimits,
+      Resource nowRequired, Resource resourceCouldBeUnreserved) {
+    // Get label of this queue can access, it's (nodeLabel AND queueLabel)
+    Set<String> labelCanAccess;
+    if (null == nodeLabels || nodeLabels.isEmpty()) {
+      labelCanAccess = new HashSet<String>();
+      // Any queue can always access any node without label
+      labelCanAccess.add(RMNodeLabelsManager.NO_LABEL);
+    } else {
+      labelCanAccess = new HashSet<String>(
+          accessibleLabels.contains(CommonNodeLabelsManager.ANY) ? nodeLabels
+              : Sets.intersection(accessibleLabels, nodeLabels));
+    }
+    
+    for (String label : labelCanAccess) {
+      // New total resource = used + required
+      Resource newTotalResource =
+          Resources.add(queueUsage.getUsed(label), nowRequired);
+
+      Resource currentLimitResource =
+          getCurrentLimitResource(label, clusterResource, currentResourceLimits);
+
+      // if reservation continous looking enabled, check to see if could we
+      // potentially use this node instead of a reserved node if the application
+      // has reserved containers.
+      // TODO, now only consider reservation cases when the node has no label
+      if (this.reservationsContinueLooking
+          && label.equals(RMNodeLabelsManager.NO_LABEL)
+          && Resources.greaterThan(resourceCalculator, clusterResource,
+              resourceCouldBeUnreserved, Resources.none())) {
+        // resource-without-reserved = used - reserved
+        Resource newTotalWithoutReservedResource =
+            Resources.subtract(newTotalResource, resourceCouldBeUnreserved);
+        
+        // when total-used-without-reserved-resource < currentLimit, we still
+        // have chance to allocate on this node by unreserving some containers
+        if (Resources.lessThan(resourceCalculator, clusterResource,
+            newTotalWithoutReservedResource, currentLimitResource)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("try to use reserved: " + getQueueName()
+                + " usedResources: " + queueUsage.getUsed()
+                + ", clusterResources: " + clusterResource
+                + ", reservedResources: " + resourceCouldBeUnreserved
+                + ", capacity-without-reserved: "
+                + newTotalWithoutReservedResource + ", maxLimitCapacity: "
+                + currentLimitResource); 
+          }
+          return true;
+        }
+      }
+      
+      // Otherwise, if any of the label of this node beyond queue limit, we
+      // cannot allocate on this node. Consider a small epsilon here.
+      if (Resources.greaterThan(resourceCalculator, clusterResource,
+          newTotalResource, currentLimitResource)) {
+        return false;
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(getQueueName()
+            + "Check assign to queue, label=" + label
+            + " usedResources: " + queueUsage.getUsed(label)
+            + " clusterResources: " + clusterResource
+            + " currentUsedCapacity "
+            + Resources.divide(resourceCalculator, clusterResource,
+                queueUsage.getUsed(label),
+                labelManager.getResourceByLabel(label, clusterResource))
+            + " max-capacity: "
+            + queueCapacities.getAbsoluteMaximumCapacity(label)
+            + ")");
+      }
+      return true;
+    }
+    
+    // Actually, this will not happen, since labelCanAccess will be always
+    // non-empty
+    return false;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.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/CSQueue.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/CSQueue.java
index 0a60acc..1a9448a 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/CSQueue.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/CSQueue.java
@@ -189,13 +189,11 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    * Assign containers to applications in the queue or it's children (if any).
    * @param clusterResource the resource of the cluster.
    * @param node node on which resources are available
-   * @param needToUnreserve assign container only if it can unreserve one first
    * @param resourceLimits how much overall resource of this queue can use. 
    * @return the assignment
    */
   public CSAssignment assignContainers(Resource clusterResource,
-      FiCaSchedulerNode node, boolean needToUnreserve,
-      ResourceLimits resourceLimits);
+      FiCaSchedulerNode node, ResourceLimits resourceLimits);
   
   /**
    * A container assigned to the queue has completed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/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 756e537..c86c0ff 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
@@ -1061,9 +1061,14 @@ public class CapacityScheduler extends
           node.getNodeID());
       
       LeafQueue queue = ((LeafQueue)reservedApplication.getQueue());
-      CSAssignment assignment = queue.assignContainers(clusterResource, node,
-          false, new ResourceLimits(
-              clusterResource));
+      CSAssignment assignment =
+          queue.assignContainers(
+              clusterResource,
+              node,
+              // TODO, now we only consider limits for parent for non-labeled
+              // resources, should consider labeled resources as well.
+              new ResourceLimits(labelManager.getResourceByLabel(
+                  RMNodeLabelsManager.NO_LABEL, clusterResource)));
       
       RMContainer excessReservation = assignment.getExcessReservation();
       if (excessReservation != null) {
@@ -1087,8 +1092,13 @@ public class CapacityScheduler extends
           LOG.debug("Trying to schedule on node: " + node.getNodeName() +
               ", available: " + node.getAvailableResource());
         }
-        root.assignContainers(clusterResource, node, false, new ResourceLimits(
-            clusterResource));
+        root.assignContainers(
+            clusterResource,
+            node,
+            // TODO, now we only consider limits for parent for non-labeled
+            // resources, should consider labeled resources as well.
+            new ResourceLimits(labelManager.getResourceByLabel(
+                RMNodeLabelsManager.NO_LABEL, clusterResource)));
       }
     } else {
       LOG.info("Skipping scheduling since node " + node.getNodeID() + 
@@ -1209,6 +1219,13 @@ public class CapacityScheduler extends
         usePortForNodeName, nodeManager.getNodeLabels());
     this.nodes.put(nodeManager.getNodeID(), schedulerNode);
     Resources.addTo(clusterResource, nodeManager.getTotalCapability());
+
+    // update this node to node label manager
+    if (labelManager != null) {
+      labelManager.activateNode(nodeManager.getNodeID(),
+          nodeManager.getTotalCapability());
+    }
+    
     root.updateClusterResource(clusterResource, new ResourceLimits(
         clusterResource));
     int numNodes = numNodeManagers.incrementAndGet();
@@ -1220,12 +1237,6 @@ public class CapacityScheduler extends
     if (scheduleAsynchronously && numNodes == 1) {
       asyncSchedulerThread.beginSchedule();
     }
-    
-    // update this node to node label manager
-    if (labelManager != null) {
-      labelManager.activateNode(nodeManager.getNodeID(),
-          nodeManager.getTotalCapability());
-    }
   }
 
   private synchronized void removeNode(RMNode nodeInfo) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.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/LeafQueue.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/LeafQueue.java
index a607a62..dd6a894 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/LeafQueue.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/LeafQueue.java
@@ -76,7 +76,6 @@ import org.apache.hadoop.yarn.server.utils.Lock.NoLock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Sets;
 
 @Private
 @Unstable
@@ -157,7 +156,7 @@ public class LeafQueue extends AbstractCSQueue {
     // and all queues may not be realized yet, we'll use (optimistic) 
     // absoluteMaxCapacity (it will be replaced with the more accurate 
     // absoluteMaxAvailCapacity during headroom/userlimit/allocation events)
-    computeQueueCurrentLimitAndSetHeadroomInfo(clusterResource);
+    setQueueResourceLimitsInfo(clusterResource);
 
     CapacitySchedulerConfiguration conf = csContext.getConfiguration();
     userLimit = conf.getUserLimit(getQueuePath());
@@ -739,9 +738,8 @@ public class LeafQueue extends AbstractCSQueue {
   
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
-      FiCaSchedulerNode node, boolean needToUnreserve,
-      ResourceLimits currentResourceLimits) {
-    this.currentResourceLimits = currentResourceLimits;
+      FiCaSchedulerNode node, ResourceLimits currentResourceLimits) {
+    updateCurrentResourceLimits(currentResourceLimits, clusterResource);
     
     if(LOG.isDebugEnabled()) {
       LOG.debug("assignContainers: node=" + node.getNodeName()
@@ -796,7 +794,7 @@ public class LeafQueue extends AbstractCSQueue {
             continue;
           }
           if (!this.reservationsContinueLooking) {
-            if (!needContainers(application, priority, required)) {
+            if (!shouldAllocOrReserveNewContainer(application, priority, required)) {
               if (LOG.isDebugEnabled()) {
                 LOG.debug("doesn't need containers based on reservation algo!");
               }
@@ -818,8 +816,8 @@ public class LeafQueue extends AbstractCSQueue {
                   required, requestedNodeLabels);          
           
           // Check queue max-capacity limit
-          if (!canAssignToThisQueue(clusterResource, required,
-              node.getLabels(), application, true)) {
+          if (!super.canAssignToThisQueue(clusterResource, node.getLabels(),
+              this.currentResourceLimits, required, application.getCurrentReservation())) {
             return NULL_ASSIGNMENT;
           }
 
@@ -835,7 +833,7 @@ public class LeafQueue extends AbstractCSQueue {
           // Try to schedule
           CSAssignment assignment =  
             assignContainersOnNode(clusterResource, node, application, priority, 
-                null, needToUnreserve);
+                null);
 
           // Did the application skip this node?
           if (assignment.getSkipped()) {
@@ -896,7 +894,7 @@ public class LeafQueue extends AbstractCSQueue {
 
     // Try to assign if we have sufficient resources
     assignContainersOnNode(clusterResource, node, application, priority, 
-        rmContainer, false);
+        rmContainer);
     
     // Doesn't matter... since it's already charged for at time of reservation
     // "re-reservation" is *free*
@@ -938,102 +936,14 @@ public class LeafQueue extends AbstractCSQueue {
         Resources.roundDown(resourceCalculator, headroom, minimumAllocation);
     return headroom;
   }
-
-  synchronized boolean canAssignToThisQueue(Resource clusterResource,
-      Resource required, Set<String> nodeLabels, FiCaSchedulerApp application, 
-      boolean checkReservations) {
-    // Get label of this queue can access, it's (nodeLabel AND queueLabel)
-    Set<String> labelCanAccess;
-    if (null == nodeLabels || nodeLabels.isEmpty()) {
-      labelCanAccess = new HashSet<String>();
-      // Any queue can always access any node without label
-      labelCanAccess.add(RMNodeLabelsManager.NO_LABEL);
-    } else {
-      labelCanAccess = new HashSet<String>(Sets.intersection(accessibleLabels, nodeLabels));
-    }
-    
-    boolean canAssign = true;
-    for (String label : labelCanAccess) {
-      Resource potentialTotalCapacity =
-          Resources.add(queueUsage.getUsed(label), required);
-      
-      float potentialNewCapacity =
-          Resources.divide(resourceCalculator, clusterResource,
-              potentialTotalCapacity,
-              labelManager.getResourceByLabel(label, clusterResource));
-      // if enabled, check to see if could we potentially use this node instead
-      // of a reserved node if the application has reserved containers
-      // TODO, now only consider reservation cases when the node has no label
-      if (this.reservationsContinueLooking && checkReservations
-          && label.equals(RMNodeLabelsManager.NO_LABEL)) {
-        float potentialNewWithoutReservedCapacity = Resources.divide(
-            resourceCalculator,
-            clusterResource,
-            Resources.subtract(potentialTotalCapacity,
-               application.getCurrentReservation()),
-            labelManager.getResourceByLabel(label, clusterResource));
-
-        if (potentialNewWithoutReservedCapacity <= queueCapacities
-            .getAbsoluteMaximumCapacity()) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("try to use reserved: "
-                + getQueueName()
-                + " usedResources: "
-                + queueUsage.getUsed()
-                + " clusterResources: "
-                + clusterResource
-                + " reservedResources: "
-                + application.getCurrentReservation()
-                + " currentCapacity "
-                + Resources.divide(resourceCalculator, clusterResource,
-                    queueUsage.getUsed(), clusterResource) + " required " + required
-                + " potentialNewWithoutReservedCapacity: "
-                + potentialNewWithoutReservedCapacity + " ( "
-                + " max-capacity: "
-                + queueCapacities.getAbsoluteMaximumCapacity() + ")");
-          }
-          // we could potentially use this node instead of reserved node
-          return true;
-        }
-      }
-      
-      // Otherwise, if any of the label of this node beyond queue limit, we
-      // cannot allocate on this node. Consider a small epsilon here.
-      if (potentialNewCapacity > queueCapacities
-          .getAbsoluteMaximumCapacity(label) + 1e-4) {
-        canAssign = false;
-        break;
-      }
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(getQueueName()
-            + "Check assign to queue, label=" + label
-            + " usedResources: " + queueUsage.getUsed(label)
-            + " clusterResources: " + clusterResource
-            + " currentCapacity "
-            + Resources.divide(resourceCalculator, clusterResource,
-                queueUsage.getUsed(label),
-                labelManager.getResourceByLabel(label, clusterResource))
-            + " potentialNewCapacity: " + potentialNewCapacity + " ( "
-            + " max-capacity: " + queueCapacities.getAbsoluteMaximumCapacity()
-            + ")");
-      }
-    }
-    
-    return canAssign;
-  }
   
-  private Resource computeQueueCurrentLimitAndSetHeadroomInfo(
+  private void setQueueResourceLimitsInfo(
       Resource clusterResource) {
-    Resource queueCurrentResourceLimit =
-        getCurrentResourceLimit(clusterResource, currentResourceLimits);
-    
     synchronized (queueResourceLimitsInfo) {
-      queueResourceLimitsInfo.setQueueCurrentLimit(queueCurrentResourceLimit);
+      queueResourceLimitsInfo.setQueueCurrentLimit(currentResourceLimits
+          .getLimit());
       queueResourceLimitsInfo.setClusterResource(clusterResource);
     }
-
-    return queueCurrentResourceLimit;
   }
 
   @Lock({LeafQueue.class, FiCaSchedulerApp.class})
@@ -1048,16 +958,16 @@ public class LeafQueue extends AbstractCSQueue {
         computeUserLimit(application, clusterResource, required,
             queueUser, requestedLabels);
 
-    Resource currentResourceLimit =
-        computeQueueCurrentLimitAndSetHeadroomInfo(clusterResource);
+    setQueueResourceLimitsInfo(clusterResource);
     
     Resource headroom =
-        getHeadroom(queueUser, currentResourceLimit, clusterResource, userLimit);
+        getHeadroom(queueUser, currentResourceLimits.getLimit(),
+            clusterResource, userLimit);
     
     if (LOG.isDebugEnabled()) {
       LOG.debug("Headroom calculation for user " + user + ": " + 
           " userLimit=" + userLimit + 
-          " queueMaxAvailRes=" + currentResourceLimit + 
+          " queueMaxAvailRes=" + currentResourceLimits.getLimit() + 
           " consumed=" + queueUser.getUsed() + 
           " headroom=" + headroom);
     }
@@ -1207,8 +1117,8 @@ public class LeafQueue extends AbstractCSQueue {
     return true;
   }
 
-  boolean needContainers(FiCaSchedulerApp application, Priority priority,
-      Resource required) {
+  boolean shouldAllocOrReserveNewContainer(FiCaSchedulerApp application,
+      Priority priority, Resource required) {
     int requiredContainers = application.getTotalRequiredResources(priority);
     int reservedContainers = application.getNumReservedContainers(priority);
     int starvation = 0;
@@ -1240,7 +1150,7 @@ public class LeafQueue extends AbstractCSQueue {
 
   private CSAssignment assignContainersOnNode(Resource clusterResource,
       FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority,
-      RMContainer reservedContainer, boolean needToUnreserve) {
+      RMContainer reservedContainer) {
     Resource assigned = Resources.none();
 
     NodeType requestType = null;
@@ -1252,7 +1162,7 @@ public class LeafQueue extends AbstractCSQueue {
       requestType = NodeType.NODE_LOCAL;
       assigned =
           assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest, 
-            node, application, priority, reservedContainer, needToUnreserve,
+            node, application, priority, reservedContainer,
             allocatedContainer);
       if (Resources.greaterThan(resourceCalculator, clusterResource,
           assigned, Resources.none())) {
@@ -1280,7 +1190,7 @@ public class LeafQueue extends AbstractCSQueue {
 
       assigned = 
           assignRackLocalContainers(clusterResource, rackLocalResourceRequest, 
-            node, application, priority, reservedContainer, needToUnreserve,
+            node, application, priority, reservedContainer,
             allocatedContainer);
       if (Resources.greaterThan(resourceCalculator, clusterResource,
           assigned, Resources.none())) {
@@ -1308,7 +1218,7 @@ public class LeafQueue extends AbstractCSQueue {
 
       assigned =
           assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
-            node, application, priority, reservedContainer, needToUnreserve,
+            node, application, priority, reservedContainer,
             allocatedContainer);
 
       // update locality statistics
@@ -1320,13 +1230,24 @@ public class LeafQueue extends AbstractCSQueue {
     
     return SKIP_ASSIGNMENT;
   }
+  
+  private Resource getMinimumResourceNeedUnreserved(Resource askedResource) {
+    // First we need to get minimum resource we need unreserve
+    // minimum-resource-need-unreserve = used + asked - limit
+    Resource minimumUnreservedResource =
+        Resources.subtract(Resources.add(queueUsage.getUsed(), askedResource),
+            currentResourceLimits.getLimit());
+    return minimumUnreservedResource;
+  }
 
   @Private
   protected boolean findNodeToUnreserve(Resource clusterResource,
       FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority,
-      Resource capability) {
+      Resource askedResource, Resource minimumUnreservedResource) {
     // need to unreserve some other container first
-    NodeId idToUnreserve = application.getNodeIdToUnreserve(priority, capability);
+    NodeId idToUnreserve =
+        application.getNodeIdToUnreserve(priority, minimumUnreservedResource,
+            resourceCalculator, clusterResource);
     if (idToUnreserve == null) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("checked to see if could unreserve for app but nothing "
@@ -1343,7 +1264,7 @@ public class LeafQueue extends AbstractCSQueue {
       LOG.debug("unreserving for app: " + application.getApplicationId()
         + " on nodeId: " + idToUnreserve
         + " in order to replace reserved application and place it on node: "
-        + node.getNodeID() + " needing: " + capability);
+        + node.getNodeID() + " needing: " + askedResource);
     }
 
     // headroom
@@ -1364,15 +1285,7 @@ public class LeafQueue extends AbstractCSQueue {
 
   @Private
   protected boolean checkLimitsToReserve(Resource clusterResource,
-      FiCaSchedulerApp application, Resource capability,
-      boolean needToUnreserve) {
-    if (needToUnreserve) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("we needed to unreserve to be able to allocate");
-      }
-      return false;
-    }
-
+      FiCaSchedulerApp application, Resource capability) {
     // we can't reserve if we got here based on the limit
     // checks assuming we could unreserve!!!
     Resource userLimit = computeUserLimitAndSetHeadroom(application,
@@ -1380,7 +1293,8 @@ public class LeafQueue extends AbstractCSQueue {
 
     // Check queue max-capacity limit,
     // TODO: Consider reservation on labels
-    if (!canAssignToThisQueue(clusterResource, capability, null, application, false)) {
+    if (!canAssignToThisQueue(clusterResource, null,
+        this.currentResourceLimits, capability, Resources.none())) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("was going to reserve but hit queue limit");
       }
@@ -1402,43 +1316,40 @@ public class LeafQueue extends AbstractCSQueue {
   private Resource assignNodeLocalContainers(Resource clusterResource,
       ResourceRequest nodeLocalResourceRequest, FiCaSchedulerNode node,
       FiCaSchedulerApp application, Priority priority,
-      RMContainer reservedContainer, boolean needToUnreserve,
-      MutableObject allocatedContainer) {
+      RMContainer reservedContainer, MutableObject allocatedContainer) {
     if (canAssign(application, priority, node, NodeType.NODE_LOCAL, 
         reservedContainer)) {
       return assignContainer(clusterResource, node, application, priority,
           nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer,
-          needToUnreserve, allocatedContainer);
+          allocatedContainer);
     }
     
     return Resources.none();
   }
 
-  private Resource assignRackLocalContainers(
-      Resource clusterResource, ResourceRequest rackLocalResourceRequest,  
-      FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority,
-      RMContainer reservedContainer, boolean needToUnreserve,
-      MutableObject allocatedContainer) {
+  private Resource assignRackLocalContainers(Resource clusterResource,
+      ResourceRequest rackLocalResourceRequest, FiCaSchedulerNode node,
+      FiCaSchedulerApp application, Priority priority,
+      RMContainer reservedContainer, MutableObject allocatedContainer) {
     if (canAssign(application, priority, node, NodeType.RACK_LOCAL,
         reservedContainer)) {
       return assignContainer(clusterResource, node, application, priority,
           rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer,
-          needToUnreserve, allocatedContainer);
+          allocatedContainer);
     }
     
     return Resources.none();
   }
 
-  private Resource assignOffSwitchContainers(
-      Resource clusterResource, ResourceRequest offSwitchResourceRequest,
-      FiCaSchedulerNode node, FiCaSchedulerApp application, Priority priority, 
-      RMContainer reservedContainer, boolean needToUnreserve,
-      MutableObject allocatedContainer) {
+  private Resource assignOffSwitchContainers(Resource clusterResource,
+      ResourceRequest offSwitchResourceRequest, FiCaSchedulerNode node,
+      FiCaSchedulerApp application, Priority priority,
+      RMContainer reservedContainer, MutableObject allocatedContainer) {
     if (canAssign(application, priority, node, NodeType.OFF_SWITCH,
         reservedContainer)) {
       return assignContainer(clusterResource, node, application, priority,
           offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer,
-          needToUnreserve, allocatedContainer);
+          allocatedContainer);
     }
     
     return Resources.none();
@@ -1522,13 +1433,12 @@ public class LeafQueue extends AbstractCSQueue {
   private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode node, 
       FiCaSchedulerApp application, Priority priority, 
       ResourceRequest request, NodeType type, RMContainer rmContainer,
-      boolean needToUnreserve, MutableObject createdContainer) {
+      MutableObject createdContainer) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("assignContainers: node=" + node.getNodeName()
         + " application=" + application.getApplicationId()
         + " priority=" + priority.getPriority()
-        + " request=" + request + " type=" + type
-        + " needToUnreserve= " + needToUnreserve);
+        + " request=" + request + " type=" + type);
     }
     
     // check if the resource request can access the label
@@ -1548,12 +1458,14 @@ public class LeafQueue extends AbstractCSQueue {
     Resource available = node.getAvailableResource();
     Resource totalResource = node.getTotalResource();
 
-    if (!Resources.fitsIn(capability, totalResource)) {
+    if (!Resources.lessThanOrEqual(resourceCalculator, clusterResource,
+        capability, totalResource)) {
       LOG.warn("Node : " + node.getNodeID()
           + " does not have sufficient resource for request : " + request
           + " node total capability : " + node.getTotalResource());
       return Resources.none();
     }
+
     assert Resources.greaterThan(
         resourceCalculator, clusterResource, available, Resources.none());
 
@@ -1566,18 +1478,9 @@ public class LeafQueue extends AbstractCSQueue {
       LOG.warn("Couldn't get container for allocation!");
       return Resources.none();
     }
-
-    // default to true since if reservation continue look feature isn't on
-    // needContainers is checked earlier and we wouldn't have gotten this far
-    boolean canAllocContainer = true;
-    if (this.reservationsContinueLooking) {
-      // based on reservations can we allocate/reserve more or do we need
-      // to unreserve one first
-      canAllocContainer = needContainers(application, priority, capability);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("can alloc container is: " + canAllocContainer);
-      }
-    }
+    
+    boolean shouldAllocOrReserveNewContainer = shouldAllocOrReserveNewContainer(
+        application, priority, capability);
 
     // Can we allocate a container on this node?
     int availableContainers = 
@@ -1588,25 +1491,25 @@ public class LeafQueue extends AbstractCSQueue {
       // Did we previously reserve containers at this 'priority'?
       if (rmContainer != null) {
         unreserve(application, priority, node, rmContainer);
-      } else if (this.reservationsContinueLooking
-          && (!canAllocContainer || needToUnreserve)) {
-        // need to unreserve some other container first
-        boolean res = findNodeToUnreserve(clusterResource, node, application,
-            priority, capability);
-        if (!res) {
-          return Resources.none();
-        }
-      } else {
-        // we got here by possibly ignoring queue capacity limits. If the
-        // parameter needToUnreserve is true it means we ignored one of those
-        // limits in the chance we could unreserve. If we are here we aren't
-        // trying to unreserve so we can't allocate anymore due to that parent
-        // limit.
-        if (needToUnreserve) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("we needed to unreserve to be able to allocate, skipping");
+      } else if (this.reservationsContinueLooking && node.getLabels().isEmpty()) {
+        // when reservationsContinueLooking is set, we may need to unreserve
+        // some containers to meet this queue and its parents' resource limits
+        // TODO, need change here when we want to support continuous reservation
+        // looking for labeled partitions.
+        Resource minimumUnreservedResource =
+            getMinimumResourceNeedUnreserved(capability);
+        if (!shouldAllocOrReserveNewContainer
+            || Resources.greaterThan(resourceCalculator, clusterResource,
+                minimumUnreservedResource, Resources.none())) {
+          boolean containerUnreserved =
+              findNodeToUnreserve(clusterResource, node, application, priority,
+                  capability, minimumUnreservedResource);
+          // When (minimum-unreserved-resource > 0 OR we cannot allocate new/reserved 
+          // container (That means we *have to* unreserve some resource to
+          // continue)). If we failed to unreserve some resource,
+          if (!containerUnreserved) {
+            return Resources.none();
           }
-          return Resources.none();
         }
       }
 
@@ -1632,17 +1535,16 @@ public class LeafQueue extends AbstractCSQueue {
     } else {
       // if we are allowed to allocate but this node doesn't have space, reserve it or
       // if this was an already a reserved container, reserve it again
-      if ((canAllocContainer) || (rmContainer != null)) {
-
-        if (reservationsContinueLooking) {
-          // we got here by possibly ignoring parent queue capacity limits. If
-          // the parameter needToUnreserve is true it means we ignored one of
-          // those limits in the chance we could unreserve. If we are here
-          // we aren't trying to unreserve so we can't allocate
-          // anymore due to that parent limit
-          boolean res = checkLimitsToReserve(clusterResource, application, capability, 
-              needToUnreserve);
-          if (!res) {
+      if (shouldAllocOrReserveNewContainer || rmContainer != null) {
+
+        if (reservationsContinueLooking && rmContainer == null) {
+          // we could possibly ignoring parent queue capacity limits when
+          // reservationsContinueLooking is set.
+          // If we're trying to reserve a container here, not container will be
+          // unreserved for reserving the new one. Check limits again before
+          // reserve the new container
+          if (!checkLimitsToReserve(clusterResource, 
+              application, capability)) {
             return Resources.none();
           }
         }
@@ -1784,18 +1686,36 @@ public class LeafQueue extends AbstractCSQueue {
         Resources.multiplyAndNormalizeUp(resourceCalculator, clusterResource,
             queueCapacities.getAbsoluteCapacity(), minimumAllocation);
   }
+  
+  private void updateCurrentResourceLimits(
+      ResourceLimits currentResourceLimits, Resource clusterResource) {
+    // TODO: need consider non-empty node labels when resource limits supports
+    // node labels
+    // Even if ParentQueue will set limits respect child's max queue capacity,
+    // but when allocating reserved container, CapacityScheduler doesn't do
+    // this. So need cap limits by queue's max capacity here.
+    this.currentResourceLimits = currentResourceLimits;
+    Resource queueMaxResource =
+        Resources.multiplyAndNormalizeDown(resourceCalculator, labelManager
+            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
+            queueCapacities
+                .getAbsoluteMaximumCapacity(RMNodeLabelsManager.NO_LABEL),
+            minimumAllocation);
+    this.currentResourceLimits.setLimit(Resources.min(resourceCalculator,
+        clusterResource, queueMaxResource, currentResourceLimits.getLimit()));
+  }
 
   @Override
   public synchronized void updateClusterResource(Resource clusterResource,
       ResourceLimits currentResourceLimits) {
-    this.currentResourceLimits = currentResourceLimits;
+    updateCurrentResourceLimits(currentResourceLimits, clusterResource);
     lastClusterResource = clusterResource;
     updateAbsoluteCapacityResource(clusterResource);
     
     // Update headroom info based on new cluster resource value
     // absoluteMaxCapacity now,  will be replaced with absoluteMaxAvailCapacity
     // during allocation
-    computeQueueCurrentLimitAndSetHeadroomInfo(clusterResource);
+    setQueueResourceLimitsInfo(clusterResource);
     
     // Update metrics
     CSQueueUtils.updateQueueStatistics(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.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/ParentQueue.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/ParentQueue.java
index 7feaa15..5ed6bb8 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/ParentQueue.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/ParentQueue.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -48,7 +47,6 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -63,8 +61,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import com.google.common.collect.Sets;
-
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -380,8 +376,7 @@ public class ParentQueue extends AbstractCSQueue {
 
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
-      FiCaSchedulerNode node, boolean needToUnreserve,
-      ResourceLimits resourceLimits) {
+      FiCaSchedulerNode node, ResourceLimits resourceLimits) {
     CSAssignment assignment = 
         new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
     Set<String> nodeLabels = node.getLabels();
@@ -397,21 +392,18 @@ public class ParentQueue extends AbstractCSQueue {
           + getQueueName());
       }
       
-      boolean localNeedToUnreserve = false;
-      
       // Are we over maximum-capacity for this queue?
-      if (!canAssignToThisQueue(clusterResource, nodeLabels)) {
-        // check to see if we could if we unreserve first
-        localNeedToUnreserve = assignToQueueIfUnreserve(clusterResource);
-        if (!localNeedToUnreserve) {
-          break;
-        }
+      // This will also consider parent's limits and also continuous reservation
+      // looking
+      if (!super.canAssignToThisQueue(clusterResource, nodeLabels, resourceLimits,
+          minimumAllocation, Resources.createResource(getMetrics()
+              .getReservedMB(), getMetrics().getReservedVirtualCores()))) {
+        break;
       }
       
       // Schedule
       CSAssignment assignedToChild = 
-          assignContainersToChildQueues(clusterResource, node,
-              localNeedToUnreserve | needToUnreserve, resourceLimits);
+          assignContainersToChildQueues(clusterResource, node, resourceLimits);
       assignment.setType(assignedToChild.getType());
       
       // Done if no child-queue assigned anything
@@ -459,74 +451,6 @@ public class ParentQueue extends AbstractCSQueue {
     return assignment;
   }
 
-  private synchronized boolean canAssignToThisQueue(Resource clusterResource,
-      Set<String> nodeLabels) {
-    Set<String> labelCanAccess =
-        new HashSet<String>(
-            accessibleLabels.contains(CommonNodeLabelsManager.ANY) ? nodeLabels
-                : Sets.intersection(accessibleLabels, nodeLabels));
-    if (nodeLabels.isEmpty()) {
-      // Any queue can always access any node without label
-      labelCanAccess.add(RMNodeLabelsManager.NO_LABEL);
-    }
-    
-    boolean canAssign = true;
-    for (String label : labelCanAccess) {
-      float currentAbsoluteLabelUsedCapacity =
-          Resources.divide(resourceCalculator, clusterResource,
-              queueUsage.getUsed(label),
-              labelManager.getResourceByLabel(label, clusterResource));
-      // if any of the label doesn't beyond limit, we can allocate on this node
-      if (currentAbsoluteLabelUsedCapacity >= 
-            queueCapacities.getAbsoluteMaximumCapacity(label)) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(getQueueName() + " used=" + queueUsage.getUsed()
-              + " current-capacity (" + queueUsage.getUsed(label) + ") "
-              + " >= max-capacity ("
-              + labelManager.getResourceByLabel(label, clusterResource) + ")");
-        }
-        canAssign = false;
-        break;
-      }
-    }
-    
-    return canAssign;
-  }
-
-  
-  private synchronized boolean assignToQueueIfUnreserve(Resource clusterResource) {
-    if (this.reservationsContinueLooking) {      
-      // check to see if we could potentially use this node instead of a reserved
-      // node
-
-      Resource reservedResources = Resources.createResource(getMetrics()
-          .getReservedMB(), getMetrics().getReservedVirtualCores());
-      float capacityWithoutReservedCapacity = Resources.divide(
-          resourceCalculator, clusterResource,
-          Resources.subtract(queueUsage.getUsed(), reservedResources),
-          clusterResource);
-
-      if (capacityWithoutReservedCapacity <= queueCapacities
-          .getAbsoluteMaximumCapacity()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("parent: try to use reserved: " + getQueueName()
-            + " usedResources: " + queueUsage.getUsed().getMemory()
-            + " clusterResources: " + clusterResource.getMemory()
-            + " reservedResources: " + reservedResources.getMemory()
-            + " currentCapacity " + ((float) queueUsage.getUsed().getMemory())
-            / clusterResource.getMemory()
-            + " potentialNewWithoutReservedCapacity: "
-            + capacityWithoutReservedCapacity + " ( " + " max-capacity: "
-            + queueCapacities.getAbsoluteMaximumCapacity() + ")");
-        }
-        // we could potentially use this node instead of reserved node
-        return true;
-      }
-    }
-    return false;
-   }
-
-  
   private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
     return (node.getReservedContainer() == null) && 
         Resources.greaterThanOrEqual(resourceCalculator, clusterResource, 
@@ -534,28 +458,38 @@ public class ParentQueue extends AbstractCSQueue {
   }
   
   private ResourceLimits getResourceLimitsOfChild(CSQueue child,
-      Resource clusterResource, ResourceLimits myLimits) {
-    /*
-     * Set head-room of a given child, limit =
-     * min(minimum-of-limit-of-this-queue-and-ancestors, this.max) - this.used
-     * + child.used. To avoid any of this queue's and its ancestors' limit
-     * being violated
-     */
-    Resource myCurrentLimit =
-        getCurrentResourceLimit(clusterResource, myLimits);
-    // My available resource = my-current-limit - my-used-resource
-    Resource myMaxAvailableResource = Resources.subtract(myCurrentLimit,
-        getUsedResources());
-    // Child's limit = my-available-resource + resource-already-used-by-child
+      Resource clusterResource, ResourceLimits parentLimits) {
+    // Set resource-limit of a given child, child.limit =
+    // min(my.limit - my.used + child.used, child.max)
+
+    // Parent available resource = parent-limit - parent-used-resource
+    Resource parentMaxAvailableResource =
+        Resources.subtract(parentLimits.getLimit(), getUsedResources());
+
+    // Child's limit = parent-available-resource + child-used
     Resource childLimit =
-        Resources.add(myMaxAvailableResource, child.getUsedResources());
-    
+        Resources.add(parentMaxAvailableResource, child.getUsedResources());
+
+    // Get child's max resource
+    Resource childConfiguredMaxResource =
+        Resources.multiplyAndNormalizeDown(resourceCalculator, labelManager
+            .getResourceByLabel(RMNodeLabelsManager.NO_LABEL, clusterResource),
+            child.getAbsoluteMaximumCapacity(), minimumAllocation);
+
+    // Child's limit should be capped by child configured max resource
+    childLimit =
+        Resources.min(resourceCalculator, clusterResource, childLimit,
+            childConfiguredMaxResource);
+
+    // Normalize before return
+    childLimit =
+        Resources.roundDown(resourceCalculator, childLimit, minimumAllocation);
+
     return new ResourceLimits(childLimit);
   }
   
   private synchronized CSAssignment assignContainersToChildQueues(
-      Resource cluster, FiCaSchedulerNode node, boolean needToUnreserve,
-      ResourceLimits limits) {
+      Resource cluster, FiCaSchedulerNode node, ResourceLimits limits) {
     CSAssignment assignment = 
         new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
     
@@ -573,9 +507,7 @@ public class ParentQueue extends AbstractCSQueue {
       ResourceLimits childLimits =
           getResourceLimitsOfChild(childQueue, cluster, limits);
       
-      assignment =
-          childQueue.assignContainers(cluster, node, needToUnreserve,
-              childLimits);
+      assignment = childQueue.assignContainers(cluster, node, childLimits);
       if(LOG.isDebugEnabled()) {
         LOG.debug("Assigned to queue: " + childQueue.getQueuePath() +
           " stats: " + childQueue + " --> " + 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.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/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 9f97b13..6cc2777 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -274,7 +274,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
   
   synchronized public NodeId getNodeIdToUnreserve(Priority priority,
-      Resource capability) {
+      Resource resourceNeedUnreserve, ResourceCalculator rc,
+      Resource clusterResource) {
 
     // first go around make this algorithm simple and just grab first
     // reservation that has enough resources
@@ -283,16 +284,19 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
       for (Map.Entry<NodeId, RMContainer> entry : reservedContainers.entrySet()) {
+        NodeId nodeId = entry.getKey();
+        Resource containerResource = entry.getValue().getContainer().getResource();
+        
         // make sure we unreserve one with at least the same amount of
         // resources, otherwise could affect capacity limits
-        if (Resources.fitsIn(capability, entry.getValue().getContainer()
-            .getResource())) {
+        if (Resources.lessThanOrEqual(rc, clusterResource,
+            resourceNeedUnreserve, containerResource)) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("unreserving node with reservation size: "
-                + entry.getValue().getContainer().getResource()
-                + " in order to allocate container with size: " + capability);
+                + containerResource
+                + " in order to allocate container with size: " + resourceNeedUnreserve);
           }
-          return entry.getKey();
+          return nodeId;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.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/TestApplicationLimits.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/TestApplicationLimits.java
index 8cad057..1ca5c97 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/TestApplicationLimits.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/TestApplicationLimits.java
@@ -611,7 +611,7 @@ public class TestApplicationLimits {
     app_0_0.updateResourceRequests(app_0_0_requests);
 
     // Schedule to compute 
-    queue.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+    queue.assignContainers(clusterResource, node_0, new ResourceLimits(
         clusterResource));
     Resource expectedHeadroom = Resources.createResource(10*16*GB, 1);
     assertEquals(expectedHeadroom, app_0_0.getHeadroom());
@@ -631,7 +631,7 @@ public class TestApplicationLimits {
     app_0_1.updateResourceRequests(app_0_1_requests);
 
     // Schedule to compute 
-    queue.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+    queue.assignContainers(clusterResource, node_0, new ResourceLimits(
         clusterResource)); // Schedule to compute
     assertEquals(expectedHeadroom, app_0_0.getHeadroom());
     assertEquals(expectedHeadroom, app_0_1.getHeadroom());// no change
@@ -651,7 +651,7 @@ public class TestApplicationLimits {
     app_1_0.updateResourceRequests(app_1_0_requests);
     
     // Schedule to compute 
-    queue.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+    queue.assignContainers(clusterResource, node_0, new ResourceLimits(
         clusterResource)); // Schedule to compute
     expectedHeadroom = Resources.createResource(10*16*GB / 2, 1); // changes
     assertEquals(expectedHeadroom, app_0_0.getHeadroom());
@@ -660,7 +660,7 @@ public class TestApplicationLimits {
 
     // Now reduce cluster size and check for the smaller headroom
     clusterResource = Resources.createResource(90*16*GB);
-    queue.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+    queue.assignContainers(clusterResource, node_0, new ResourceLimits(
         clusterResource)); // Schedule to compute
     expectedHeadroom = Resources.createResource(9*16*GB / 2, 1); // changes
     assertEquals(expectedHeadroom, app_0_0.getHeadroom());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/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 83ab104..7a265dc 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
@@ -125,6 +125,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.ComparisonFailure;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -2483,6 +2484,64 @@ public class TestCapacityScheduler {
     Assert.assertEquals(30 * GB,
         am1.doHeartbeat().getAvailableResources().getMemory());
   }
+  
+  @Test
+  public void testParentQueueMaxCapsAreRespected() throws Exception {
+    /*
+     * Queue tree:
+     *          Root
+     *        /     \
+     *       A       B
+     *      / \
+     *     A1 A2 
+     */
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+    csConf.setCapacity(A, 50);
+    csConf.setMaximumCapacity(A, 50);
+    csConf.setCapacity(B, 50);
+    
+    // Define 2nd-level queues
+    csConf.setQueues(A, new String[] {"a1", "a2"});
+    csConf.setCapacity(A1, 50);
+    csConf.setUserLimitFactor(A1, 100.0f);
+    csConf.setCapacity(A2, 50);
+    csConf.setUserLimitFactor(A2, 100.0f);
+    csConf.setCapacity(B1, B1_CAPACITY);
+    csConf.setUserLimitFactor(B1, 100.0f);
+    
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+    conf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true);
+
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    MockRM rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 24 * GB, rm1.getResourceTrackerService());
+    nm1.registerNode();
+    
+    // Launch app1 in a1, resource usage is 1GB (am) + 4GB * 2 = 9GB 
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    waitContainerAllocated(am1, 4 * GB, 2, 2, rm1, nm1);
+    
+    // Try to launch app2 in a2, asked 2GB, should success 
+    RMApp app2 = rm1.submitApp(2 * GB, "app", "user", null, "a2");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+    try {
+      // Try to allocate a container, a's usage=11G/max=12
+      // a1's usage=9G/max=12
+      // a2's usage=2G/max=12
+      // In this case, if a2 asked 2G, should fail.
+      waitContainerAllocated(am2, 2 * GB, 1, 2, rm1, nm1);
+    } catch (AssertionError failure) {
+      // Expected, return;
+      return;
+    }
+    Assert.fail("Shouldn't successfully allocate containers for am2, "
+        + "queue-a's max capacity will be violated if container allocated");
+  }
 
   private void setMaxAllocMb(Configuration conf, int maxAllocMb) {
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.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/TestChildQueueOrder.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/TestChildQueueOrder.java
index 7edb17d..71dc523 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/TestChildQueueOrder.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/TestChildQueueOrder.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doNothing;
@@ -145,7 +144,7 @@ public class TestChildQueueOrder {
         if (allocation > 0) {
           doReturn(new CSAssignment(Resources.none(), type)).
           when(queue)
-              .assignContainers(eq(clusterResource), eq(node), anyBoolean(),
+              .assignContainers(eq(clusterResource), eq(node),
                   any(ResourceLimits.class));
 
           // Mock the node's resource availability
@@ -157,7 +156,7 @@ public class TestChildQueueOrder {
         return new CSAssignment(allocatedResource, type);
       }
     }).
-    when(queue).assignContainers(eq(clusterResource), eq(node), anyBoolean(), 
+    when(queue).assignContainers(eq(clusterResource), eq(node), 
         any(ResourceLimits.class));
     doNothing().when(node).releaseContainer(any(Container.class));
   }
@@ -274,7 +273,7 @@ public class TestChildQueueOrder {
     stubQueueAllocation(b, clusterResource, node_0, 0*GB);
     stubQueueAllocation(c, clusterResource, node_0, 0*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
-    root.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+    root.assignContainers(clusterResource, node_0, new ResourceLimits(
         clusterResource));
     for(int i=0; i < 2; i++)
     {
@@ -282,7 +281,7 @@ public class TestChildQueueOrder {
       stubQueueAllocation(b, clusterResource, node_0, 1*GB);
       stubQueueAllocation(c, clusterResource, node_0, 0*GB);
       stubQueueAllocation(d, clusterResource, node_0, 0*GB);
-      root.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+      root.assignContainers(clusterResource, node_0, new ResourceLimits(
           clusterResource));
     } 
     for(int i=0; i < 3; i++)
@@ -291,7 +290,7 @@ public class TestChildQueueOrder {
       stubQueueAllocation(b, clusterResource, node_0, 0*GB);
       stubQueueAllocation(c, clusterResource, node_0, 1*GB);
       stubQueueAllocation(d, clusterResource, node_0, 0*GB);
-      root.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+      root.assignContainers(clusterResource, node_0, new ResourceLimits(
           clusterResource));
     }  
     for(int i=0; i < 4; i++)
@@ -300,7 +299,7 @@ public class TestChildQueueOrder {
       stubQueueAllocation(b, clusterResource, node_0, 0*GB);
       stubQueueAllocation(c, clusterResource, node_0, 0*GB);
       stubQueueAllocation(d, clusterResource, node_0, 1*GB);
-      root.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+      root.assignContainers(clusterResource, node_0, new ResourceLimits(
           clusterResource));
     }    
     verifyQueueMetrics(a, 1*GB, clusterResource);
@@ -334,7 +333,7 @@ public class TestChildQueueOrder {
       stubQueueAllocation(b, clusterResource, node_0, 0*GB);
       stubQueueAllocation(c, clusterResource, node_0, 0*GB);
       stubQueueAllocation(d, clusterResource, node_0, 0*GB);
-      root.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+      root.assignContainers(clusterResource, node_0, new ResourceLimits(
           clusterResource));
     }
     verifyQueueMetrics(a, 3*GB, clusterResource);
@@ -362,7 +361,7 @@ public class TestChildQueueOrder {
     stubQueueAllocation(b, clusterResource, node_0, 1*GB);
     stubQueueAllocation(c, clusterResource, node_0, 0*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
-    root.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+    root.assignContainers(clusterResource, node_0, new ResourceLimits(
         clusterResource));
     verifyQueueMetrics(a, 2*GB, clusterResource);
     verifyQueueMetrics(b, 3*GB, clusterResource);
@@ -389,7 +388,7 @@ public class TestChildQueueOrder {
     stubQueueAllocation(b, clusterResource, node_0, 0*GB);
     stubQueueAllocation(c, clusterResource, node_0, 0*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
-    root.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+    root.assignContainers(clusterResource, node_0, new ResourceLimits(
         clusterResource));
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
@@ -404,13 +403,13 @@ public class TestChildQueueOrder {
     stubQueueAllocation(b, clusterResource, node_0, 1*GB);
     stubQueueAllocation(c, clusterResource, node_0, 0*GB);
     stubQueueAllocation(d, clusterResource, node_0, 1*GB);
-    root.assignContainers(clusterResource, node_0, false, new ResourceLimits(
+    root.assignContainers(clusterResource, node_0, new ResourceLimits(
         clusterResource));
     InOrder allocationOrder = inOrder(d,b);
     allocationOrder.verify(d).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), any(ResourceLimits.class));
+        any(FiCaSchedulerNode.class), any(ResourceLimits.class));
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), any(ResourceLimits.class));
+        any(FiCaSchedulerNode.class), any(ResourceLimits.class));
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
     verifyQueueMetrics(c, 3*GB, clusterResource);


[34/50] hadoop git commit: MAPREDUCE-5755. MapTask.MapOutputBuffer#compare/swap should have @Override annotation. (ozawa)

Posted by zj...@apache.org.
MAPREDUCE-5755. MapTask.MapOutputBuffer#compare/swap should have @Override annotation. (ozawa)


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

Branch: refs/heads/YARN-2928
Commit: bb243cea93b6872ef8956311a2290ca0b83ebb22
Parents: f222bde
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Mar 17 14:55:15 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Mar 17 14:55:15 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../src/main/java/org/apache/hadoop/mapred/MapTask.java           | 2 ++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bb243cea/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index ee21b70..b5baf51 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -263,6 +263,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-4414. Add main methods to JobConf and YarnConfiguration,
     for debug purposes. (Plamen Jeliazkov via harsh)
 
+    MAPREDUCE-5755. MapTask.MapOutputBuffer#compare/swap should have
+    @Override annotation. (ozawa)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bb243cea/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
index 8094317..c4957b7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
@@ -1255,6 +1255,7 @@ public class MapTask extends Task {
      * Compare by partition, then by key.
      * @see IndexedSortable#compare
      */
+    @Override
     public int compare(final int mi, final int mj) {
       final int kvi = offsetFor(mi % maxRec);
       final int kvj = offsetFor(mj % maxRec);
@@ -1278,6 +1279,7 @@ public class MapTask extends Task {
      * Swap metadata for items i, j
      * @see IndexedSortable#swap
      */
+    @Override
     public void swap(final int mi, final int mj) {
       int iOff = (mi % maxRec) * METASIZE;
       int jOff = (mj % maxRec) * METASIZE;


[17/50] hadoop git commit: YARN-3171. Sort by Application id, AppAttempt and ContainerID doesn't work in ATS / RM web ui. Contributed by Naganarasimha G R

Posted by zj...@apache.org.
YARN-3171. Sort by Application id, AppAttempt and ContainerID doesn't
work in ATS / RM web ui. Contributed by Naganarasimha G R


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

Branch: refs/heads/YARN-2928
Commit: 3ff1ba2a7b00fdf06270d00b2193bde4b56b06b3
Parents: bc9cb3e
Author: Xuan <xg...@apache.org>
Authored: Sun Mar 15 20:26:10 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Sun Mar 15 20:26:10 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 3 +++
 .../org/apache/hadoop/yarn/server/webapp/WebPageUtils.java     | 6 +++---
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ff1ba2a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 77f8819..bcab88c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -766,6 +766,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3267. Timelineserver applies the ACL rules after applying the limit on
     the number of records (Chang Li via jeagles)
 
+    YARN-3171. Sort by Application id, AppAttempt and ContainerID doesn't work
+    in ATS / RM web ui. (Naganarasimha G R via xgong)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ff1ba2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
index 384a976..5acabf5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
@@ -44,7 +44,7 @@ public class WebPageUtils {
     StringBuilder sb = new StringBuilder();
     return sb
       .append("[\n")
-      .append("{'sType':'numeric', 'aTargets': [0]")
+      .append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }")
       .append("\n, {'sType':'numeric', 'aTargets': " +
           (isFairSchedulerPage ? "[6, 7]": "[5, 6]"))
@@ -63,7 +63,7 @@ public class WebPageUtils {
 
   private static String getAttemptsTableColumnDefs() {
     StringBuilder sb = new StringBuilder();
-    return sb.append("[\n").append("{'sType':'numeric', 'aTargets': [0]")
+    return sb.append("[\n").append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }")
       .append("\n, {'sType':'numeric', 'aTargets': [1]")
       .append(", 'mRender': renderHadoopDate }]").toString();
@@ -79,7 +79,7 @@ public class WebPageUtils {
 
   private static String getContainersTableColumnDefs() {
     StringBuilder sb = new StringBuilder();
-    return sb.append("[\n").append("{'sType':'numeric', 'aTargets': [0]")
+    return sb.append("[\n").append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }]").toString();
   }
 


[24/50] hadoop git commit: MAPREDUCE-4414. Add main methods to JobConf and YarnConfiguration, for debug purposes. Contributed by Plamen Jeliazkov.

Posted by zj...@apache.org.
MAPREDUCE-4414. Add main methods to JobConf and YarnConfiguration, for debug purposes. Contributed by Plamen Jeliazkov.


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

Branch: refs/heads/YARN-2928
Commit: 587d8be17bb9e71bad2881e24e7372d3e15125d3
Parents: bf3275d
Author: Harsh J <ha...@cloudera.com>
Authored: Tue Mar 17 01:01:06 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Tue Mar 17 01:03:08 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                            | 3 +++
 .../src/main/java/org/apache/hadoop/mapred/JobConf.java         | 5 +++++
 .../java/org/apache/hadoop/yarn/conf/YarnConfiguration.java     | 5 +++++
 3 files changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/587d8be1/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 28460d3..d02d725 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -253,6 +253,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-4414. Add main methods to JobConf and YarnConfiguration,
+    for debug purposes. (Plamen Jeliazkov via harsh)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/587d8be1/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 c388bda..9cac685 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
@@ -2140,5 +2140,10 @@ public class JobConf extends Configuration {
     }
   }
 
+  /* For debugging. Dump configurations to system output as XML format. */
+  public static void main(String[] args) throws Exception {
+    new JobConf(new Configuration()).writeXml(System.out);
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/587d8be1/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 f40c999..a527af4 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
@@ -1808,4 +1808,9 @@ public class YarnConfiguration extends Configuration {
     }
     return clusterId;
   }
+
+  /* For debugging. mp configurations to system output as XML format. */
+  public static void main(String[] args) throws Exception {
+    new YarnConfiguration(new Configuration()).writeXml(System.out);
+  }
 }


[42/50] hadoop git commit: HDFS-7940. Add tracing to DFSClient#setQuotaByStorageType (Rakesh R via Colin P. McCabe)

Posted by zj...@apache.org.
HDFS-7940. Add tracing to DFSClient#setQuotaByStorageType (Rakesh R via Colin P. McCabe)


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

Branch: refs/heads/YARN-2928
Commit: d8846707c58c5c3ec542128df13a82ddc05fb347
Parents: 487374b
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Mar 17 10:47:21 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Mar 17 10:47:21 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java           | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8846707/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index bbe1f02..3e11356 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -756,6 +756,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-2605. Remove redundant "Release 0.21.1" section from CHANGES.txt.
     (Allen Wittenauer via shv)
 
+    HDFS-7940. Add tracing to DFSClient#setQuotaByStorageType (Rakesh R via
+    Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8846707/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index f970fef..3c8fd31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -3089,6 +3089,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throw new IllegalArgumentException("Don't support Quota for storage type : "
         + type.toString());
     }
+    TraceScope scope = getPathTraceScope("setQuotaByStorageType", src);
     try {
       namenode.setQuota(src, HdfsConstants.QUOTA_DONT_SET, quota, type);
     } catch (RemoteException re) {
@@ -3097,6 +3098,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
         QuotaByStorageTypeExceededException.class,
         UnresolvedPathException.class,
         SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
   /**


[04/50] hadoop git commit: HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not idempotent. Contributed by Tsz Wo Nicholas Sze

Posted by zj...@apache.org.
HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not idempotent. Contributed by Tsz Wo Nicholas Sze


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

Branch: refs/heads/YARN-2928
Commit: f446669afb5c3d31a00c65449f27088b39e11ae3
Parents: 8180e67
Author: Brandon Li <br...@apache.org>
Authored: Fri Mar 13 10:42:22 2015 -0700
Committer: Brandon Li <br...@apache.org>
Committed: Fri Mar 13 10:42:22 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt          |  3 +++
 .../BlockInfoContiguousUnderConstruction.java        |  1 +
 .../hadoop/hdfs/server/namenode/FSNamesystem.java    | 15 +++++++++++++++
 .../hdfs/server/namenode/TestFileTruncate.java       |  2 ++
 4 files changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f446669a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 153453c..909182b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1142,6 +1142,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-6833.  DirectoryScanner should not register a deleting block with
     memory of DataNode.  (Shinichi Yamashita via szetszwo)
 
+    HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not 
+    idempotent (Tsz Wo Nicholas Sze via brandonli)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f446669a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
index 91b76cc..ae809a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
@@ -383,6 +383,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
 
   private void appendUCParts(StringBuilder sb) {
     sb.append("{UCState=").append(blockUCState)
+      .append(", truncateBlock=" + truncateBlock)
       .append(", primaryNodeIndex=").append(primaryNodeIndex)
       .append(", replicas=[");
     if (replicas != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f446669a/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 77b4a27..b384ce6 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
@@ -1966,6 +1966,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new UnsupportedOperationException(
           "Cannot truncate lazy persist file " + src);
     }
+
+    // Check if the file is already being truncated with the same length
+    final BlockInfoContiguous last = file.getLastBlock();
+    if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
+      final Block truncateBlock
+          = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock();
+      if (truncateBlock != null) {
+        final long truncateLength = file.computeFileSize(false, false)
+            + truncateBlock.getNumBytes();
+        if (newLength == truncateLength) {
+          return false;
+        }
+      }
+    }
+
     // Opening an existing file for truncate. May need lease recovery.
     recoverLeaseInternal(RecoverLeaseOp.TRUNCATE_FILE,
         iip, src, clientName, clientMachine, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f446669a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index b69d345..260d8bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -166,6 +166,8 @@ public class TestFileTruncate {
       LOG.info("newLength=" + newLength + ", isReady=" + isReady);
       assertEquals("File must be closed for truncating at the block boundary",
           isReady, newLength % BLOCK_SIZE == 0);
+      assertEquals("Truncate is not idempotent",
+          isReady, fs.truncate(p, newLength));
       if (!isReady) {
         checkBlockRecovery(p);
       }


[40/50] hadoop git commit: YARN-3243. CapacityScheduler should pass headroom from parent to children to make sure ParentQueue obey its capacity limits. Contributed by Wangda Tan.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.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/TestLeafQueue.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/TestLeafQueue.java
index a5a2e5f..972cabb 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/TestLeafQueue.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/TestLeafQueue.java
@@ -350,8 +350,8 @@ public class TestLeafQueue {
     // Start testing...
     
     // Only 1 container
-    a.assignContainers(clusterResource, node_0, false,
-        new ResourceLimits(clusterResource));
+    a.assignContainers(clusterResource, node_0, new ResourceLimits(
+        clusterResource));
     assertEquals(
         (int)(node_0.getTotalResource().getMemory() * a.getCapacity()) - (1*GB),
         a.getMetrics().getAvailableMB());
@@ -486,7 +486,7 @@ public class TestLeafQueue {
     // Start testing...
     
     // Only 1 container
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -497,7 +497,7 @@ public class TestLeafQueue {
 
     // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
     // you can get one container more than user-limit
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -506,7 +506,7 @@ public class TestLeafQueue {
     assertEquals(2*GB, a.getMetrics().getAllocatedMB());
     
     // Can't allocate 3rd due to user-limit
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -516,7 +516,7 @@ public class TestLeafQueue {
     
     // Bump up user-limit-factor, now allocate should work
     a.setUserLimitFactor(10);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(3*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
@@ -525,7 +525,7 @@ public class TestLeafQueue {
     assertEquals(3*GB, a.getMetrics().getAllocatedMB());
 
     // One more should work, for app_1, due to user-limit-factor
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(4*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
@@ -536,8 +536,8 @@ public class TestLeafQueue {
     // Test max-capacity
     // Now - no more allocs since we are at max-cap
     a.setMaxCapacity(0.5f);
-    a.assignContainers(clusterResource, node_0, false,
-        new ResourceLimits(clusterResource));
+    a.assignContainers(clusterResource, node_0, new ResourceLimits(
+        clusterResource));
     assertEquals(4*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
@@ -652,21 +652,21 @@ public class TestLeafQueue {
 //            recordFactory)));
 
     // 1 container to user_0
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
 
     // Again one to user_0 since he hasn't exceeded user limit yet
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(3*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
 
     // One more to user_0 since he is the only active user
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(4*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -718,7 +718,7 @@ public class TestLeafQueue {
     assertEquals("There should only be 1 active user!",
         1, qb.getActiveUsersManager().getNumActiveUsers());
     //get headroom
-    qb.assignContainers(clusterResource, node_0, false,
+    qb.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
         .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
@@ -738,7 +738,7 @@ public class TestLeafQueue {
         TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
             u1Priority, recordFactory)));
     qb.submitApplicationAttempt(app_2, user_1);
-    qb.assignContainers(clusterResource, node_1, false,
+    qb.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
         .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
@@ -781,9 +781,9 @@ public class TestLeafQueue {
              u1Priority, recordFactory)));
     qb.submitApplicationAttempt(app_1, user_0);
     qb.submitApplicationAttempt(app_3, user_1);
-    qb.assignContainers(clusterResource, node_0, false,
+    qb.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
-    qb.assignContainers(clusterResource, node_0, false,
+    qb.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
         .getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
@@ -802,7 +802,7 @@ public class TestLeafQueue {
     app_4.updateResourceRequests(Collections.singletonList(
               TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true,
                       u0Priority, recordFactory)));
-    qb.assignContainers(clusterResource, node_1, false,
+    qb.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     qb.computeUserLimitAndSetHeadroom(app_4, clusterResource, app_4
         .getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
@@ -875,7 +875,7 @@ public class TestLeafQueue {
             TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true,
                 priority, recordFactory)));
 
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -892,7 +892,7 @@ public class TestLeafQueue {
         TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 2, true,
             priority, recordFactory)));
 
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -981,7 +981,7 @@ public class TestLeafQueue {
         1, a.getActiveUsersManager().getNumActiveUsers());
 
     // 1 container to user_0
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -992,7 +992,7 @@ public class TestLeafQueue {
       // the application is not yet active
 
     // Again one to user_0 since he hasn't exceeded user limit yet
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(3*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1009,7 +1009,7 @@ public class TestLeafQueue {
 
     // No more to user_0 since he is already over user-limit
     // and no more containers to queue since it's already at max-cap
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(3*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1023,7 +1023,7 @@ public class TestLeafQueue {
         TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 0, true,
             priority, recordFactory)));
     assertEquals(1, a.getActiveUsersManager().getNumActiveUsers());
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(0*GB, app_2.getHeadroom().getMemory());   // hit queue max-cap 
   }
@@ -1094,7 +1094,7 @@ public class TestLeafQueue {
      */
     
     // Only 1 container
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -1102,7 +1102,7 @@ public class TestLeafQueue {
 
     // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
     // you can get one container more than user-limit
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1110,7 +1110,7 @@ public class TestLeafQueue {
     
     // Can't allocate 3rd due to user-limit
     a.setUserLimit(25);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1129,7 +1129,7 @@ public class TestLeafQueue {
     // Now allocations should goto app_2 since 
     // user_0 is at limit inspite of high user-limit-factor
     a.setUserLimitFactor(10);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1139,7 +1139,7 @@ public class TestLeafQueue {
 
     // Now allocations should goto app_0 since 
     // user_0 is at user-limit not above it
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(6*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
@@ -1150,7 +1150,7 @@ public class TestLeafQueue {
     // Test max-capacity
     // Now - no more allocs since we are at max-cap
     a.setMaxCapacity(0.5f);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(6*GB, a.getUsedResources().getMemory());
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
@@ -1162,7 +1162,7 @@ public class TestLeafQueue {
     // Now, allocations should goto app_3 since it's under user-limit 
     a.setMaxCapacity(1.0f);
     a.setUserLimitFactor(1);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(7*GB, a.getUsedResources().getMemory()); 
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
@@ -1171,7 +1171,7 @@ public class TestLeafQueue {
     assertEquals(1*GB, app_3.getCurrentConsumption().getMemory());
 
     // Now we should assign to app_3 again since user_2 is under user-limit
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(8*GB, a.getUsedResources().getMemory()); 
     assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
@@ -1271,7 +1271,7 @@ public class TestLeafQueue {
     // Start testing...
     
     // Only 1 container
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -1282,7 +1282,7 @@ public class TestLeafQueue {
 
     // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
     // you can get one container more than user-limit
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1291,7 +1291,7 @@ public class TestLeafQueue {
     assertEquals(2*GB, a.getMetrics().getAllocatedMB());
     
     // Now, reservation should kick in for app_1
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(6*GB, a.getUsedResources().getMemory()); 
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1308,7 +1308,7 @@ public class TestLeafQueue {
             ContainerState.COMPLETE, "",
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5*GB, a.getUsedResources().getMemory()); 
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -1325,7 +1325,7 @@ public class TestLeafQueue {
             ContainerState.COMPLETE, "",
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(4*GB, a.getUsedResources().getMemory());
     assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
@@ -1393,7 +1393,7 @@ public class TestLeafQueue {
 
     // Start testing...
 
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1403,7 +1403,7 @@ public class TestLeafQueue {
     assertEquals(0*GB, a.getMetrics().getAvailableMB());
 
     // Now, reservation should kick in for app_1
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(6*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1417,7 +1417,7 @@ public class TestLeafQueue {
     // We do not need locality delay here
     doReturn(-1).when(a).getNodeLocalityDelay();
     
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(10*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1434,7 +1434,7 @@ public class TestLeafQueue {
             ContainerState.COMPLETE, "",
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(8*GB, a.getUsedResources().getMemory());
     assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
@@ -1503,7 +1503,7 @@ public class TestLeafQueue {
     // Start testing...
     
     // Only 1 container
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(1*GB, a.getUsedResources().getMemory());
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -1511,14 +1511,14 @@ public class TestLeafQueue {
 
     // Also 2nd -> minCapacity = 1024 since (.1 * 8G) < minAlloc, also
     // you can get one container more than user-limit
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2*GB, a.getUsedResources().getMemory());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
     
     // Now, reservation should kick in for app_1
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(6*GB, a.getUsedResources().getMemory()); 
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
@@ -1533,7 +1533,7 @@ public class TestLeafQueue {
             ContainerState.COMPLETE, "",
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5*GB, a.getUsedResources().getMemory()); 
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -1543,7 +1543,7 @@ public class TestLeafQueue {
     assertEquals(1, app_1.getReReservations(priority));
 
     // Re-reserve
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5*GB, a.getUsedResources().getMemory()); 
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -1553,7 +1553,7 @@ public class TestLeafQueue {
     assertEquals(2, app_1.getReReservations(priority));
     
     // Try to schedule on node_1 now, should *move* the reservation
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(9*GB, a.getUsedResources().getMemory()); 
     assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
@@ -1571,7 +1571,7 @@ public class TestLeafQueue {
             ContainerState.COMPLETE, "",
             ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
         RMContainerEventType.KILL, null, true);
-    CSAssignment assignment = a.assignContainers(clusterResource, node_0, false,
+    CSAssignment assignment = a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(8*GB, a.getUsedResources().getMemory());
     assertEquals(0*GB, app_0.getCurrentConsumption().getMemory());
@@ -1643,7 +1643,7 @@ public class TestLeafQueue {
     CSAssignment assignment = null;
     
     // Start with off switch, shouldn't allocate due to delay scheduling
-    assignment = a.assignContainers(clusterResource, node_2, false,
+    assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1652,7 +1652,7 @@ public class TestLeafQueue {
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
 
     // Another off switch, shouldn't allocate due to delay scheduling
-    assignment = a.assignContainers(clusterResource, node_2, false,
+    assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1661,7 +1661,7 @@ public class TestLeafQueue {
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
     
     // Another off switch, shouldn't allocate due to delay scheduling
-    assignment = a.assignContainers(clusterResource, node_2, false,
+    assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1671,7 +1671,7 @@ public class TestLeafQueue {
     
     // Another off switch, now we should allocate 
     // since missedOpportunities=3 and reqdContainers=3
-    assignment = a.assignContainers(clusterResource, node_2, false,
+    assignment = a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1680,7 +1680,7 @@ public class TestLeafQueue {
     assertEquals(NodeType.OFF_SWITCH, assignment.getType());
     
     // NODE_LOCAL - node_0
-    assignment = a.assignContainers(clusterResource, node_0, false,
+    assignment = a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1689,7 +1689,7 @@ public class TestLeafQueue {
     assertEquals(NodeType.NODE_LOCAL, assignment.getType());
     
     // NODE_LOCAL - node_1
-    assignment = a.assignContainers(clusterResource, node_1, false,
+    assignment = a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1718,14 +1718,14 @@ public class TestLeafQueue {
     doReturn(1).when(a).getNodeLocalityDelay();
     
     // Shouldn't assign RACK_LOCAL yet
-    assignment = a.assignContainers(clusterResource, node_3, false,
+    assignment = a.assignContainers(clusterResource, node_3,
         new ResourceLimits(clusterResource));
     assertEquals(1, app_0.getSchedulingOpportunities(priority));
     assertEquals(2, app_0.getTotalRequiredResources(priority));
     assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
 
     // Should assign RACK_LOCAL now
-    assignment = a.assignContainers(clusterResource, node_3, false,
+    assignment = a.assignContainers(clusterResource, node_3,
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.RACK_LOCAL), eq(node_3), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1807,7 +1807,7 @@ public class TestLeafQueue {
     
     // Start with off switch, shouldn't allocate P1 due to delay scheduling
     // thus, no P2 either!
-    a.assignContainers(clusterResource, node_2, false,
+    a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
@@ -1820,7 +1820,7 @@ public class TestLeafQueue {
 
     // Another off-switch, shouldn't allocate P1 due to delay scheduling
     // thus, no P2 either!
-    a.assignContainers(clusterResource, node_2, false,
+    a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_2), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
@@ -1832,7 +1832,7 @@ public class TestLeafQueue {
     assertEquals(1, app_0.getTotalRequiredResources(priority_2));
 
     // Another off-switch, shouldn't allocate OFF_SWITCH P1
-    a.assignContainers(clusterResource, node_2, false,
+    a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.OFF_SWITCH), eq(node_2), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
@@ -1844,7 +1844,7 @@ public class TestLeafQueue {
     assertEquals(1, app_0.getTotalRequiredResources(priority_2));
 
     // Now, DATA_LOCAL for P1
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
@@ -1856,7 +1856,7 @@ public class TestLeafQueue {
     assertEquals(1, app_0.getTotalRequiredResources(priority_2));
 
     // Now, OFF_SWITCH for P2
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1), 
         eq(priority_1), any(ResourceRequest.class), any(Container.class));
@@ -1933,7 +1933,7 @@ public class TestLeafQueue {
     app_0.updateResourceRequests(app_0_requests_0);
     
     // NODE_LOCAL - node_0_1
-    a.assignContainers(clusterResource, node_0_0, false,
+    a.assignContainers(clusterResource, node_0_0,
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1942,7 +1942,7 @@ public class TestLeafQueue {
 
     // No allocation on node_1_0 even though it's node/rack local since
     // required(ANY) == 0
-    a.assignContainers(clusterResource, node_1_0, false,
+    a.assignContainers(clusterResource, node_1_0,
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1959,7 +1959,7 @@ public class TestLeafQueue {
 
     // No allocation on node_0_1 even though it's node/rack local since
     // required(rack_1) == 0
-    a.assignContainers(clusterResource, node_0_1, false,
+    a.assignContainers(clusterResource, node_0_1,
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -1967,7 +1967,7 @@ public class TestLeafQueue {
     assertEquals(1, app_0.getTotalRequiredResources(priority));
     
     // NODE_LOCAL - node_1
-    a.assignContainers(clusterResource, node_1_0, false,
+    a.assignContainers(clusterResource, node_1_0,
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -2220,7 +2220,7 @@ public class TestLeafQueue {
     
     // node_0_1  
     // Shouldn't allocate since RR(rack_0) = null && RR(ANY) = relax: false
-    a.assignContainers(clusterResource, node_0_1, false, 
+    a.assignContainers(clusterResource, node_0_1, 
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -2243,7 +2243,7 @@ public class TestLeafQueue {
 
     // node_1_1  
     // Shouldn't allocate since RR(rack_1) = relax: false
-    a.assignContainers(clusterResource, node_1_1, false, 
+    a.assignContainers(clusterResource, node_1_1, 
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_0_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -2274,7 +2274,7 @@ public class TestLeafQueue {
 
     // node_1_1  
     // Shouldn't allocate since node_1_1 is blacklisted
-    a.assignContainers(clusterResource, node_1_1, false, 
+    a.assignContainers(clusterResource, node_1_1, 
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -2303,7 +2303,7 @@ public class TestLeafQueue {
 
     // node_1_1  
     // Shouldn't allocate since rack_1 is blacklisted
-    a.assignContainers(clusterResource, node_1_1, false, 
+    a.assignContainers(clusterResource, node_1_1, 
         new ResourceLimits(clusterResource));
     verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -2330,7 +2330,7 @@ public class TestLeafQueue {
     // Blacklist: < host_0_0 >       <----
 
     // Now, should allocate since RR(rack_1) = relax: true
-    a.assignContainers(clusterResource, node_1_1, false, 
+    a.assignContainers(clusterResource, node_1_1, 
         new ResourceLimits(clusterResource));
     verify(app_0,never()).allocate(eq(NodeType.RACK_LOCAL), eq(node_1_1), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -2361,7 +2361,7 @@ public class TestLeafQueue {
     // host_1_0: 8G
     // host_1_1: 7G
 
-    a.assignContainers(clusterResource, node_1_0, false, 
+    a.assignContainers(clusterResource, node_1_0, 
         new ResourceLimits(clusterResource));
     verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), 
         any(Priority.class), any(ResourceRequest.class), any(Container.class));
@@ -2444,7 +2444,7 @@ public class TestLeafQueue {
             recordFactory)));
 
     try {
-      a.assignContainers(clusterResource, node_0, false, 
+      a.assignContainers(clusterResource, node_0, 
           new ResourceLimits(clusterResource));
     } catch (NullPointerException e) {
       Assert.fail("NPE when allocating container on node but "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.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/TestParentQueue.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/TestParentQueue.java
index 4f89386..7da1c97 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/TestParentQueue.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/TestParentQueue.java
@@ -156,7 +156,7 @@ public class TestParentQueue {
         // Next call - nothing
         if (allocation > 0) {
           doReturn(new CSAssignment(Resources.none(), type)).when(queue)
-              .assignContainers(eq(clusterResource), eq(node), eq(false),
+              .assignContainers(eq(clusterResource), eq(node),
                   any(ResourceLimits.class));
 
           // Mock the node's resource availability
@@ -167,8 +167,7 @@ public class TestParentQueue {
 
         return new CSAssignment(allocatedResource, type);
       }
-    }).
-when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
+    }).when(queue).assignContainers(eq(clusterResource), eq(node),
         any(ResourceLimits.class));
   }
   
@@ -232,7 +231,7 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // Simulate B returning a container on node_0
     stubQueueAllocation(a, clusterResource, node_0, 0*GB);
     stubQueueAllocation(b, clusterResource, node_0, 1*GB);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     verifyQueueMetrics(a, 0*GB, clusterResource);
     verifyQueueMetrics(b, 1*GB, clusterResource);
@@ -240,13 +239,13 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // Now, A should get the scheduling opportunity since A=0G/6G, B=1G/14G
     stubQueueAllocation(a, clusterResource, node_1, 2*GB);
     stubQueueAllocation(b, clusterResource, node_1, 1*GB);
-    root.assignContainers(clusterResource, node_1, false, 
+    root.assignContainers(clusterResource, node_1, 
         new ResourceLimits(clusterResource));
     InOrder allocationOrder = inOrder(a, b);
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(a, 2*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
 
@@ -254,13 +253,13 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // since A has 2/6G while B has 2/14G
     stubQueueAllocation(a, clusterResource, node_0, 1*GB);
     stubQueueAllocation(b, clusterResource, node_0, 2*GB);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     allocationOrder = inOrder(b, a);
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 4*GB, clusterResource);
 
@@ -268,13 +267,13 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // since A has 3/6G while B has 4/14G
     stubQueueAllocation(a, clusterResource, node_0, 0*GB);
     stubQueueAllocation(b, clusterResource, node_0, 4*GB);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     allocationOrder = inOrder(b, a);
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 8*GB, clusterResource);
 
@@ -282,13 +281,13 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // since A has 3/6G while B has 8/14G
     stubQueueAllocation(a, clusterResource, node_1, 1*GB);
     stubQueueAllocation(b, clusterResource, node_1, 1*GB);
-    root.assignContainers(clusterResource, node_1, false, 
+    root.assignContainers(clusterResource, node_1, 
         new ResourceLimits(clusterResource));
     allocationOrder = inOrder(a, b);
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(a, 4*GB, clusterResource);
     verifyQueueMetrics(b, 9*GB, clusterResource);
   }
@@ -405,6 +404,22 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
 
   @Test
   public void testMultiLevelQueues() throws Exception {
+    /*
+     * Structure of queue:
+     *            Root
+     *           ____________
+     *          /    |   \   \
+     *         A     B    C   D
+     *       / |   / | \   \
+     *      A1 A2 B1 B2 B3  C1
+     *                        \
+     *                         C11
+     *                           \
+     *                           C111
+     *                             \
+     *                              C1111
+     */
+    
     // Setup queue configs
     setupMultiLevelQueues(csConf);
     
@@ -449,7 +464,7 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     stubQueueAllocation(b, clusterResource, node_0, 0*GB);
     stubQueueAllocation(c, clusterResource, node_0, 1*GB);
     stubQueueAllocation(d, clusterResource, node_0, 0*GB);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     verifyQueueMetrics(a, 0*GB, clusterResource);
     verifyQueueMetrics(b, 0*GB, clusterResource);
@@ -462,7 +477,7 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     stubQueueAllocation(a, clusterResource, node_1, 0*GB);
     stubQueueAllocation(b2, clusterResource, node_1, 4*GB);
     stubQueueAllocation(c, clusterResource, node_1, 0*GB);
-    root.assignContainers(clusterResource, node_1, false, 
+    root.assignContainers(clusterResource, node_1, 
         new ResourceLimits(clusterResource));
     verifyQueueMetrics(a, 0*GB, clusterResource);
     verifyQueueMetrics(b, 4*GB, clusterResource);
@@ -474,15 +489,15 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     stubQueueAllocation(a1, clusterResource, node_0, 1*GB);
     stubQueueAllocation(b3, clusterResource, node_0, 2*GB);
     stubQueueAllocation(c, clusterResource, node_0, 2*GB);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     InOrder allocationOrder = inOrder(a, c, b);
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(c).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(a, 1*GB, clusterResource);
     verifyQueueMetrics(b, 6*GB, clusterResource);
     verifyQueueMetrics(c, 3*GB, clusterResource);
@@ -501,17 +516,17 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     stubQueueAllocation(b3, clusterResource, node_2, 1*GB);
     stubQueueAllocation(b1, clusterResource, node_2, 1*GB);
     stubQueueAllocation(c, clusterResource, node_2, 1*GB);
-    root.assignContainers(clusterResource, node_2, false, 
+    root.assignContainers(clusterResource, node_2, 
         new ResourceLimits(clusterResource));
     allocationOrder = inOrder(a, a2, a1, b, c);
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(a2).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(c).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(a, 3*GB, clusterResource);
     verifyQueueMetrics(b, 8*GB, clusterResource);
     verifyQueueMetrics(c, 4*GB, clusterResource);
@@ -611,7 +626,7 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // Simulate B returning a container on node_0
     stubQueueAllocation(a, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
     stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     verifyQueueMetrics(a, 0*GB, clusterResource);
     verifyQueueMetrics(b, 1*GB, clusterResource);
@@ -620,13 +635,13 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // also, B gets a scheduling opportunity since A allocates RACK_LOCAL
     stubQueueAllocation(a, clusterResource, node_1, 2*GB, NodeType.RACK_LOCAL);
     stubQueueAllocation(b, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
-    root.assignContainers(clusterResource, node_1, false, 
+    root.assignContainers(clusterResource, node_1, 
         new ResourceLimits(clusterResource));
     InOrder allocationOrder = inOrder(a, b);
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(a, 2*GB, clusterResource);
     verifyQueueMetrics(b, 2*GB, clusterResource);
     
@@ -635,13 +650,13 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // However, since B returns off-switch, A won't get an opportunity
     stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
     stubQueueAllocation(b, clusterResource, node_0, 2*GB, NodeType.OFF_SWITCH);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     allocationOrder = inOrder(b, a);
     allocationOrder.verify(b).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(a).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(a, 2*GB, clusterResource);
     verifyQueueMetrics(b, 4*GB, clusterResource);
 
@@ -680,7 +695,7 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // Simulate B3 returning a container on node_0
     stubQueueAllocation(b2, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
     stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     verifyQueueMetrics(b2, 0*GB, clusterResource);
     verifyQueueMetrics(b3, 1*GB, clusterResource);
@@ -689,13 +704,13 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // also, B3 gets a scheduling opportunity since B2 allocates RACK_LOCAL
     stubQueueAllocation(b2, clusterResource, node_1, 1*GB, NodeType.RACK_LOCAL);
     stubQueueAllocation(b3, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
-    root.assignContainers(clusterResource, node_1, false, 
+    root.assignContainers(clusterResource, node_1, 
         new ResourceLimits(clusterResource));
     InOrder allocationOrder = inOrder(b2, b3);
     allocationOrder.verify(b2).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(b3).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(b2, 1*GB, clusterResource);
     verifyQueueMetrics(b3, 2*GB, clusterResource);
     
@@ -704,13 +719,13 @@ when(queue).assignContainers(eq(clusterResource), eq(node), eq(false),
     // However, since B3 returns off-switch, B2 won't get an opportunity
     stubQueueAllocation(b2, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
     stubQueueAllocation(b3, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
-    root.assignContainers(clusterResource, node_0, false, 
+    root.assignContainers(clusterResource, node_0, 
         new ResourceLimits(clusterResource));
     allocationOrder = inOrder(b3, b2);
     allocationOrder.verify(b3).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     allocationOrder.verify(b2).assignContainers(eq(clusterResource), 
-        any(FiCaSchedulerNode.class), anyBoolean(), anyResourceLimits());
+        any(FiCaSchedulerNode.class), anyResourceLimits());
     verifyQueueMetrics(b2, 1*GB, clusterResource);
     verifyQueueMetrics(b3, 3*GB, clusterResource);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/487374b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.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/TestReservations.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/TestReservations.java
index b3250e5..c5b7587 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/TestReservations.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/TestReservations.java
@@ -265,7 +265,7 @@ public class TestReservations {
 
     // Start testing...
     // Only AM
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
@@ -277,7 +277,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
     // Only 1 map - simulating reduce
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
@@ -289,7 +289,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
     // Only 1 map to other node - simulating reduce
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -304,7 +304,7 @@ public class TestReservations {
     assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
 
     // try to assign reducer (5G on node 0 and should reserve)
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -320,7 +320,7 @@ public class TestReservations {
     assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
 
     // assign reducer to node 2
-    a.assignContainers(clusterResource, node_2, false,
+    a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     assertEquals(18 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
@@ -337,7 +337,7 @@ public class TestReservations {
 
     // node_1 heartbeat and unreserves from node_0 in order to allocate
     // on node_1
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(18 * GB, a.getUsedResources().getMemory());
     assertEquals(18 * GB, app_0.getCurrentConsumption().getMemory());
@@ -421,7 +421,7 @@ public class TestReservations {
 
     // Start testing...
     // Only AM
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
@@ -433,7 +433,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
     // Only 1 map - simulating reduce
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
@@ -445,7 +445,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
     // Only 1 map to other node - simulating reduce
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -460,7 +460,7 @@ public class TestReservations {
     assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
 
     // try to assign reducer (5G on node 0 and should reserve)
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -476,7 +476,7 @@ public class TestReservations {
     assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
 
     // assign reducer to node 2
-    a.assignContainers(clusterResource, node_2, false,
+    a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     assertEquals(18 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
@@ -493,7 +493,7 @@ public class TestReservations {
 
     // node_1 heartbeat and won't unreserve from node_0, potentially stuck
     // if AM doesn't handle
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(18 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
@@ -569,7 +569,7 @@ public class TestReservations {
 
     // Start testing...
     // Only AM
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
@@ -580,7 +580,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_1.getUsedResource().getMemory());
 
     // Only 1 map - simulating reduce
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
@@ -591,7 +591,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_1.getUsedResource().getMemory());
 
     // Only 1 map to other node - simulating reduce
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -605,7 +605,7 @@ public class TestReservations {
     assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
 
     // try to assign reducer (5G on node 0 and should reserve)
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -620,7 +620,7 @@ public class TestReservations {
     assertEquals(2, app_0.getTotalRequiredResources(priorityReduce));
 
     // could allocate but told need to unreserve first
-    a.assignContainers(clusterResource, node_1, true,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
@@ -652,6 +652,8 @@ public class TestReservations {
     String host_1 = "host_1";
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0,
         8 * GB);
+    
+    Resource clusterResource = Resources.createResource(2 * 8 * GB);
 
     // Setup resource-requests
     Priority priorityMap = TestUtils.createMockPriority(5);
@@ -681,23 +683,28 @@ public class TestReservations {
         node_0.getNodeID(), "user", rmContext);
 
     // no reserved containers
-    NodeId unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability);
+    NodeId unreserveId =
+        app_0.getNodeIdToUnreserve(priorityMap, capability,
+            cs.getResourceCalculator(), clusterResource);
     assertEquals(null, unreserveId);
 
     // no reserved containers - reserve then unreserve
     app_0.reserve(node_0, priorityMap, rmContainer_1, container_1);
     app_0.unreserve(node_0, priorityMap);
-    unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability);
+    unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
+        cs.getResourceCalculator(), clusterResource);
     assertEquals(null, unreserveId);
 
     // no container large enough is reserved
     app_0.reserve(node_0, priorityMap, rmContainer_1, container_1);
-    unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability);
+    unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
+        cs.getResourceCalculator(), clusterResource);
     assertEquals(null, unreserveId);
 
     // reserve one that is now large enough
     app_0.reserve(node_1, priorityMap, rmContainer, container);
-    unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability);
+    unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
+        cs.getResourceCalculator(), clusterResource);
     assertEquals(node_1.getNodeID(), unreserveId);
   }
 
@@ -741,14 +748,14 @@ public class TestReservations {
 
     // nothing reserved
     boolean res = a.findNodeToUnreserve(csContext.getClusterResource(),
-        node_1, app_0, priorityMap, capability);
+        node_1, app_0, priorityMap, capability, capability);
     assertFalse(res);
 
     // reserved but scheduler doesn't know about that node.
     app_0.reserve(node_1, priorityMap, rmContainer, container);
     node_1.reserveResource(app_0, priorityMap, rmContainer);
     res = a.findNodeToUnreserve(csContext.getClusterResource(), node_1, app_0,
-        priorityMap, capability);
+        priorityMap, capability, capability);
     assertFalse(res);
   }
 
@@ -815,7 +822,7 @@ public class TestReservations {
 
     // Start testing...
     // Only AM
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
@@ -826,7 +833,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_1.getUsedResource().getMemory());
 
     // Only 1 map - simulating reduce
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
@@ -837,7 +844,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_1.getUsedResource().getMemory());
 
     // Only 1 map to other node - simulating reduce
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -852,14 +859,15 @@ public class TestReservations {
     // absoluteMaxCapacity
     Resource capability = Resources.createResource(32 * GB, 0);
     boolean res =
-        a.canAssignToThisQueue(clusterResource, capability,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
+        a.canAssignToThisQueue(clusterResource,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
+                clusterResource), capability, Resources.none());
     assertFalse(res);
 
     // now add in reservations and make sure it continues if config set
     // allocate to queue so that the potential new capacity is greater then
     // absoluteMaxCapacity
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -872,14 +880,17 @@ public class TestReservations {
 
     capability = Resources.createResource(5 * GB, 0);
     res =
-        a.canAssignToThisQueue(clusterResource, capability,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
+        a.canAssignToThisQueue(clusterResource,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
+                clusterResource), capability, Resources
+                .createResource(5 * GB));
     assertTrue(res);
 
     // tell to not check reservations
     res =
-        a.canAssignToThisQueue(clusterResource, capability,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, false);
+        a.canAssignToThisQueue(clusterResource,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
+                clusterResource), capability, Resources.none());
     assertFalse(res);
 
     refreshQueuesTurnOffReservationsContLook(a, csConf);
@@ -887,13 +898,16 @@ public class TestReservations {
     // should return false no matter what checkReservations is passed
     // in since feature is off
     res =
-        a.canAssignToThisQueue(clusterResource, capability,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, false);
+        a.canAssignToThisQueue(clusterResource,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
+                clusterResource), capability, Resources.none());
     assertFalse(res);
 
     res =
-        a.canAssignToThisQueue(clusterResource, capability,
-            CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
+        a.canAssignToThisQueue(clusterResource,
+            CommonNodeLabelsManager.EMPTY_STRING_SET, new ResourceLimits(
+                clusterResource), capability, Resources
+                .createResource(5 * GB));
     assertFalse(res);
   }
 
@@ -985,15 +999,15 @@ public class TestReservations {
         .createResourceRequest(ResourceRequest.ANY, 2 * GB, 1, true,
             priorityAM, recordFactory)));
     app_0.updateResourceRequests(Collections.singletonList(TestUtils
-        .createResourceRequest(ResourceRequest.ANY, 5 * GB, 2, true,
-            priorityReduce, recordFactory)));
-    app_0.updateResourceRequests(Collections.singletonList(TestUtils
         .createResourceRequest(ResourceRequest.ANY, 3 * GB, 2, true,
             priorityMap, recordFactory)));
+    app_0.updateResourceRequests(Collections.singletonList(TestUtils
+        .createResourceRequest(ResourceRequest.ANY, 5 * GB, 2, true,
+            priorityReduce, recordFactory)));
 
     // Start testing...
     // Only AM
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1004,7 +1018,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_1.getUsedResource().getMemory());
 
     // Only 1 map - simulating reduce
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1015,7 +1029,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_1.getUsedResource().getMemory());
 
     // Only 1 map to other node - simulating reduce
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1029,7 +1043,7 @@ public class TestReservations {
     // now add in reservations and make sure it continues if config set
     // allocate to queue so that the potential new capacity is greater then
     // absoluteMaxCapacity
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1117,18 +1131,18 @@ public class TestReservations {
         .createResourceRequest(ResourceRequest.ANY, 2 * GB, 1, true,
             priorityAM, recordFactory)));
     app_0.updateResourceRequests(Collections.singletonList(TestUtils
-        .createResourceRequest(ResourceRequest.ANY, 5 * GB, 1, true,
-            priorityReduce, recordFactory)));
-    app_0.updateResourceRequests(Collections.singletonList(TestUtils
         .createResourceRequest(ResourceRequest.ANY, 3 * GB, 2, true,
             priorityMap, recordFactory)));
     app_0.updateResourceRequests(Collections.singletonList(TestUtils
+        .createResourceRequest(ResourceRequest.ANY, 5 * GB, 1, true,
+            priorityReduce, recordFactory)));
+    app_0.updateResourceRequests(Collections.singletonList(TestUtils
         .createResourceRequest(ResourceRequest.ANY, 8 * GB, 2, true,
             priorityLast, recordFactory)));
 
     // Start testing...
     // Only AM
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(2 * GB, a.getUsedResources().getMemory());
     assertEquals(2 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1140,7 +1154,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
     // Only 1 map - simulating reduce
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(5 * GB, a.getUsedResources().getMemory());
     assertEquals(5 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1152,7 +1166,7 @@ public class TestReservations {
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
     // Only 1 map to other node - simulating reduce
-    a.assignContainers(clusterResource, node_1, false,
+    a.assignContainers(clusterResource, node_1,
         new ResourceLimits(clusterResource));
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1164,38 +1178,41 @@ public class TestReservations {
     assertEquals(3 * GB, node_1.getUsedResource().getMemory());
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
-    // try to assign reducer (5G on node 0), but tell it
-    // it has to unreserve. No room to allocate and shouldn't reserve
-    // since nothing currently reserved.
-    a.assignContainers(clusterResource, node_0, true,
-        new ResourceLimits(clusterResource));
+    // try to assign reducer (5G on node 0), but tell it's resource limits <
+    // used (8G) + required (5G). It will not reserved since it has to unreserve
+    // some resource. Even with continous reservation looking, we don't allow 
+    // unreserve resource to reserve container.
+    a.assignContainers(clusterResource, node_0,
+        new ResourceLimits(Resources.createResource(10 * GB)));
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
     assertEquals(8 * GB, a.getMetrics().getAllocatedMB());
     assertEquals(16 * GB, a.getMetrics().getAvailableMB());
-    assertEquals(16 * GB, app_0.getHeadroom().getMemory());
+    // app_0's headroom = limit (10G) - used (8G) = 2G 
+    assertEquals(2 * GB, app_0.getHeadroom().getMemory());
     assertEquals(5 * GB, node_0.getUsedResource().getMemory());
     assertEquals(3 * GB, node_1.getUsedResource().getMemory());
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
-    // try to assign reducer (5G on node 2), but tell it
-    // it has to unreserve. Has room but shouldn't reserve
-    // since nothing currently reserved.
-    a.assignContainers(clusterResource, node_2, true,
-        new ResourceLimits(clusterResource));
+    // try to assign reducer (5G on node 0), but tell it's resource limits <
+    // used (8G) + required (5G). It will not reserved since it has to unreserve
+    // some resource. Unfortunately, there's nothing to unreserve.
+    a.assignContainers(clusterResource, node_2,
+        new ResourceLimits(Resources.createResource(10 * GB)));
     assertEquals(8 * GB, a.getUsedResources().getMemory());
     assertEquals(8 * GB, app_0.getCurrentConsumption().getMemory());
     assertEquals(0 * GB, a.getMetrics().getReservedMB());
     assertEquals(8 * GB, a.getMetrics().getAllocatedMB());
     assertEquals(16 * GB, a.getMetrics().getAvailableMB());
-    assertEquals(16 * GB, app_0.getHeadroom().getMemory());
+    // app_0's headroom = limit (10G) - used (8G) = 2G 
+    assertEquals(2 * GB, app_0.getHeadroom().getMemory());
     assertEquals(5 * GB, node_0.getUsedResource().getMemory());
     assertEquals(3 * GB, node_1.getUsedResource().getMemory());
     assertEquals(0 * GB, node_2.getUsedResource().getMemory());
 
     // let it assign 5G to node_2
-    a.assignContainers(clusterResource, node_2, false,
+    a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     assertEquals(13 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1208,7 +1225,7 @@ public class TestReservations {
     assertEquals(5 * GB, node_2.getUsedResource().getMemory());
 
     // reserve 8G node_0
-    a.assignContainers(clusterResource, node_0, false,
+    a.assignContainers(clusterResource, node_0,
         new ResourceLimits(clusterResource));
     assertEquals(21 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());
@@ -1223,7 +1240,7 @@ public class TestReservations {
     // try to assign (8G on node 2). No room to allocate,
     // continued to try due to having reservation above,
     // but hits queue limits so can't reserve anymore.
-    a.assignContainers(clusterResource, node_2, false,
+    a.assignContainers(clusterResource, node_2,
         new ResourceLimits(clusterResource));
     assertEquals(21 * GB, a.getUsedResources().getMemory());
     assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory());


[49/50] hadoop git commit: YARN-3039. Implemented the app-level timeline aggregator discovery service. Contributed by Junping Du.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewAggregatorsInfoRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewAggregatorsInfoRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewAggregatorsInfoRequestPBImpl.java
new file mode 100644
index 0000000..eb7beef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewAggregatorsInfoRequestPBImpl.java
@@ -0,0 +1,142 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
+import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
+import org.apache.hadoop.yarn.server.api.records.impl.pb.AppAggregatorsMapPBImpl;
+
+public class ReportNewAggregatorsInfoRequestPBImpl extends
+    ReportNewAggregatorsInfoRequest {
+
+  ReportNewAggregatorsInfoRequestProto proto = 
+      ReportNewAggregatorsInfoRequestProto.getDefaultInstance();
+  
+  ReportNewAggregatorsInfoRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private List<AppAggregatorsMap> aggregatorsList = null;
+
+  public ReportNewAggregatorsInfoRequestPBImpl() {
+    builder = ReportNewAggregatorsInfoRequestProto.newBuilder();
+  }
+
+  public ReportNewAggregatorsInfoRequestPBImpl(
+      ReportNewAggregatorsInfoRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReportNewAggregatorsInfoRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+  
+  @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;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (aggregatorsList != null) {
+      addLocalAggregatorsToProto();
+    }
+  }
+  
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReportNewAggregatorsInfoRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void addLocalAggregatorsToProto() {
+    maybeInitBuilder();
+    builder.clearAppAggregators();
+    List<AppAggregatorsMapProto> protoList =
+        new ArrayList<AppAggregatorsMapProto>();
+    for (AppAggregatorsMap m : this.aggregatorsList) {
+      protoList.add(convertToProtoFormat(m));
+    }
+    builder.addAllAppAggregators(protoList);
+  }
+
+  private void initLocalAggregatorsList() {
+    ReportNewAggregatorsInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<AppAggregatorsMapProto> aggregatorsList =
+        p.getAppAggregatorsList();
+    this.aggregatorsList = new ArrayList<AppAggregatorsMap>();
+    for (AppAggregatorsMapProto m : aggregatorsList) {
+      this.aggregatorsList.add(convertFromProtoFormat(m));
+    }
+  }
+
+  @Override
+  public List<AppAggregatorsMap> getAppAggregatorsList() {  
+    if (this.aggregatorsList == null) {
+      initLocalAggregatorsList();
+    }
+    return this.aggregatorsList;
+  }
+
+  @Override
+  public void setAppAggregatorsList(List<AppAggregatorsMap> appAggregatorsList) {
+    maybeInitBuilder();
+    if (appAggregatorsList == null) {
+      builder.clearAppAggregators();
+    }
+    this.aggregatorsList = appAggregatorsList;
+  }
+  
+  private AppAggregatorsMapPBImpl convertFromProtoFormat(
+      AppAggregatorsMapProto p) {
+    return new AppAggregatorsMapPBImpl(p);
+  }
+
+  private AppAggregatorsMapProto convertToProtoFormat(
+      AppAggregatorsMap m) {
+    return ((AppAggregatorsMapPBImpl) m).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewAggregatorsInfoResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewAggregatorsInfoResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewAggregatorsInfoResponsePBImpl.java
new file mode 100644
index 0000000..0f0925a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReportNewAggregatorsInfoResponsePBImpl.java
@@ -0,0 +1,74 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.ReportNewAggregatorsInfoResponseProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
+
+import com.google.protobuf.TextFormat;
+
+@Private
+@Unstable
+public class ReportNewAggregatorsInfoResponsePBImpl extends
+    ReportNewAggregatorsInfoResponse {
+
+  ReportNewAggregatorsInfoResponseProto proto = 
+      ReportNewAggregatorsInfoResponseProto.getDefaultInstance();
+  
+  ReportNewAggregatorsInfoResponseProto.Builder builder = null;
+  
+  boolean viaProto = false;
+  
+  public ReportNewAggregatorsInfoResponsePBImpl() {
+    builder = ReportNewAggregatorsInfoResponseProto.newBuilder();
+  }
+
+  public ReportNewAggregatorsInfoResponsePBImpl(ReportNewAggregatorsInfoResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+  
+  public ReportNewAggregatorsInfoResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @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/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/AppAggregatorsMap.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/AppAggregatorsMap.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/AppAggregatorsMap.java
new file mode 100644
index 0000000..67c377d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/AppAggregatorsMap.java
@@ -0,0 +1,33 @@
+package org.apache.hadoop.yarn.server.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.Records;
+
+
+@Private
+public abstract class AppAggregatorsMap {
+
+  public static AppAggregatorsMap newInstance(
+      ApplicationId id, String aggregatorAddr) {
+    AppAggregatorsMap appAggregatorMap =
+        Records.newRecord(AppAggregatorsMap.class);
+    appAggregatorMap.setApplicationId(id);
+    appAggregatorMap.setAggregatorAddr(aggregatorAddr);
+    return appAggregatorMap;
+  }
+  
+  public abstract ApplicationId getApplicationId();
+  
+  public abstract void setApplicationId(
+      ApplicationId id);
+  
+  public abstract String getAggregatorAddr();
+  
+  public abstract void setAggregatorAddr(
+      String addr);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppAggregatorsMapPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppAggregatorsMapPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppAggregatorsMapPBImpl.java
new file mode 100644
index 0000000..32903e2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/AppAggregatorsMapPBImpl.java
@@ -0,0 +1,151 @@
+/**
+* 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.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.ApplicationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
+
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppAggregatorsMapProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+@Private
+@Unstable
+public class AppAggregatorsMapPBImpl extends AppAggregatorsMap {
+
+  AppAggregatorsMapProto proto = 
+      AppAggregatorsMapProto.getDefaultInstance();
+  
+  AppAggregatorsMapProto.Builder builder = null;
+  boolean viaProto = false;
+  
+  private ApplicationId appId = null;
+  private String aggregatorAddr = null;
+  
+  public AppAggregatorsMapPBImpl() {
+    builder = AppAggregatorsMapProto.newBuilder();
+  }
+
+  public AppAggregatorsMapPBImpl(AppAggregatorsMapProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+  
+  public AppAggregatorsMapProto getProto() {
+      mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @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());
+  }
+  
+  @Override
+  public ApplicationId getApplicationId() {
+    AppAggregatorsMapProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.appId == null && p.hasAppId()) {
+      this.appId = convertFromProtoFormat(p.getAppId());
+    }
+    return this.appId;
+  }
+  
+  @Override
+  public String getAggregatorAddr() {
+    AppAggregatorsMapProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.aggregatorAddr == null 
+        && p.hasAppAggregatorAddr()) {
+      this.aggregatorAddr = p.getAppAggregatorAddr();
+    }
+    return this.aggregatorAddr;
+  }
+
+  @Override
+  public void setApplicationId(ApplicationId appId) {
+    maybeInitBuilder();
+    if (appId == null) {
+      builder.clearAppId();
+    }
+    this.appId = appId;
+  }
+  
+  @Override
+  public void setAggregatorAddr(String aggregatorAddr) {
+    maybeInitBuilder();
+    if (aggregatorAddr == null) {
+      builder.clearAppAggregatorAddr();
+    }
+    this.aggregatorAddr = aggregatorAddr;
+  }
+  
+  private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
+    return new ApplicationIdPBImpl(p);
+  }
+  
+  private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
+    return ((ApplicationIdPBImpl) t).getProto();
+  }
+  
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = AppAggregatorsMapProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+  
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+  
+  private void mergeLocalToBuilder() {
+    if (this.appId != null) {
+      builder.setAppId(convertToProtoFormat(this.appId));
+    }
+    if (this.aggregatorAddr != null) {
+      builder.setAppAggregatorAddr(this.aggregatorAddr);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/aggregatornodemanager_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/aggregatornodemanager_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/aggregatornodemanager_protocol.proto
new file mode 100644
index 0000000..d7b05c1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/aggregatornodemanager_protocol.proto
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.yarn.proto";
+option java_outer_classname = "AggregatorNodemanagerProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.yarn;
+
+import "yarn_server_common_service_protos.proto";
+
+service AggregatorNodemanagerProtocolService {
+  rpc reportNewAggregatorInfo (ReportNewAggregatorsInfoRequestProto) returns (ReportNewAggregatorsInfoResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index 91473c5..3b03f58 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -47,6 +47,7 @@ message NodeHeartbeatRequestProto {
   optional NodeStatusProto node_status = 1;
   optional MasterKeyProto last_known_container_token_master_key = 2;
   optional MasterKeyProto last_known_nm_token_master_key = 3;
+  repeated AppAggregatorsMapProto registered_aggregators = 4;
 }
 
 message NodeHeartbeatResponseProto {
@@ -60,6 +61,7 @@ message NodeHeartbeatResponseProto {
   optional string diagnostics_message = 8;
   repeated ContainerIdProto containers_to_be_removed_from_nm = 9;
   repeated SystemCredentialsForAppsProto system_credentials_for_apps = 10;
+  repeated AppAggregatorsMapProto app_aggregators_map = 11;
 }
 
 message SystemCredentialsForAppsProto {
@@ -67,6 +69,25 @@ message SystemCredentialsForAppsProto {
   optional bytes credentialsForApp = 2;
 }
 
+////////////////////////////////////////////////////////////////////////
+////// From aggregator_nodemanager_protocol ////////////////////////////
+////////////////////////////////////////////////////////////////////////
+message AppAggregatorsMapProto {
+  optional ApplicationIdProto appId = 1;
+  optional string appAggregatorAddr = 2;
+}
+
+//////////////////////////////////////////////////////
+/////// aggregator_nodemanager_protocol //////////////
+//////////////////////////////////////////////////////
+message ReportNewAggregatorsInfoRequestProto {
+  repeated AppAggregatorsMapProto app_aggregators = 1;
+}
+
+message ReportNewAggregatorsInfoResponseProto {
+}
+
+
 message NMContainerStatusProto {
   optional ContainerIdProto container_id = 1;
   optional ContainerStateProto container_state = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
new file mode 100644
index 0000000..af9d60f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
@@ -0,0 +1,345 @@
+/**
+* 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;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
+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.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.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.ContainerLaunchContext;
+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.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.exceptions.YarnException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
+import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRPC {
+
+  private static final String EXCEPTION_MSG = "test error";
+  private static final String EXCEPTION_CAUSE = "exception cause";
+  private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+  
+  public static final String ILLEGAL_NUMBER_MESSAGE = 
+      "aggregators' number in ReportNewAggregatorsInfoRequest is not ONE.";
+  
+  public static final String DEFAULT_AGGREGATOR_ADDR = "localhost:0";
+  
+  public static final ApplicationId DEFAULT_APP_ID = 
+      ApplicationId.newInstance(0, 0);
+  
+  @Test
+  public void testUnknownCall() {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class
+        .getName());
+    YarnRPC rpc = YarnRPC.create(conf);
+    String bindAddr = "localhost:0";
+    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+    Server server = rpc.getServer(ContainerManagementProtocol.class,
+        new DummyContainerManager(), addr, conf, null, 1);
+    server.start();
+
+    // Any unrelated protocol would do
+    ApplicationClientProtocol proxy = (ApplicationClientProtocol) rpc.getProxy(
+        ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), conf);
+
+    try {
+      proxy.getNewApplication(Records
+          .newRecord(GetNewApplicationRequest.class));
+      Assert.fail("Excepted RPC call to fail with unknown method.");
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().matches(
+          "Unknown method getNewApplication called on.*"
+              + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol"
+              + "\\$ApplicationClientProtocolService\\$BlockingInterface protocol."));
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      server.stop();
+    }
+  }
+  
+  @Test
+  public void testRPCOnAggregatorNodeManagerProtocol() throws IOException {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class
+        .getName());
+    YarnRPC rpc = YarnRPC.create(conf);
+    String bindAddr = "localhost:0";
+    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+    Server server = rpc.getServer(AggregatorNodemanagerProtocol.class,
+        new DummyNMAggregatorService(), addr, conf, null, 1);
+    server.start();
+
+    // Test unrelated protocol wouldn't get response
+    ApplicationClientProtocol unknownProxy = (ApplicationClientProtocol) rpc.getProxy(
+        ApplicationClientProtocol.class, NetUtils.getConnectAddress(server), conf);
+
+    try {
+      unknownProxy.getNewApplication(Records
+          .newRecord(GetNewApplicationRequest.class));
+      Assert.fail("Excepted RPC call to fail with unknown method.");
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().matches(
+          "Unknown method getNewApplication called on.*"
+              + "org.apache.hadoop.yarn.proto.ApplicationClientProtocol"
+              + "\\$ApplicationClientProtocolService\\$BlockingInterface protocol."));
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+    
+    // Test AggregatorNodemanagerProtocol get proper response
+    AggregatorNodemanagerProtocol proxy = (AggregatorNodemanagerProtocol)rpc.getProxy(
+        AggregatorNodemanagerProtocol.class, NetUtils.getConnectAddress(server), conf);
+    // Verify request with DEFAULT_APP_ID and DEFAULT_AGGREGATOR_ADDR get 
+    // normally response.
+    try {
+      ReportNewAggregatorsInfoRequest request = 
+          ReportNewAggregatorsInfoRequest.newInstance(
+              DEFAULT_APP_ID, DEFAULT_AGGREGATOR_ADDR);
+      proxy.reportNewAggregatorInfo(request);
+    } catch (YarnException e) {
+      Assert.fail("RPC call failured is not expected here.");
+    }
+    
+    // Verify empty request get YarnException back (by design in 
+    // DummyNMAggregatorService)
+    try {
+      proxy.reportNewAggregatorInfo(Records
+          .newRecord(ReportNewAggregatorsInfoRequest.class));
+      Assert.fail("Excepted RPC call to fail with YarnException.");
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().contains(ILLEGAL_NUMBER_MESSAGE));
+    }
+    
+    server.stop();
+  }
+
+  @Test
+  public void testHadoopProtoRPC() throws Exception {
+    test(HadoopYarnProtoRPC.class.getName());
+  }
+  
+  private void test(String rpcClass) throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass);
+    YarnRPC rpc = YarnRPC.create(conf);
+    String bindAddr = "localhost:0";
+    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+    Server server = rpc.getServer(ContainerManagementProtocol.class, 
+            new DummyContainerManager(), addr, conf, null, 1);
+    server.start();
+    RPC.setProtocolEngine(conf, ContainerManagementProtocolPB.class, ProtobufRpcEngine.class);
+    ContainerManagementProtocol proxy = (ContainerManagementProtocol) 
+        rpc.getProxy(ContainerManagementProtocol.class, 
+            NetUtils.getConnectAddress(server), conf);
+    ContainerLaunchContext containerLaunchContext = 
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    ApplicationId applicationId = ApplicationId.newInstance(0, 0);
+    ApplicationAttemptId applicationAttemptId =
+        ApplicationAttemptId.newInstance(applicationId, 0);
+    ContainerId containerId =
+        ContainerId.newContainerId(applicationAttemptId, 100);
+    NodeId nodeId = NodeId.newInstance("localhost", 1234);
+    Resource resource = Resource.newInstance(1234, 2);
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(containerId, "localhost", "user",
+          resource, System.currentTimeMillis() + 10000, 42, 42,
+          Priority.newInstance(0), 0);
+    Token containerToken = newContainerToken(nodeId, "password".getBytes(),
+          containerTokenIdentifier);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          containerToken);
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    proxy.startContainers(allRequests);
+
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
+    GetContainerStatusesResponse response =
+        proxy.getContainerStatuses(gcsRequest);
+    List<ContainerStatus> statuses = response.getContainerStatuses();
+
+    //test remote exception
+    boolean exception = false;
+    try {
+      StopContainersRequest stopRequest =
+          recordFactory.newRecordInstance(StopContainersRequest.class);
+      stopRequest.setContainerIds(containerIds);
+      proxy.stopContainers(stopRequest);
+      } catch (YarnException e) {
+      exception = true;
+      Assert.assertTrue(e.getMessage().contains(EXCEPTION_MSG));
+      Assert.assertTrue(e.getMessage().contains(EXCEPTION_CAUSE));
+      System.out.println("Test Exception is " + e.getMessage());
+    } catch (Exception ex) {
+      ex.printStackTrace();
+    } finally {
+      server.stop();
+    }
+    Assert.assertTrue(exception);
+    Assert.assertNotNull(statuses.get(0));
+    Assert.assertEquals(ContainerState.RUNNING, statuses.get(0).getState());
+  }
+
+  public class DummyContainerManager implements ContainerManagementProtocol {
+
+    private List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
+
+    @Override
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request)
+    throws YarnException {
+      GetContainerStatusesResponse response = 
+          recordFactory.newRecordInstance(GetContainerStatusesResponse.class);
+      response.setContainerStatuses(statuses);
+      return response;
+    }
+
+    @Override
+    public StartContainersResponse startContainers(
+        StartContainersRequest requests) throws YarnException {
+      StartContainersResponse response =
+          recordFactory.newRecordInstance(StartContainersResponse.class);
+      for (StartContainerRequest request : requests.getStartContainerRequests()) {
+        Token containerToken = request.getContainerToken();
+        ContainerTokenIdentifier tokenId = null;
+
+        try {
+          tokenId = newContainerTokenIdentifier(containerToken);
+        } catch (IOException e) {
+          throw RPCUtil.getRemoteException(e);
+        }
+        ContainerStatus status =
+            recordFactory.newRecordInstance(ContainerStatus.class);
+        status.setState(ContainerState.RUNNING);
+        status.setContainerId(tokenId.getContainerID());
+        status.setExitStatus(0);
+        statuses.add(status);
+
+      }
+      return response;
+    }
+
+    @Override
+    public StopContainersResponse stopContainers(StopContainersRequest request) 
+    throws YarnException {
+      Exception e = new Exception(EXCEPTION_MSG, 
+          new Exception(EXCEPTION_CAUSE));
+      throw new YarnException(e);
+    }
+  }
+
+  public static ContainerTokenIdentifier newContainerTokenIdentifier(
+      Token containerToken) throws IOException {
+    org.apache.hadoop.security.token.Token<ContainerTokenIdentifier> token =
+        new org.apache.hadoop.security.token.Token<ContainerTokenIdentifier>(
+            containerToken.getIdentifier()
+                .array(), containerToken.getPassword().array(), new Text(
+                containerToken.getKind()),
+            new Text(containerToken.getService()));
+    return token.decodeIdentifier();
+  }
+
+  public static Token newContainerToken(NodeId nodeId, byte[] password,
+      ContainerTokenIdentifier tokenIdentifier) {
+    // RPC layer client expects ip:port as service for tokens
+    InetSocketAddress addr =
+        NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort());
+    // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token
+    Token containerToken =
+        Token.newInstance(tokenIdentifier.getBytes(),
+          ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil
+            .buildTokenService(addr).toString());
+    return containerToken;
+  }
+  
+  // A dummy implementation for AggregatorNodemanagerProtocol for test purpose, 
+  // it only can accept one appID, aggregatorAddr pair or throw exceptions
+  public class DummyNMAggregatorService 
+      implements AggregatorNodemanagerProtocol {
+    
+    @Override
+    public ReportNewAggregatorsInfoResponse reportNewAggregatorInfo(
+        ReportNewAggregatorsInfoRequest request)
+        throws YarnException, IOException {
+      List<AppAggregatorsMap> appAggregators = request.getAppAggregatorsList();
+      if (appAggregators.size() == 1) {
+        // check default appID and aggregatorAddr
+        AppAggregatorsMap appAggregator = appAggregators.get(0);
+        Assert.assertEquals(appAggregator.getApplicationId(), 
+            DEFAULT_APP_ID);
+        Assert.assertEquals(appAggregator.getAggregatorAddr(), 
+            DEFAULT_AGGREGATOR_ADDR);
+      } else {
+        throw new YarnException(ILLEGAL_NUMBER_MESSAGE);
+      }
+      
+      ReportNewAggregatorsInfoResponse response =
+          recordFactory.newRecordInstance(ReportNewAggregatorsInfoResponse.class);
+      return response;
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
index 20983b6..47cf8ad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
@@ -24,6 +24,8 @@ import static org.junit.Assert.assertTrue;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -89,11 +91,14 @@ public class TestYarnServerApiClasses {
     original.setLastKnownContainerTokenMasterKey(getMasterKey());
     original.setLastKnownNMTokenMasterKey(getMasterKey());
     original.setNodeStatus(getNodeStatus());
+    Map<ApplicationId, String> aggregators = getAggregators();
+    original.setRegisteredAggregators(aggregators);
     NodeHeartbeatRequestPBImpl copy = new NodeHeartbeatRequestPBImpl(
         original.getProto());
     assertEquals(1, copy.getLastKnownContainerTokenMasterKey().getKeyId());
     assertEquals(1, copy.getLastKnownNMTokenMasterKey().getKeyId());
     assertEquals("localhost", copy.getNodeStatus().getNodeId().getHost());
+    assertEquals(aggregators, copy.getRegisteredAggregators());
   }
 
   /**
@@ -110,6 +115,8 @@ public class TestYarnServerApiClasses {
     original.setNextHeartBeatInterval(1000);
     original.setNodeAction(NodeAction.NORMAL);
     original.setResponseId(100);
+    Map<ApplicationId, String> aggregators = getAggregators();
+    original.setAppAggregatorsMap(aggregators);
 
     NodeHeartbeatResponsePBImpl copy = new NodeHeartbeatResponsePBImpl(
         original.getProto());
@@ -119,6 +126,7 @@ public class TestYarnServerApiClasses {
     assertEquals(1, copy.getContainerTokenMasterKey().getKeyId());
     assertEquals(1, copy.getNMTokenMasterKey().getKeyId());
     assertEquals("testDiagnosticMessage", copy.getDiagnosticsMessage());
+    assertEquals(aggregators, copy.getAppAggregatorsMap());
   }
 
   /**
@@ -208,6 +216,15 @@ public class TestYarnServerApiClasses {
 
   }
 
+  private Map<ApplicationId, String> getAggregators() {
+    ApplicationId appID = ApplicationId.newInstance(1L, 1);
+    String aggregatorAddr = "localhost:0";
+    Map<ApplicationId, String> aggregatorMap = 
+        new HashMap<ApplicationId, String>();
+    aggregatorMap.put(appID, aggregatorAddr);
+    return aggregatorMap;
+  }
+  
   private ContainerStatus getContainerStatus(int applicationId,
       int containerID, int appAttemptId) {
     ContainerStatus status = recordFactory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.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/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
index 6e7e2ec..85f3f0d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
@@ -57,6 +57,19 @@ public interface Context {
   ConcurrentMap<ApplicationId, Application> getApplications();
 
   Map<ApplicationId, Credentials> getSystemCredentialsForApps();
+  
+  /**
+   * Get the registered aggregators that located on this NM. 
+   * @return registered
+   */
+  Map<ApplicationId, String> getRegisteredAggregators();
+  
+  /**
+   * Return the known aggregators which get from RM for all active applications
+   * running on this NM.
+   * @return known aggregators.
+   */
+  Map<ApplicationId, String> getKnownAggregators();
 
   ConcurrentMap<ContainerId, Container> getContainers();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index a4be120..10143db 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
+import org.apache.hadoop.yarn.server.nodemanager.aggregatormanager.NMAggregatorService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
@@ -84,8 +85,9 @@ public class NodeManager extends CompositeService
   private Context context;
   private AsyncDispatcher dispatcher;
   private ContainerManagerImpl containerManager;
+  private NMAggregatorService nmAggregatorService;
   private NodeStatusUpdater nodeStatusUpdater;
-  private static CompositeServiceShutdownHook nodeManagerShutdownHook; 
+  private static CompositeServiceShutdownHook nodeManagerShutdownHook;
   private NMStateStoreService nmStore = null;
   
   private AtomicBoolean isStopping = new AtomicBoolean(false);
@@ -112,6 +114,10 @@ public class NodeManager extends CompositeService
     return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
       metrics, aclsManager, dirsHandler);
   }
+  
+  protected NMAggregatorService createNMAggregatorService(Context context) {
+    return new NMAggregatorService(context);
+  }
 
   protected WebServer createWebServer(Context nmContext,
       ResourceView resourceView, ApplicationACLsManager aclsManager,
@@ -268,6 +274,9 @@ public class NodeManager extends CompositeService
     addService(dispatcher);
     
     DefaultMetricsSystem.initialize("NodeManager");
+    
+    this.nmAggregatorService = createNMAggregatorService(context);
+    addService(nmAggregatorService);
 
     // StatusUpdater should be added last so that it get started last 
     // so that we make sure everything is up before registering with RM. 
@@ -345,6 +354,12 @@ public class NodeManager extends CompositeService
 
     protected final ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentSkipListMap<ContainerId, Container>();
+    
+    protected Map<ApplicationId, String> registeredAggregators =
+        new ConcurrentHashMap<ApplicationId, String>();
+    
+    protected Map<ApplicationId, String> knownAggregators =
+        new ConcurrentHashMap<ApplicationId, String>();
 
     private final NMContainerTokenSecretManager containerTokenSecretManager;
     private final NMTokenSecretManagerInNM nmTokenSecretManager;
@@ -460,6 +475,30 @@ public class NodeManager extends CompositeService
         Map<ApplicationId, Credentials> systemCredentials) {
       this.systemCredentials = systemCredentials;
     }
+    
+    @Override
+    public Map<ApplicationId, String> getRegisteredAggregators() {
+      return this.registeredAggregators;
+    }
+
+    public void addRegisteredAggregators(
+        Map<ApplicationId, String> newRegisteredAggregators) {
+      this.registeredAggregators.putAll(newRegisteredAggregators);
+      // Update to knownAggregators as well so it can immediately be consumed by 
+      // this NM's TimelineClient.
+      this.knownAggregators.putAll(newRegisteredAggregators);
+    }
+    
+    @Override
+    public Map<ApplicationId, String> getKnownAggregators() {
+      return this.knownAggregators;
+    }
+
+    public void addKnownAggregators(
+        Map<ApplicationId, String> knownAggregators) {
+      this.knownAggregators.putAll(knownAggregators);
+    }
+    
   }
 
 
@@ -523,6 +562,11 @@ public class NodeManager extends CompositeService
   public Context getNMContext() {
     return this.context;
   }
+  
+  // For testing
+  NMAggregatorService getNMAggregatorService() {
+    return this.nmAggregatorService;
+  }
 
   public static void main(String[] args) throws IOException {
     Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.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/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 6ddd7e4..c855833 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -592,7 +592,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                   NodeStatusUpdaterImpl.this.context
                     .getContainerTokenSecretManager().getCurrentKey(),
                   NodeStatusUpdaterImpl.this.context.getNMTokenSecretManager()
-                    .getCurrentKey());
+                    .getCurrentKey(),
+                  NodeStatusUpdaterImpl.this.context.getRegisteredAggregators());
             response = resourceTracker.nodeHeartbeat(request);
             //get next heartbeat interval from response
             nextHeartBeatInterval = response.getNextHeartBeatInterval();
@@ -655,6 +656,10 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               ((NMContext) context)
                 .setSystemCrendentialsForApps(parseCredentials(systemCredentials));
             }
+            
+            Map<ApplicationId, String> knownAggregators = response.getAppAggregatorsMap();
+            ((NodeManager.NMContext)context).addKnownAggregators(knownAggregators);
+            
           } catch (ConnectException e) {
             //catch and throw the exception if tried MAX wait time to connect RM
             dispatcher.getEventHandler().handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/aggregatormanager/NMAggregatorService.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/aggregatormanager/NMAggregatorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/aggregatormanager/NMAggregatorService.java
new file mode 100644
index 0000000..17150ba
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/aggregatormanager/NMAggregatorService.java
@@ -0,0 +1,113 @@
+/**
+ * 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.aggregatormanager;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.CompositeService;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
+import org.apache.hadoop.yarn.server.api.records.AppAggregatorsMap;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoResponse;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
+
+public class NMAggregatorService extends CompositeService implements 
+    AggregatorNodemanagerProtocol {
+
+  private static final Log LOG = LogFactory.getLog(NMAggregatorService.class);
+
+  final Context context;
+  
+  private Server server;
+
+  public NMAggregatorService(Context context) {
+    
+    super(NMAggregatorService.class.getName());
+    this.context = context;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    Configuration conf = getConfig();
+    
+    InetSocketAddress aggregatorServerAddress = conf.getSocketAddr(
+        YarnConfiguration.NM_BIND_HOST,
+        YarnConfiguration.NM_AGGREGATOR_SERVICE_ADDRESS,
+        YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_ADDRESS,
+        YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_PORT);
+
+    Configuration serverConf = new Configuration(conf);
+
+    // TODO Security settings.
+    YarnRPC rpc = YarnRPC.create(conf);
+
+    server =
+        rpc.getServer(AggregatorNodemanagerProtocol.class, this, 
+            aggregatorServerAddress, serverConf,
+            this.context.getNMTokenSecretManager(),
+            conf.getInt(YarnConfiguration.NM_AGGREGATOR_SERVICE_THREAD_COUNT, 
+                YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_THREAD_COUNT));
+
+    server.start();
+    // start remaining services
+    super.serviceStart();
+    LOG.info("NMAggregatorService started at " + aggregatorServerAddress);
+  }
+
+  
+  @Override
+  public void serviceStop() throws Exception {
+    if (server != null) {
+      server.stop();
+    }
+    // TODO may cleanup app aggregators running on this NM in future.
+    super.serviceStop();
+  }
+
+  @Override
+  public ReportNewAggregatorsInfoResponse reportNewAggregatorInfo(
+      ReportNewAggregatorsInfoRequest request) throws IOException {
+    List<AppAggregatorsMap> newAggregatorsList = request.getAppAggregatorsList();
+    if (newAggregatorsList != null && !newAggregatorsList.isEmpty()) {
+      Map<ApplicationId, String> newAggregatorsMap = 
+          new HashMap<ApplicationId, String>();
+      for (AppAggregatorsMap aggregator : newAggregatorsList) {
+        newAggregatorsMap.put(aggregator.getApplicationId(), aggregator.getAggregatorAddr());
+      }
+      ((NodeManager.NMContext)context).addRegisteredAggregators(newAggregatorsMap);
+    }
+    
+    return ReportNewAggregatorsInfoResponse.newInstance();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.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/application/ApplicationImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
index a73b113..6bf3bbf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
@@ -425,6 +425,10 @@ public class ApplicationImpl implements Application {
           new LogHandlerAppFinishedEvent(app.appId));
 
       app.context.getNMTokenSecretManager().appFinished(app.getAppId());
+      // Remove aggregator info for finished apps.
+      // TODO check we remove related aggregators info in failure cases (YARN-3038)
+      app.context.getRegisteredAggregators().remove(app.getAppId());
+      app.context.getKnownAggregators().remove(app.getAppId());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.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/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index 1c7f987..2eb1a7f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -344,6 +344,8 @@ public class ApplicationMasterService extends AbstractService implements
 
     RMApp rmApp =
         rmContext.getRMApps().get(applicationAttemptId.getApplicationId());
+    // Remove aggregator address when app get finished.
+    rmApp.removeAggregatorAddr();
     // checking whether the app exits in RMStateStore at first not to throw
     // ApplicationDoesNotExistInCacheException before and after
     // RM work-preserving restart.
@@ -576,6 +578,10 @@ public class ApplicationMasterService extends AbstractService implements
       allocateResponse.setAvailableResources(allocation.getResourceLimit());
 
       allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
+      
+      // add aggregator address for this application
+      allocateResponse.setAggregatorAddr(
+          this.rmContext.getRMApps().get(applicationId).getAggregatorAddr());
 
       // add preemption to the allocateResponse message (if any)
       allocateResponse

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.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/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 0de556b..f163a28 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -21,6 +21,9 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.logging.Log;
@@ -57,6 +60,7 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppAggregatorUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
@@ -406,6 +410,11 @@ public class ResourceTrackerService extends AbstractService implements
           new RMNodeEvent(nodeId, RMNodeEventType.REBOOTING));
       return resync;
     }
+    
+    // Check & update aggregators info from request.
+    // TODO make sure it won't have race condition issue for AM failed over case
+    // that the older registration could possible override the newer one.
+    updateAppAggregatorsMap(request);
 
     // Heartbeat response
     NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
@@ -421,15 +430,72 @@ public class ResourceTrackerService extends AbstractService implements
     if (!systemCredentials.isEmpty()) {
       nodeHeartBeatResponse.setSystemCredentialsForApps(systemCredentials);
     }
+    
+    // Return aggregators' map that NM needs to know
+    // TODO we should optimize this to only include aggreator info that NM 
+    // doesn't know yet.
+    List<ApplicationId> keepAliveApps = remoteNodeStatus.getKeepAliveApplications();
+    if (keepAliveApps != null) {
+      setAppAggregatorsMapToResponse(keepAliveApps, nodeHeartBeatResponse);
+    }
 
     // 4. Send status to RMNode, saving the latest response.
     this.rmContext.getDispatcher().getEventHandler().handle(
         new RMNodeStatusEvent(nodeId, remoteNodeStatus.getNodeHealthStatus(),
             remoteNodeStatus.getContainersStatuses(), 
-            remoteNodeStatus.getKeepAliveApplications(), nodeHeartBeatResponse));
+            keepAliveApps, nodeHeartBeatResponse));
 
     return nodeHeartBeatResponse;
   }
+  
+  private void setAppAggregatorsMapToResponse(
+      List<ApplicationId> liveApps, NodeHeartbeatResponse response) {
+    Map<ApplicationId, String> liveAppAggregatorsMap = new 
+        ConcurrentHashMap<ApplicationId, String>();
+    Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
+      for (ApplicationId appId : liveApps) {
+        String appAggregatorAddr = rmApps.get(appId).getAggregatorAddr();
+        if (appAggregatorAddr != null) {
+          liveAppAggregatorsMap.put(appId, appAggregatorAddr);
+        } else {
+          // Log a debug info if aggregator address is not found.
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Aggregator for applicaton: " + appId + " hasn't registered yet!");
+          }
+        }
+      }
+    response.setAppAggregatorsMap(liveAppAggregatorsMap);
+  }
+  
+  private void updateAppAggregatorsMap(NodeHeartbeatRequest request) {
+    Map<ApplicationId, String> registeredAggregatorsMap = 
+        request.getRegisteredAggregators();
+    if (registeredAggregatorsMap != null 
+        && !registeredAggregatorsMap.isEmpty()) {
+      Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
+      for (Map.Entry<ApplicationId, String> entry: 
+          registeredAggregatorsMap.entrySet()) {
+        ApplicationId appId = entry.getKey();
+        String aggregatorAddr = entry.getValue();
+        if (aggregatorAddr != null && !aggregatorAddr.isEmpty()) {
+          RMApp rmApp = rmApps.get(appId);
+          if (rmApp == null) {
+            LOG.warn("Cannot update aggregator info because application ID: " + 
+                appId + " is not found in RMContext!");
+          } else {
+            String previousAggregatorAddr = rmApp.getAggregatorAddr();
+            if (previousAggregatorAddr == null || 
+                previousAggregatorAddr != aggregatorAddr) {
+              // sending aggregator update event.
+              RMAppAggregatorUpdateEvent event =
+                  new RMAppAggregatorUpdateEvent(appId, aggregatorAddr);
+              rmContext.getDispatcher().getEventHandler().handle(event);
+            }
+          }
+        }
+      }
+    }
+  }
 
   private void populateKeys(NodeHeartbeatRequest request,
       NodeHeartbeatResponse nodeHeartBeatResponse) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.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/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index fbcaab9..f81edb2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -172,6 +172,23 @@ public interface RMApp extends EventHandler<RMAppEvent> {
    * @return the tracking url for the application master.
    */
   String getTrackingUrl();
+  
+  /**
+   * The aggregator address for the application.
+   * @return the address for the application's aggregator.
+   */
+  String getAggregatorAddr();
+  
+  /**
+   * Set aggregator address for the application
+   * @param aggregatorAddr the address of aggregator
+   */
+  void setAggregatorAddr(String aggregatorAddr);
+  
+  /**
+   * Remove aggregator address when application is finished or killed.
+   */
+  void removeAggregatorAddr();
 
   /**
    * The original tracking url for the application master.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppAggregatorUpdateEvent.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/RMAppAggregatorUpdateEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppAggregatorUpdateEvent.java
new file mode 100644
index 0000000..b43de44
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppAggregatorUpdateEvent.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.yarn.server.resourcemanager.rmapp;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+
+public class RMAppAggregatorUpdateEvent extends RMAppEvent {
+
+  private final String appAggregatorAddr;
+  
+  public RMAppAggregatorUpdateEvent(ApplicationId appId, String appAggregatorAddr) {
+    super(appId, RMAppEventType.AGGREGATOR_UPDATE);
+    this.appAggregatorAddr = appAggregatorAddr;
+  }
+  
+  public String getAppAggregatorAddr(){
+    return this.appAggregatorAddr;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.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/RMAppEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
index 668c5e1..6e9460a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
@@ -30,6 +30,9 @@ public enum RMAppEventType {
 
   // Source: Scheduler
   APP_ACCEPTED,
+  
+  // TODO add source later
+  AGGREGATOR_UPDATE,
 
   // Source: RMAppAttempt
   ATTEMPT_REGISTERED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/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 2d1737a..6a076ac 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
@@ -134,6 +134,7 @@ public class RMAppImpl implements RMApp, Recoverable {
   private long startTime;
   private long finishTime = 0;
   private long storedFinishTime = 0;
+  private String aggregatorAddr;
   // This field isn't protected by readlock now.
   private volatile RMAppAttempt currentAttempt;
   private String queue;
@@ -165,6 +166,8 @@ public class RMAppImpl implements RMApp, Recoverable {
      // Transitions from NEW state
     .addTransition(RMAppState.NEW, RMAppState.NEW,
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
+    .addTransition(RMAppState.NEW, RMAppState.NEW,
+        RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
     .addTransition(RMAppState.NEW, RMAppState.NEW_SAVING,
         RMAppEventType.START, new RMAppNewlySavingTransition())
     .addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED,
@@ -181,6 +184,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     // Transitions from NEW_SAVING state
     .addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING,
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
+    .addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING,
+        RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
     .addTransition(RMAppState.NEW_SAVING, RMAppState.SUBMITTED,
         RMAppEventType.APP_NEW_SAVED, new AddApplicationToSchedulerTransition())
     .addTransition(RMAppState.NEW_SAVING, RMAppState.FINAL_SAVING,
@@ -199,6 +204,8 @@ public class RMAppImpl implements RMApp, Recoverable {
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
     .addTransition(RMAppState.SUBMITTED, RMAppState.SUBMITTED,
         RMAppEventType.MOVE, new RMAppMoveTransition())
+    .addTransition(RMAppState.SUBMITTED, RMAppState.SUBMITTED,
+        RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
     .addTransition(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING,
         RMAppEventType.APP_REJECTED,
         new FinalSavingTransition(
@@ -215,6 +222,8 @@ public class RMAppImpl implements RMApp, Recoverable {
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
     .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
         RMAppEventType.MOVE, new RMAppMoveTransition())
+    .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
+        RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
     .addTransition(RMAppState.ACCEPTED, RMAppState.RUNNING,
         RMAppEventType.ATTEMPT_REGISTERED)
     .addTransition(RMAppState.ACCEPTED,
@@ -241,6 +250,8 @@ public class RMAppImpl implements RMApp, Recoverable {
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
     .addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
         RMAppEventType.MOVE, new RMAppMoveTransition())
+    .addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
+        RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
     .addTransition(RMAppState.RUNNING, RMAppState.FINAL_SAVING,
         RMAppEventType.ATTEMPT_UNREGISTERED,
         new FinalSavingTransition(
@@ -270,6 +281,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING, 
         RMAppEventType.APP_RUNNING_ON_NODE,
         new AppRunningOnNodeTransition())
+    .addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
+        RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
     // ignorable transitions
     .addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
         EnumSet.of(RMAppEventType.NODE_UPDATE, RMAppEventType.KILL,
@@ -281,6 +294,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.FINISHING, RMAppState.FINISHING, 
         RMAppEventType.APP_RUNNING_ON_NODE,
         new AppRunningOnNodeTransition())
+    .addTransition(RMAppState.FINISHING, RMAppState.FINISHING,
+        RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
     // ignorable transitions
     .addTransition(RMAppState.FINISHING, RMAppState.FINISHING,
       EnumSet.of(RMAppEventType.NODE_UPDATE,
@@ -292,6 +307,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.KILLING, RMAppState.KILLING, 
         RMAppEventType.APP_RUNNING_ON_NODE,
         new AppRunningOnNodeTransition())
+    .addTransition(RMAppState.KILLING, RMAppState.KILLING,
+        RMAppEventType.AGGREGATOR_UPDATE, new RMAppAggregatorUpdateTransition())
     .addTransition(RMAppState.KILLING, RMAppState.FINAL_SAVING,
         RMAppEventType.ATTEMPT_KILLED,
         new FinalSavingTransition(
@@ -488,6 +505,21 @@ public class RMAppImpl implements RMApp, Recoverable {
   public void setQueue(String queue) {
     this.queue = queue;
   }
+  
+  @Override
+  public String getAggregatorAddr() {
+    return this.aggregatorAddr;
+  }
+  
+  @Override
+  public void setAggregatorAddr(String aggregatorAddr) {
+    this.aggregatorAddr = aggregatorAddr;
+  }
+  
+  @Override
+  public void removeAggregatorAddr() {
+    this.aggregatorAddr = null;
+  }
 
   @Override
   public String getName() {
@@ -737,6 +769,8 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.diagnostics.append(appState.getDiagnostics());
     this.storedFinishTime = appState.getFinishTime();
     this.startTime = appState.getStartTime();
+    //TODO recover aggregator address.
+    //this.aggregatorAddr = appState.getAggregatorAddr();
 
     for(int i=0; i<appState.getAttemptCount(); ++i) {
       // create attempt
@@ -778,9 +812,24 @@ public class RMAppImpl implements RMApp, Recoverable {
       SingleArcTransition<RMAppImpl, RMAppEvent> {
     public void transition(RMAppImpl app, RMAppEvent event) {
     };
-
   }
 
+  private static final class RMAppAggregatorUpdateTransition 
+      extends RMAppTransition {
+  
+    public void transition(RMAppImpl app, RMAppEvent event) {
+      LOG.info("Updating aggregator info for app: " + app.getApplicationId());
+    
+      RMAppAggregatorUpdateEvent appAggregatorUpdateEvent = 
+          (RMAppAggregatorUpdateEvent) event;
+      // Update aggregator address
+      app.setAggregatorAddr(appAggregatorUpdateEvent.getAppAggregatorAddr());
+      
+      // TODO persistent to RMStateStore for recover
+      // Save to RMStateStore
+    };
+  }
+  
   private static final class RMAppNodeUpdateTransition extends RMAppTransition {
     public void transition(RMAppImpl app, RMAppEvent event) {
       RMAppNodeUpdateEvent nodeUpdateEvent = (RMAppNodeUpdateEvent) event;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.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/MockAsm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
index f8d92aa..0d0895a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
@@ -92,6 +92,18 @@ public abstract class MockAsm extends MockApps {
       throw new UnsupportedOperationException("Not supported yet.");
     }
     @Override
+    public String getAggregatorAddr() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+    @Override
+    public void setAggregatorAddr(String aggregatorAddr) {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+    @Override
+    public void removeAggregatorAddr() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+    @Override
     public ApplicationId getApplicationId() {
       throw new UnsupportedOperationException("Not supported yet.");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.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/MockRMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
index ec990f9..96952d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
@@ -271,4 +271,19 @@ public class MockRMApp implements RMApp {
   public ResourceRequest getAMResourceRequest() {
     return this.amReq; 
   }
+
+  @Override
+  public String getAggregatorAddr() {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+  
+  @Override
+  public void removeAggregatorAddr() {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
+
+  @Override
+  public void setAggregatorAddr(String aggregatorAddr) {
+    throw new UnsupportedOperationException("Not supported yet.");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeTimelineAggregatorsAuxService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeTimelineAggregatorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeTimelineAggregatorsAuxService.java
index cdc4e35..19920fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeTimelineAggregatorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/PerNodeTimelineAggregatorsAuxService.java
@@ -94,10 +94,9 @@ public class PerNodeTimelineAggregatorsAuxService extends AuxiliaryService {
    * @return whether it was added successfully
    */
   public boolean addApplication(ApplicationId appId) {
-    String appIdString = appId.toString();
     AppLevelTimelineAggregator aggregator =
-        new AppLevelTimelineAggregator(appIdString);
-    return (aggregatorCollection.putIfAbsent(appIdString, aggregator)
+        new AppLevelTimelineAggregator(appId.toString());
+    return (aggregatorCollection.putIfAbsent(appId, aggregator)
         == aggregator);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorsCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorsCollection.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorsCollection.java
index 73b6d52..d6e2a18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorsCollection.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TimelineAggregatorsCollection.java
@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.aggregator;
 
+import java.io.IOException;
 import java.net.URI;
+import java.net.InetSocketAddress;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -30,9 +32,15 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.lib.StaticUserWebFilter;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.server.api.AggregatorNodemanagerProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewAggregatorsInfoRequest;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -62,6 +70,12 @@ public class TimelineAggregatorsCollection extends CompositeService {
 
   // REST server for this aggregator collection
   private HttpServer2 timelineRestServer;
+  
+  private String timelineRestServerBindAddress;
+  
+  private AggregatorNodemanagerProtocol nmAggregatorService;
+  
+  private InetSocketAddress nmAggregatorServiceAddress;
 
   static final String AGGREGATOR_COLLECTION_ATTR_KEY = "aggregator.collection";
 
@@ -74,6 +88,16 @@ public class TimelineAggregatorsCollection extends CompositeService {
   }
 
   @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    this.nmAggregatorServiceAddress = conf.getSocketAddr(
+        YarnConfiguration.NM_BIND_HOST,
+        YarnConfiguration.NM_AGGREGATOR_SERVICE_ADDRESS,
+        YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_ADDRESS,
+        YarnConfiguration.DEFAULT_NM_AGGREGATOR_SERVICE_PORT);
+    
+  }
+  
+  @Override
   protected void serviceStart() throws Exception {
     startWebApp();
     super.serviceStart();
@@ -95,9 +119,13 @@ public class TimelineAggregatorsCollection extends CompositeService {
    * starting the app level service
    * @return the aggregator associated with id after the potential put.
    */
-  public TimelineAggregator putIfAbsent(String id, TimelineAggregator aggregator) {
+  public TimelineAggregator putIfAbsent(ApplicationId appId, 
+      TimelineAggregator aggregator) {
+    String id = appId.toString();
+    TimelineAggregator aggregatorInTable;
+    boolean aggregatorIsNew = false;
     synchronized (aggregators) {
-      TimelineAggregator aggregatorInTable = aggregators.get(id);
+      aggregatorInTable = aggregators.get(id);
       if (aggregatorInTable == null) {
         try {
           // initialize, start, and add it to the collection so it can be
@@ -106,16 +134,30 @@ public class TimelineAggregatorsCollection extends CompositeService {
           aggregator.start();
           aggregators.put(id, aggregator);
           LOG.info("the aggregator for " + id + " was added");
-          return aggregator;
+          aggregatorInTable = aggregator;
+          aggregatorIsNew = true;
         } catch (Exception e) {
           throw new YarnRuntimeException(e);
         }
       } else {
         String msg = "the aggregator for " + id + " already exists!";
         LOG.error(msg);
-        return aggregatorInTable;
+      }
+      
+    }
+    // Report to NM if a new aggregator is added.
+    if (aggregatorIsNew) {
+      try {
+        reportNewAggregatorToNM(appId);
+      } catch (Exception e) {
+        // throw exception here as it cannot be used if failed report to NM
+        LOG.error("Failed to report a new aggregator for application: " + appId + 
+            " to NM Aggregator Services.");
+        throw new YarnRuntimeException(e);
       }
     }
+    
+    return aggregatorInTable;
   }
 
   /**
@@ -167,7 +209,10 @@ public class TimelineAggregatorsCollection extends CompositeService {
     String bindAddress = WebAppUtils.getWebAppBindURL(conf,
         YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
         WebAppUtils.getAHSWebAppURLWithoutScheme(conf));
-    LOG.info("Instantiating the per-node aggregator webapp at " + bindAddress);
+    this.timelineRestServerBindAddress = WebAppUtils.getResolvedAddress(
+        NetUtils.createSocketAddr(bindAddress));
+    LOG.info("Instantiating the per-node aggregator webapp at " + 
+        timelineRestServerBindAddress);
     try {
       Configuration confForInfoServer = new Configuration(conf);
       confForInfoServer.setInt(HttpServer2.HTTP_MAX_THREADS, 10);
@@ -200,4 +245,27 @@ public class TimelineAggregatorsCollection extends CompositeService {
       throw new YarnRuntimeException(msg, e);
     }
   }
+  
+  private void reportNewAggregatorToNM(ApplicationId appId) 
+      throws YarnException, IOException {
+    this.nmAggregatorService = getNMAggregatorService();
+    ReportNewAggregatorsInfoRequest request = 
+        ReportNewAggregatorsInfoRequest.newInstance(appId,
+            this.timelineRestServerBindAddress);
+    LOG.info("Report a new aggregator for application: " + appId + 
+        " to NM Aggregator Services.");
+    nmAggregatorService.reportNewAggregatorInfo(request);
+  }
+  
+  // protected for test
+  protected AggregatorNodemanagerProtocol getNMAggregatorService(){
+    Configuration conf = getConfig();
+    final YarnRPC rpc = YarnRPC.create(conf);
+    
+    // TODO Security settings.
+    return (AggregatorNodemanagerProtocol) rpc.getProxy(
+        AggregatorNodemanagerProtocol.class,
+        nmAggregatorServiceAddress, conf);
+  }
+  
 }


[12/50] hadoop git commit: Moving CHANGES.txt entry for MAPREDUCE-4742 to branch-2.7.

Posted by zj...@apache.org.
Moving CHANGES.txt entry for MAPREDUCE-4742 to branch-2.7.


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

Branch: refs/heads/YARN-2928
Commit: bd0a9ba8e3b1c75dd2fc4cc65cb00c3e31d609ce
Parents: 9d38520
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sat Mar 14 16:53:50 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sat Mar 14 16:53:50 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd0a9ba8/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index ab6eef5..28460d3 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -257,8 +257,6 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
-    MAPREDUCE-4742. Fix typo in nnbench#displayUsage. (Liang Xie via ozawa)
-
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -440,6 +438,8 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc
     comments. (Akira AJISAKA and Andrew Purtell via ozawa)
 
+    MAPREDUCE-4742. Fix typo in nnbench#displayUsage. (Liang Xie via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[19/50] hadoop git commit: YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA, Andrew Purtell, and Allen Wittenauer.

Posted by zj...@apache.org.
YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA, Andrew Purtell, and Allen Wittenauer.


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

Branch: refs/heads/YARN-2928
Commit: 3da9a97cfbcc3a1c50aaf85b1a129d4d269cd5fd
Parents: 3ff1ba2
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon Mar 16 23:19:05 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Mon Mar 16 23:19:05 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../yarn/api/ApplicationBaseProtocol.java       | 44 ++++---------
 .../yarn/api/ApplicationClientProtocol.java     |  3 -
 .../api/protocolrecords/AllocateRequest.java    | 25 ++++---
 .../api/protocolrecords/AllocateResponse.java   | 68 ++++++++++----------
 .../FinishApplicationMasterRequest.java         | 25 ++++---
 .../FinishApplicationMasterResponse.java        |  7 +-
 .../protocolrecords/GetApplicationsRequest.java |  1 -
 .../GetClusterMetricsResponse.java              |  4 +-
 .../GetContainerStatusesRequest.java            |  2 -
 .../GetContainerStatusesResponse.java           |  2 -
 .../protocolrecords/GetQueueInfoRequest.java    |  2 +-
 .../protocolrecords/GetQueueInfoResponse.java   | 11 ++--
 .../KillApplicationResponse.java                |  9 ++-
 .../RegisterApplicationMasterRequest.java       | 33 +++++-----
 .../RegisterApplicationMasterResponse.java      | 11 ++--
 .../protocolrecords/StartContainerRequest.java  |  9 +--
 .../api/records/ApplicationAttemptReport.java   | 23 +++----
 .../yarn/api/records/ApplicationReport.java     | 47 +++++++-------
 .../records/ApplicationSubmissionContext.java   | 50 +++++++-------
 .../hadoop/yarn/api/records/Container.java      | 49 +++++++-------
 .../api/records/ContainerLaunchContext.java     | 35 +++++-----
 .../yarn/api/records/ContainerReport.java       | 29 ++++-----
 .../yarn/api/records/ContainerStatus.java       | 21 +++---
 .../yarn/api/records/LocalResourceType.java     | 32 ++++-----
 .../api/records/LocalResourceVisibility.java    | 31 +++++----
 .../yarn/api/records/LogAggregationContext.java | 39 ++++++-----
 .../hadoop/yarn/api/records/NodeReport.java     | 25 ++++---
 .../yarn/api/records/PreemptionMessage.java     | 32 ++++-----
 .../hadoop/yarn/api/records/QueueACL.java       | 13 ++--
 .../hadoop/yarn/api/records/QueueInfo.java      | 25 ++++---
 .../hadoop/yarn/api/records/QueueState.java     | 15 ++---
 .../yarn/api/records/ReservationRequest.java    | 17 ++---
 .../records/ReservationRequestInterpreter.java  | 38 +++++------
 .../yarn/api/records/ResourceRequest.java       | 51 +++++++--------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  5 +-
 .../UpdateNodeResourceRequest.java              |  4 +-
 .../hadoop/yarn/client/api/AHSClient.java       | 24 +++----
 .../hadoop/yarn/client/api/AMRMClient.java      |  4 +-
 .../apache/hadoop/yarn/client/api/NMClient.java |  4 +-
 .../hadoop/yarn/client/api/NMTokenCache.java    | 58 ++++++++---------
 .../hadoop/yarn/client/api/YarnClient.java      | 23 ++++---
 .../nodelabels/CommonNodeLabelsManager.java     |  6 +-
 .../hadoop/yarn/nodelabels/NodeLabelsStore.java |  3 +-
 .../server/security/ApplicationACLsManager.java |  1 -
 .../apache/hadoop/yarn/util/StringHelper.java   |  6 +-
 .../org/apache/hadoop/yarn/webapp/WebApps.java  |  4 +-
 .../registry/client/binding/RegistryUtils.java  |  8 +--
 .../client/impl/RegistryOperationsClient.java   |  2 +-
 .../client/impl/zk/ZookeeperConfigOptions.java  |  3 +-
 .../server/services/MicroZookeeperService.java  | 10 +--
 .../registry/server/services/package-info.java  |  9 ++-
 ...TimelineAuthenticationFilterInitializer.java | 13 ++--
 .../org/apache/hadoop/yarn/lib/ZKClient.java    |  2 +-
 .../RegisterNodeManagerRequest.java             |  3 +-
 .../server/api/records/NodeHealthStatus.java    | 24 ++++---
 .../server/nodemanager/ContainerExecutor.java   |  8 ++-
 .../util/NodeManagerHardwareUtils.java          |  8 +--
 .../rmapp/attempt/RMAppAttempt.java             | 11 ++--
 .../scheduler/SchedulerNode.java                |  2 +-
 .../scheduler/SchedulerUtils.java               |  3 +-
 .../fair/policies/ComputeFairShares.java        | 19 +++---
 .../security/DelegationTokenRenewer.java        |  2 -
 .../yarn/server/webproxy/ProxyUriUtils.java     |  2 +-
 64 files changed, 517 insertions(+), 585 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index bcab88c..26ef7d3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -769,6 +769,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3171. Sort by Application id, AppAttempt and ContainerID doesn't work
     in ATS / RM web ui. (Naganarasimha G R via xgong)
 
+    YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in 
+    doc comments. (Akira AJISAKA, Andrew Purtell, and Allen Wittenauer via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
index 2a8a283..8234c2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
@@ -65,41 +65,31 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 public interface ApplicationBaseProtocol {
 
   /**
-   * <p>
    * The interface used by clients to get a report of an Application from the
    * <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>.
-   * </p>
-   *
    * <p>
    * The client, via {@link GetApplicationReportRequest} provides the
    * {@link ApplicationId} of the application.
-   * </p>
-   *
    * <p>
    * In secure mode,the <code>ResourceManager</code> or
    * <code>ApplicationHistoryServer</code> verifies access to the application,
    * queue etc. before accepting the request.
-   * </p>
-   *
    * <p>
    * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
    * responds with a {@link GetApplicationReportResponse} which includes the
    * {@link ApplicationReport} for the application.
-   * </p>
-   *
    * <p>
    * If the user does not have <code>VIEW_APP</code> access then the following
    * fields in the report will be set to stubbed values:
    * <ul>
-   * <li>host - set to "N/A"</li>
-   * <li>RPC port - set to -1</li>
-   * <li>client token - set to "N/A"</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL - set to "N/A"</li>
-   * <li>original tracking URL - set to "N/A"</li>
-   * <li>resource usage report - all values are -1</li>
+   *   <li>host - set to "N/A"</li>
+   *   <li>RPC port - set to -1</li>
+   *   <li>client token - set to "N/A"</li>
+   *   <li>diagnostics - set to "N/A"</li>
+   *   <li>tracking URL - set to "N/A"</li>
+   *   <li>original tracking URL - set to "N/A"</li>
+   *   <li>resource usage report - all values are -1</li>
    * </ul>
-   * </p>
    *
    * @param request
    *          request for an application report
@@ -148,40 +138,30 @@ public interface ApplicationBaseProtocol {
           IOException;
 
   /**
-   * <p>
    * The interface used by clients to get a report of an Application Attempt
    * from the <code>ResourceManager</code> or
    * <code>ApplicationHistoryServer</code>
-   * </p>
-   *
    * <p>
    * The client, via {@link GetApplicationAttemptReportRequest} provides the
    * {@link ApplicationAttemptId} of the application attempt.
-   * </p>
-   *
    * <p>
    * In secure mode,the <code>ResourceManager</code> or
    * <code>ApplicationHistoryServer</code> verifies access to the method before
    * accepting the request.
-   * </p>
-   *
    * <p>
    * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
    * responds with a {@link GetApplicationAttemptReportResponse} which includes
    * the {@link ApplicationAttemptReport} for the application attempt.
-   * </p>
-   *
    * <p>
    * If the user does not have <code>VIEW_APP</code> access then the following
    * fields in the report will be set to stubbed values:
    * <ul>
-   * <li>host</li>
-   * <li>RPC port</li>
-   * <li>client token</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL</li>
+   *   <li>host</li>
+   *   <li>RPC port</li>
+   *   <li>client token</li>
+   *   <li>diagnostics - set to "N/A"</li>
+   *   <li>tracking URL</li>
    * </ul>
-   * </p>
    *
    * @param request
    *          request for an application attempt report

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/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 0a7d415..8b9937b 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
@@ -135,9 +135,6 @@ public interface ApplicationClientProtocol extends ApplicationBaseProtocol {
    * @return (empty) response on accepting the submission
    * @throws YarnException
    * @throws IOException
-   * @throws InvalidResourceRequestException
-   *           The exception is thrown when a {@link ResourceRequest} is out of
-   *           the range of the configured lower and upper resource boundaries.
    * @see #getNewApplication(GetNewApplicationRequest)
    */
   @Public

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
index 62316a6..2458d9b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
@@ -35,19 +35,18 @@ import org.apache.hadoop.yarn.util.Records;
  * <code>ResourceManager</code> to obtain resources in the cluster.</p> 
  *
  * <p>The request includes:
- *   <ul>
- *     <li>A response id to track duplicate responses.</li>
- *     <li>Progress information.</li>
- *     <li>
- *       A list of {@link ResourceRequest} to inform the 
- *       <code>ResourceManager</code> about the application's 
- *       resource requirements.
- *     </li>
- *     <li>
- *       A list of unused {@link Container} which are being returned. 
- *     </li>
- *   </ul>
- * </p>
+ * <ul>
+ *   <li>A response id to track duplicate responses.</li>
+ *   <li>Progress information.</li>
+ *   <li>
+ *     A list of {@link ResourceRequest} to inform the
+ *     <code>ResourceManager</code> about the application's
+ *     resource requirements.
+ *   </li>
+ *   <li>
+ *     A list of unused {@link Container} which are being returned.
+ *   </li>
+ * </ul>
  * 
  * @see ApplicationMasterProtocol#allocate(AllocateRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
index 46ac642..c4fdb79 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
@@ -39,27 +39,27 @@ import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>ResourceManager</code> the  
- * <code>ApplicationMaster</code> during resource negotiation.</p>
- *
- * <p>The response, includes:
- *   <ul>
- *     <li>Response ID to track duplicate responses.</li>
- *     <li>
- *       An AMCommand sent by ResourceManager to let the <code>ApplicationMaster</code>
- *       take some actions (resync, shutdown etc.).
- *     <li>A list of newly allocated {@link Container}.</li>
- *     <li>A list of completed {@link Container}s' statuses.</li>
- *     <li>
- *       The available headroom for resources in the cluster for the
- *       application. 
- *     </li>
- *     <li>A list of nodes whose status has been updated.</li>
- *     <li>The number of available nodes in a cluster.</li>
- *     <li>A description of resources requested back by the cluster</li>
- *     <li>AMRMToken, if AMRMToken has been rolled over</li>
- *   </ul>
- * </p>
+ * The response sent by the <code>ResourceManager</code> the
+ * <code>ApplicationMaster</code> during resource negotiation.
+ * <p>
+ * The response, includes:
+ * <ul>
+ *   <li>Response ID to track duplicate responses.</li>
+ *   <li>
+ *     An AMCommand sent by ResourceManager to let the
+ *     {@code ApplicationMaster} take some actions (resync, shutdown etc.).
+ *   </li>
+ *   <li>A list of newly allocated {@link Container}.</li>
+ *   <li>A list of completed {@link Container}s' statuses.</li>
+ *   <li>
+ *     The available headroom for resources in the cluster for the
+ *     application.
+ *   </li>
+ *   <li>A list of nodes whose status has been updated.</li>
+ *   <li>The number of available nodes in a cluster.</li>
+ *   <li>A description of resources requested back by the cluster</li>
+ *   <li>AMRMToken, if AMRMToken has been rolled over</li>
+ * </ul>
  * 
  * @see ApplicationMasterProtocol#allocate(AllocateRequest)
  */
@@ -220,16 +220,16 @@ public abstract class AllocateResponse {
   public abstract void setNumClusterNodes(int numNodes);
 
   /**
-   * <p>Get the description of containers owned by the AM, but requested back by
+   * Get the description of containers owned by the AM, but requested back by
    * the cluster. Note that the RM may have an inconsistent view of the
    * resources owned by the AM. These messages are advisory, and the AM may
-   * elect to ignore them.<p>
-   *
-   * <p>The message is a snapshot of the resources the RM wants back from the AM.
+   * elect to ignore them.
+   * <p>
+   * The message is a snapshot of the resources the RM wants back from the AM.
    * While demand persists, the RM will repeat its request; applications should
-   * not interpret each message as a request for <em>additional<em>
+   * not interpret each message as a request for <em>additional</em>
    * resources on top of previous messages. Resources requested consistently
-   * over some duration may be forcibly killed by the RM.<p>
+   * over some duration may be forcibly killed by the RM.
    *
    * @return A specification of the resources to reclaim from this AM.
    */
@@ -242,15 +242,17 @@ public abstract class AllocateResponse {
   public abstract void setPreemptionMessage(PreemptionMessage request);
 
   /**
-   * <p>Get the list of NMTokens required for communicating with NM. New NMTokens
-   * issued only if<p>
-   * <p>1) AM is receiving first container on underlying NodeManager.<br>
+   * Get the list of NMTokens required for communicating with NM. New NMTokens
+   * issued only if
+   * <p>
+   * 1) AM is receiving first container on underlying NodeManager.<br>
    * OR<br>
    * 2) NMToken master key rolled over in ResourceManager and AM is getting new
-   * container on the same underlying NodeManager.<p>
-   * <p>AM will receive one NMToken per NM irrespective of the number of containers
+   * container on the same underlying NodeManager.
+   * <p>
+   * AM will receive one NMToken per NM irrespective of the number of containers
    * issued on same NM. AM is expected to store these tokens until issued a
-   * new token for the same NM.<p>
+   * new token for the same NM.
    */
   @Public
   @Stable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
index 15c3680..cbbe9c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
@@ -25,19 +25,18 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The finalization request sent by the <code>ApplicationMaster</code> to
- * inform the <code>ResourceManager</code> about its completion.</p>
- *
- * <p>The final request includes details such:
- *   <ul>
- *     <li>Final state of the <code>ApplicationMaster</code></li>
- *     <li>
- *       Diagnostic information in case of failure of the
- *       <code>ApplicationMaster</code>
- *     </li>
- *     <li>Tracking URL</li>
- *   </ul>
- * </p>
+ * The finalization request sent by the {@code ApplicationMaster} to
+ * inform the {@code ResourceManager} about its completion.
+ * <p>
+ * The final request includes details such:
+ * <ul>
+ *   <li>Final state of the {@code ApplicationMaster}</li>
+ *   <li>
+ *     Diagnostic information in case of failure of the
+ *     {@code ApplicationMaster}
+ *   </li>
+ *   <li>Tracking URL</li>
+ * </ul>
  *
  * @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java
index 8de2c73..6647a10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java
@@ -26,22 +26,19 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * The response sent by the <code>ResourceManager</code> to a
  * <code>ApplicationMaster</code> on it's completion.
- * </p>
- * 
  * <p>
  * The response, includes:
  * <ul>
  * <li>A flag which indicates that the application has successfully unregistered
  * with the RM and the application can safely stop.</li>
  * </ul>
- * </p>
+ * <p>
  * Note: The flag indicates whether the application has successfully
  * unregistered and is safe to stop. The application may stop after the flag is
  * true. If the application stops before the flag is true then the RM may retry
- * the application .
+ * the application.
  * 
  * @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java
index 7fc58d6..35392e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.yarn.util.Records;
  * <p>The request from clients to get a report of Applications
  * in the cluster from the <code>ResourceManager</code>.</p>
  *
- *
  * @see ApplicationClientProtocol#getApplications(GetApplicationsRequest)
  */
 @Public

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java
index 6329aac..18a0807 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>ResourceManager</code> to a client
- * requesting cluster metrics.<p>
+ * The response sent by the <code>ResourceManager</code> to a client
+ * requesting cluster metrics.
  * 
  * @see YarnClusterMetrics
  * @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
index f9f77a3..60c63ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
@@ -28,11 +28,9 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * The request sent by the <code>ApplicationMaster</code> to the
  * <code>NodeManager</code> to get {@link ContainerStatus} of requested
  * containers.
- * </p>
  * 
  * @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
index b0a0f0e..68e6a8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
@@ -32,11 +32,9 @@ import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * The response sent by the <code>NodeManager</code> to the
  * <code>ApplicationMaster</code> when asked to obtain the
  * <code>ContainerStatus</code> of requested containers.
- * </p>
  * 
  * @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java
index df3342f..0e33e21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java
@@ -63,7 +63,7 @@ public abstract class GetQueueInfoRequest {
   public abstract void setQueueName(String queueName);
 
   /**
-   * Is information about <em>active applications<e/m> required?
+   * Is information about <em>active applications</em> required?
    * @return <code>true</code> if applications' information is to be included,
    *         else <code>false</code>
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java
index ea6cb7b..b97a5a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java
@@ -27,12 +27,11 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>ResourceManager</code> to a client
- * requesting information about queues in the system.</p>
- *
- * <p>The response includes a {@link QueueInfo} which has details such as
- * queue name, used/total capacities, running applications, child queues etc
- * .</p>
+ * The response sent by the {@code ResourceManager} to a client
+ * requesting information about queues in the system.
+ * <p>
+ * The response includes a {@link QueueInfo} which has details such as
+ * queue name, used/total capacities, running applications, child queues etc.
  * 
  * @see QueueInfo
  * @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java
index 77bb71d..7225bf5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java
@@ -26,21 +26,20 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * The response sent by the <code>ResourceManager</code> to the client aborting
  * a submitted application.
- * </p>
  * <p>
  * The response, includes:
  * <ul>
- * <li>A flag which indicates that the process of killing the application is
- * completed or not.</li>
+ *   <li>
+ *     A flag which indicates that the process of killing the application is
+ *     completed or not.
+ *   </li>
  * </ul>
  * Note: user is recommended to wait until this flag becomes true, otherwise if
  * the <code>ResourceManager</code> crashes before the process of killing the
  * application is completed, the <code>ResourceManager</code> may retry this
  * application on recovery.
- * </p>
  * 
  * @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
index 6b01854..395e190 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
@@ -24,16 +24,15 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The request sent by the <code>ApplicationMaster</code> to 
- * <code>ResourceManager</code> on registration.</p>
- * 
- * <p>The registration includes details such as:
- *   <ul>
- *     <li>Hostname on which the AM is running.</li>
- *     <li>RPC Port</li>
- *     <li>Tracking URL</li>
- *   </ul>
- * </p>
+ * The request sent by the {@code ApplicationMaster} to {@code ResourceManager}
+ * on registration.
+ * <p>
+ * The registration includes details such as:
+ * <ul>
+ *   <li>Hostname on which the AM is running.</li>
+ *   <li>RPC Port</li>
+ *   <li>Tracking URL</li>
+ * </ul>
  * 
  * @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)
  */
@@ -83,20 +82,20 @@ public abstract class RegisterApplicationMasterRequest {
   public abstract void setHost(String host);
 
   /**
-   * Get the <em>RPC port</em> on which the <code>ApplicationMaster</code> 
-   * is responding. 
-   * @return the <em>RPC port<em> on which the <code>ApplicationMaster</code> is 
-   *         responding
+   * Get the <em>RPC port</em> on which the {@code ApplicationMaster} is
+   * responding.
+   * @return the <em>RPC port</em> on which the {@code ApplicationMaster}
+   *         is responding
    */
   @Public
   @Stable
   public abstract int getRpcPort();
   
   /**
-   * Set the <em>RPC port<em> on which the <code>ApplicationMaster</code> is 
+   * Set the <em>RPC port</em> on which the {@code ApplicationMaster} is
    * responding.
-   * @param port <em>RPC port<em> on which the <code>ApplicationMaster</code> is 
-   *             responding
+   * @param port <em>RPC port</em> on which the {@code ApplicationMaster}
+   *             is responding
    */
   @Public
   @Stable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
index 33daf28..1a51ba6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
@@ -36,16 +36,15 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>ResourceManager</code> to a new 
- * <code>ApplicationMaster</code> on registration.</p>
- * 
- * <p>The response contains critical details such as:
+ * The response sent by the {@code ResourceManager} to a new
+ * {@code ApplicationMaster} on registration.
+ * <p>
+ * The response contains critical details such as:
  * <ul>
  *   <li>Maximum capability for allocated resources in the cluster.</li>
- *   <li><code>ApplicationACL</code>s for the application.</li>
+ *   <li>{@code ApplicationACL}s for the application.</li>
  *   <li>ClientToAMToken master key.</li>
  * </ul>
- * </p>
  * 
  * @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
index 1dcefb2..50179a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
@@ -74,10 +74,11 @@ public abstract class StartContainerRequest {
   public abstract void setContainerLaunchContext(ContainerLaunchContext context);
 
   /**
-   * <p>Get the container token to be used for authorization during starting
-   * container.</p>
-   * <p>Note: {@link NMToken} will be used for authenticating communication with </code>
-   * NodeManager</code>.</p>
+   * Get the container token to be used for authorization during starting
+   * container.
+   * <p>
+   * Note: {@link NMToken} will be used for authenticating communication with
+   * {@code NodeManager}.
    * @return the container token to be used for authorization during starting
    * container.
    * @see NMToken

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
index 53c18ae..b7f9c1b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
@@ -24,24 +24,19 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
- * <code>ApplicationAttemptReport</code> is a report of an application attempt.
- * </p>
- * 
+ * {@code ApplicationAttemptReport} is a report of an application attempt.
  * <p>
  * It includes details such as:
  * <ul>
- * <li>{@link ApplicationAttemptId} of the application.</li>
- * <li>Host on which the <code>ApplicationMaster</code> of this attempt is
- * running.</li>
- * <li>RPC port of the <code>ApplicationMaster</code> of this attempt.</li>
- * <li>Tracking URL.</li>
- * <li>Diagnostic information in case of errors.</li>
- * <li>{@link YarnApplicationAttemptState} of the application attempt.</li>
- * <li>{@link ContainerId} of the master Container.</li>
+ *   <li>{@link ApplicationAttemptId} of the application.</li>
+ *   <li>Host on which the <code>ApplicationMaster</code> of this attempt is
+ *   running.</li>
+ *   <li>RPC port of the <code>ApplicationMaster</code> of this attempt.</li>
+ *   <li>Tracking URL.</li>
+ *   <li>Diagnostic information in case of errors.</li>
+ *   <li>{@link YarnApplicationAttemptState} of the application attempt.</li>
+ *   <li>{@link ContainerId} of the master Container.</li>
  * </ul>
- * </p>
- * 
  */
 @Public
 @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
index 412c22b..ff4fb52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
@@ -28,23 +28,22 @@ import org.apache.hadoop.yarn.util.Records;
 import java.util.Set;
 
 /**
- * <p><code>ApplicationReport</code> is a report of an application.</p>
- *
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link ApplicationId} of the application.</li>
- *     <li>Applications user.</li>
- *     <li>Application queue.</li>
- *     <li>Application name.</li>
- *     <li>Host on which the <code>ApplicationMaster</code> is running.</li>
- *     <li>RPC port of the <code>ApplicationMaster</code>.</li>
- *     <li>Tracking URL.</li>
- *     <li>{@link YarnApplicationState} of the application.</li>
- *     <li>Diagnostic information in case of errors.</li>
- *     <li>Start time of the application.</li>
- *     <li>Client {@link Token} of the application (if security is enabled).</li>
- *   </ul>
- * </p>
+ * {@code ApplicationReport} is a report of an application.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link ApplicationId} of the application.</li>
+ *   <li>Applications user.</li>
+ *   <li>Application queue.</li>
+ *   <li>Application name.</li>
+ *   <li>Host on which the <code>ApplicationMaster</code> is running.</li>
+ *   <li>RPC port of the <code>ApplicationMaster</code>.</li>
+ *   <li>Tracking URL.</li>
+ *   <li>{@link YarnApplicationState} of the application.</li>
+ *   <li>Diagnostic information in case of errors.</li>
+ *   <li>Start time of the application.</li>
+ *   <li>Client {@link Token} of the application (if security is enabled).</li>
+ * </ul>
  *
  * @see ApplicationClientProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest)
  */
@@ -341,20 +340,20 @@ public abstract class ApplicationReport {
 
   /**
    * Get the AMRM token of the application.
-   * <p/>
+   * <p>
    * The AMRM token is required for AM to RM scheduling operations. For 
    * managed Application Masters Yarn takes care of injecting it. For unmanaged
    * Applications Masters, the token must be obtained via this method and set
    * in the {@link org.apache.hadoop.security.UserGroupInformation} of the
    * current user.
-   * <p/>
+   * <p>
    * The AMRM token will be returned only if all the following conditions are
    * met:
-   * <li>
-   *   <ul>the requester is the owner of the ApplicationMaster</ul>
-   *   <ul>the application master is an unmanaged ApplicationMaster</ul>
-   *   <ul>the application master is in ACCEPTED state</ul>
-   * </li>
+   * <ul>
+   *   <li>the requester is the owner of the ApplicationMaster</li>
+   *   <li>the application master is an unmanaged ApplicationMaster</li>
+   *   <li>the application master is in ACCEPTED state</li>
+   * </ul>
    * Else this method returns NULL.
    * 
    * @return the AM to RM token if available.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/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 c4014fc..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
@@ -33,32 +33,34 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>ApplicationSubmissionContext</code> represents all of the
- * information needed by the <code>ResourceManager</code> to launch 
- * the <code>ApplicationMaster</code> for an application.</p>
- * 
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link ApplicationId} of the application.</li>
- *     <li>Application user.</li>
- *     <li>Application name.</li>
- *     <li>{@link Priority} of the application.</li>
- *     <li>
- *       {@link ContainerLaunchContext} of the container in which the 
- *       <code>ApplicationMaster</code> is executed.
- *     </li>
- *     <li>maxAppAttempts. The maximum number of application attempts.
+ * {@code ApplicationSubmissionContext} represents all of the
+ * information needed by the {@code ResourceManager} to launch
+ * the {@code ApplicationMaster} for an application.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link ApplicationId} of the application.</li>
+ *   <li>Application user.</li>
+ *   <li>Application name.</li>
+ *   <li>{@link Priority} of the application.</li>
+ *   <li>
+ *     {@link ContainerLaunchContext} of the container in which the
+ *     <code>ApplicationMaster</code> is executed.
+ *   </li>
+ *   <li>
+ *     maxAppAttempts. The maximum number of application attempts.
  *     It should be no larger than the global number of max attempts in the
- *     Yarn configuration.</li>
- *     <li>attemptFailuresValidityInterval. The default value is -1.
- *     when attemptFailuresValidityInterval in milliseconds is set to > 0,
- *     the failure number will no take failures which happen out of the
- *     validityInterval into failure count. If failure count reaches to
- *     maxAppAttempts, the application will be failed.
- *     </li>
+ *     Yarn configuration.
+ *   </li>
+ *   <li>
+ *     attemptFailuresValidityInterval. The default value is -1.
+ *     when attemptFailuresValidityInterval in milliseconds is set to
+ *     {@literal >} 0, the failure number will no take failures which happen
+ *     out of the validityInterval into failure count. If failure count
+ *     reaches to maxAppAttempts, the application will be failed.
+ *   </li>
  *   <li>Optional, application-specific {@link LogAggregationContext}</li>
- *   </ul>
- * </p>
+ * </ul>
  * 
  * @see ContainerLaunchContext
  * @see ApplicationClientProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
index 279f127..38fa8b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
@@ -27,34 +27,31 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>Container</code> represents an allocated resource in the cluster.
- * </p>
- * 
- * <p>The <code>ResourceManager</code> is the sole authority to allocate any
- * <code>Container</code> to applications. The allocated <code>Container</code>
+ * {@code Container} represents an allocated resource in the cluster.
+ * <p>
+ * The {@code ResourceManager} is the sole authority to allocate any
+ * {@code Container} to applications. The allocated {@code Container}
  * is always on a single node and has a unique {@link ContainerId}. It has
- * a specific amount of {@link Resource} allocated.</p>
- * 
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link ContainerId} for the container, which is globally unique.</li>
- *     <li>
- *       {@link NodeId} of the node on which it is allocated.
- *     </li>
- *     <li>HTTP uri of the node.</li>
- *     <li>{@link Resource} allocated to the container.</li>
- *     <li>{@link Priority} at which the container was allocated.</li>
- *     <li>
- *       Container {@link Token} of the container, used to securely verify
- *       authenticity of the allocation. 
- *     </li>
- *   </ul>
- * </p>
+ * a specific amount of {@link Resource} allocated.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link ContainerId} for the container, which is globally unique.</li>
+ *   <li>
+ *     {@link NodeId} of the node on which it is allocated.
+ *   </li>
+ *   <li>HTTP uri of the node.</li>
+ *   <li>{@link Resource} allocated to the container.</li>
+ *   <li>{@link Priority} at which the container was allocated.</li>
+ *   <li>
+ *     Container {@link Token} of the container, used to securely verify
+ *     authenticity of the allocation.
+ *   </li>
+ * </ul>
  * 
- * <p>Typically, an <code>ApplicationMaster</code> receives the 
- * <code>Container</code> from the <code>ResourceManager</code> during
- * resource-negotiation and then talks to the <code>NodeManager</code> to 
- * start/stop containers.</p>
+ * Typically, an {@code ApplicationMaster} receives the {@code Container}
+ * from the {@code ResourceManager} during resource-negotiation and then
+ * talks to the {@code NodeManager} to start/stop containers.
  * 
  * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
  * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
index a648fef..932945b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
@@ -30,24 +30,23 @@ import org.apache.hadoop.yarn.server.api.AuxiliaryService;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>ContainerLaunchContext</code> represents all of the information
- * needed by the <code>NodeManager</code> to launch a container.</p>
- * 
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link ContainerId} of the container.</li>
- *     <li>{@link Resource} allocated to the container.</li>
- *     <li>User to whom the container is allocated.</li>
- *     <li>Security tokens (if security is enabled).</li>
- *     <li>
- *       {@link LocalResource} necessary for running the container such
- *       as binaries, jar, shared-objects, side-files etc. 
- *     </li>
- *     <li>Optional, application-specific binary service data.</li>
- *     <li>Environment variables for the launched process.</li>
- *     <li>Command to launch the container.</li>
- *   </ul>
- * </p>
+ * {@code ContainerLaunchContext} represents all of the information
+ * needed by the {@code NodeManager} to launch a container.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link ContainerId} of the container.</li>
+ *   <li>{@link Resource} allocated to the container.</li>
+ *   <li>User to whom the container is allocated.</li>
+ *   <li>Security tokens (if security is enabled).</li>
+ *   <li>
+ *     {@link LocalResource} necessary for running the container such
+ *     as binaries, jar, shared-objects, side-files etc.
+ *   </li>
+ *   <li>Optional, application-specific binary service data.</li>
+ *   <li>Environment variables for the launched process.</li>
+ *   <li>Command to launch the container.</li>
+ * </ul>
  * 
  * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
index 72b8edf..11d7bca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
@@ -24,27 +24,22 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
- * <code>ContainerReport</code> is a report of an container.
- * </p>
- * 
+ * {@code ContainerReport} is a report of an container.
  * <p>
  * It includes details such as:
  * <ul>
- * <li>{@link ContainerId} of the container.</li>
- * <li>Allocated Resources to the container.</li>
- * <li>Assigned Node id.</li>
- * <li>Assigned Priority.</li>
- * <li>Creation Time.</li>
- * <li>Finish Time.</li>
- * <li>Container Exit Status.</li>
- * <li>{@link ContainerState} of the container.</li>
- * <li>Diagnostic information in case of errors.</li>
- * <li>Log URL.</li>
- * <li>nodeHttpAddress</li>
+ *   <li>{@link ContainerId} of the container.</li>
+ *   <li>Allocated Resources to the container.</li>
+ *   <li>Assigned Node id.</li>
+ *   <li>Assigned Priority.</li>
+ *   <li>Creation Time.</li>
+ *   <li>Finish Time.</li>
+ *   <li>Container Exit Status.</li>
+ *   <li>{@link ContainerState} of the container.</li>
+ *   <li>Diagnostic information in case of errors.</li>
+ *   <li>Log URL.</li>
+ *   <li>nodeHttpAddress</li>
  * </ul>
- * </p>
- * 
  */
 
 @Public

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
index 574373c..5ccf6dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
@@ -25,17 +25,16 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>ContainerStatus</code> represents the current status of a 
- * <code>Container</code>.</p>
- * 
- * <p>It provides details such as:
- *   <ul>
- *     <li><code>ContainerId</code> of the container.</li>
- *     <li><code>ContainerState</code> of the container.</li>
- *     <li><em>Exit status</em> of a completed container.</li>
- *     <li><em>Diagnostic</em> message for a failed container.</li>
- *   </ul>
- * </p>
+ * {@code ContainerStatus} represents the current status of a
+ * {@code Container}.
+ * <p>
+ * It provides details such as:
+ * <ul>
+ *   <li>{@code ContainerId} of the container.</li>
+ *   <li>{@code ContainerState} of the container.</li>
+ *   <li><em>Exit status</em> of a completed container.</li>
+ *   <li><em>Diagnostic</em> message for a failed container.</li>
+ * </ul>
  */
 @Public
 @Stable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
index d1aa45b..1552cdf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
@@ -23,22 +23,22 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 
 /**
- * <p><code>LocalResourceType</code> specifies the <em>type</em>
- * of a resource localized by the <code>NodeManager</code>.</p> 
- *
- * <p>The <em>type</em> can be one of:
- *   <ul>
- *     <li>
- *       {@link #FILE} - Regular file i.e. uninterpreted bytes. 
- *     </li>
- *     <li>
- *       {@link #ARCHIVE} - Archive, which is automatically unarchived by the 
- *       <code>NodeManager</code>.
- *     </li>
- *     <li>
- *       {@link #PATTERN} - A hybrid between {@link #ARCHIVE} and {@link #FILE}.
- *   </ul>
- * </p>
+ * {@code LocalResourceType} specifies the <em>type</em>
+ * of a resource localized by the {@code NodeManager}.
+ * <p>
+ * The <em>type</em> can be one of:
+ * <ul>
+ *   <li>
+ *     {@link #FILE} - Regular file i.e. uninterpreted bytes.
+ *   </li>
+ *   <li>
+ *     {@link #ARCHIVE} - Archive, which is automatically unarchived by the
+ *     <code>NodeManager</code>.
+ *   </li>
+ *   <li>
+ *     {@link #PATTERN} - A hybrid between {@link #ARCHIVE} and {@link #FILE}.
+ *   </li>
+ * </ul>
  *
  * @see LocalResource
  * @see ContainerLaunchContext

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
index d368bfb..2b71991 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
@@ -23,22 +23,21 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 
 /**
- * <p><code>LocalResourceVisibility</code> specifies the <em>visibility</em>
- * of a resource localized by the <code>NodeManager</code>.</p>
- * 
- * <p>The <em>visibility</em> can be one of:
- *   <ul>
- *     <li>{@link #PUBLIC} - Shared by all users on the node.</li>
- *     <li>
- *       {@link #PRIVATE} - Shared among all applications of the 
- *       <em>same user</em> on the node.
- *     </li>
- *     <li>
- *       {@link #APPLICATION} - Shared only among containers of the 
- *       <em>same application</em> on the node.
- *     </li>
- *   </ul>
- * </p>
+ * {@code LocalResourceVisibility} specifies the <em>visibility</em>
+ * of a resource localized by the {@code NodeManager}.
+ * <p>
+ * The <em>visibility</em> can be one of:
+ * <ul>
+ *   <li>{@link #PUBLIC} - Shared by all users on the node.</li>
+ *   <li>
+ *     {@link #PRIVATE} - Shared among all applications of the
+ *     <em>same user</em> on the node.
+ *   </li>
+ *   <li>
+ *     {@link #APPLICATION} - Shared only among containers of the
+ *     <em>same application</em> on the node.
+ *   </li>
+ * </ul>
  * 
  * @see LocalResource
  * @see ContainerLaunchContext

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
index e582d2c..9383004 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
@@ -24,30 +24,37 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>LogAggregationContext</code> represents all of the
- * information needed by the <code>NodeManager</code> to handle
- * the logs for an application.</p>
- *
- * <p>It includes details such as:
- *   <ul>
- *     <li>includePattern. It uses Java Regex to filter the log files
+ * {@code LogAggregationContext} represents all of the
+ * information needed by the {@code NodeManager} to handle
+ * the logs for an application.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>
+ *     includePattern. It uses Java Regex to filter the log files
  *     which match the defined include pattern and those log files
- *     will be uploaded when the application finishes. </li>
- *     <li>excludePattern. It uses Java Regex to filter the log files
+ *     will be uploaded when the application finishes.
+ *   </li>
+ *   <li>
+ *     excludePattern. It uses Java Regex to filter the log files
  *     which match the defined exclude pattern and those log files
  *     will not be uploaded when application finishes. If the log file
  *     name matches both the include and the exclude pattern, this file
- *     will be excluded eventually</li>
- *     <li>rolledLogsIncludePattern. It uses Java Regex to filter the log files
+ *     will be excluded eventually.
+ *   </li>
+ *   <li>
+ *     rolledLogsIncludePattern. It uses Java Regex to filter the log files
  *     which match the defined include pattern and those log files
- *     will be aggregated in a rolling fashion.</li>
- *     <li>rolledLogsExcludePattern. It uses Java Regex to filter the log files
+ *     will be aggregated in a rolling fashion.
+ *   </li>
+ *   <li>
+ *     rolledLogsExcludePattern. It uses Java Regex to filter the log files
  *     which match the defined exclude pattern and those log files
  *     will not be aggregated in a rolling fashion. If the log file
  *     name matches both the include and the exclude pattern, this file
- *     will be excluded eventually</li>
- *   </ul>
- * </p>
+ *     will be excluded eventually.
+ *   </li>
+ * </ul>
  *
  * @see ApplicationSubmissionContext
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
index 7ba866d..aac19bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
@@ -28,19 +28,18 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>NodeReport</code> is a summary of runtime information of a 
- * node in the cluster.</p>
- * 
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link NodeId} of the node.</li>
- *     <li>HTTP Tracking URL of the node.</li>
- *     <li>Rack name for the node.</li>
- *     <li>Used {@link Resource} on the node.</li>
- *     <li>Total available {@link Resource} of the node.</li>
- *     <li>Number of running containers on the node.</li>
- *   </ul>
- * </p>
+ * {@code NodeReport} is a summary of runtime information of a node
+ * in the cluster.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link NodeId} of the node.</li>
+ *   <li>HTTP Tracking URL of the node.</li>
+ *   <li>Rack name for the node.</li>
+ *   <li>Used {@link Resource} on the node.</li>
+ *   <li>Total available {@link Resource} of the node.</li>
+ *   <li>Number of running containers on the node.</li>
+ * </ul>
  *
  * @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
index 976d181..cdf34f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
@@ -24,36 +24,36 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>A {@link PreemptionMessage} is part of the RM-AM protocol, and it is used by
+ * A {@link PreemptionMessage} is part of the RM-AM protocol, and it is used by
  * the RM to specify resources that the RM wants to reclaim from this
- * <code>ApplicationMaster</code> (AM). The AM receives a {@link
+ * {@code ApplicationMaster} (AM). The AM receives a {@link
  * StrictPreemptionContract} message encoding which containers the platform may
  * forcibly kill, granting it an opportunity to checkpoint state or adjust its
  * execution plan. The message may also include a {@link PreemptionContract}
  * granting the AM more latitude in selecting which resources to return to the
- * cluster.<p>
- *
- * <p>The AM should decode both parts of the message. The {@link
+ * cluster.
+ * <p>
+ * The AM should decode both parts of the message. The {@link
  * StrictPreemptionContract} specifies particular allocations that the RM
  * requires back. The AM can checkpoint containers' state, adjust its execution
  * plan to move the computation, or take no action and hope that conditions that
- * caused the RM to ask for the container will change.<p>
- *
- * <p>In contrast, the {@link PreemptionContract} also includes a description of
+ * caused the RM to ask for the container will change.
+ * <p>
+ * In contrast, the {@link PreemptionContract} also includes a description of
  * resources with a set of containers. If the AM releases containers matching
  * that profile, then the containers enumerated in {@link
- * PreemptionContract#getContainers()} may not be killed.<p>
- *
- * <p>Each preemption message reflects the RM's current understanding of the
- * cluster state, so a request to return <emph>N</emph> containers may not
+ * PreemptionContract#getContainers()} may not be killed.
+ * <p>
+ * Each preemption message reflects the RM's current understanding of the
+ * cluster state, so a request to return <em>N</em> containers may not
  * reflect containers the AM is releasing, recently exited containers the RM has
  * yet to learn about, or new containers allocated before the message was
  * generated. Conversely, an RM may request a different profile of containers in
- * subsequent requests.<p>
- *
- * <p>The policy enforced by the RM is part of the scheduler. Generally, only
+ * subsequent requests.
+ * <p>
+ * The policy enforced by the RM is part of the scheduler. Generally, only
  * containers that have been requested consistently should be killed, but the
- * details are not specified.<p>
+ * details are not specified.
  */
 @Public
 @Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java
index c6777db..585faf8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java
@@ -23,18 +23,15 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 
 /**
- * <p>
- * <code>QueueACL</code> enumerates the various ACLs for queues.
- * </p>
- * 
+ * {@code QueueACL} enumerates the various ACLs for queues.
  * <p>
  * The ACL is one of:
  * <ul>
- * <li>{@link #SUBMIT_APPLICATIONS} - ACL to submit applications to the
- * queue.</li>
- * <li>{@link #ADMINISTER_QUEUE} - ACL to administer the queue.</li>
+ *   <li>
+ *     {@link #SUBMIT_APPLICATIONS} - ACL to submit applications to the queue.
+ *   </li>
+ *   <li>{@link #ADMINISTER_QUEUE} - ACL to administer the queue.</li>
  * </ul>
- * </p>
  * 
  * @see QueueInfo
  * @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
index d762b41..bee5275 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
@@ -29,19 +29,18 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>QueueInfo is a report of the runtime information of the queue.</p>
- * 
- * <p>It includes information such as:
- *   <ul>
- *     <li>Queue name.</li>
- *     <li>Capacity of the queue.</li>
- *     <li>Maximum capacity of the queue.</li>
- *     <li>Current capacity of the queue.</li>
- *     <li>Child queues.</li>
- *     <li>Running applications.</li>
- *     <li>{@link QueueState} of the queue.</li>
- *   </ul>
- * </p>
+ * QueueInfo is a report of the runtime information of the queue.
+ * <p>
+ * It includes information such as:
+ * <ul>
+ *   <li>Queue name.</li>
+ *   <li>Capacity of the queue.</li>
+ *   <li>Maximum capacity of the queue.</li>
+ *   <li>Current capacity of the queue.</li>
+ *   <li>Child queues.</li>
+ *   <li>Running applications.</li>
+ *   <li>{@link QueueState} of the queue.</li>
+ * </ul>
  *
  * @see QueueState
  * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java
index 01698de..2bc0407 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java
@@ -23,14 +23,13 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 
 /**
- * <p>State of a Queue.</p>
- * 
- * <p>A queue is in one of:
- *   <ul>
- *     <li>{@link #RUNNING} - normal state.</li> 
- *     <li>{@link #STOPPED} - not accepting new application submissions.
- *   </ul>
- * </p>
+ * State of a Queue.
+ * <p>
+ * A queue is in one of:
+ * <ul>
+ *   <li>{@link #RUNNING} - normal state.</li>
+ *   <li>{@link #STOPPED} - not accepting new application submissions.</li>
+ * </ul>
  * 
  * @see QueueInfo
  * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
index a4f0585..4ebe1c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
@@ -25,23 +25,18 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * {@link ReservationRequest} represents the request made by an application to
  * the {@code ResourceManager} to reserve {@link Resource}s.
- * </p>
- * 
  * <p>
  * It includes:
  * <ul>
- * <li>{@link Resource} required for each request.</li>
- * <li>
- * Number of containers, of above specifications, which are required by the
- * application.</li>
- * <li>
- * Concurrency that indicates the gang size of the request.</li>
+ *   <li>{@link Resource} required for each request.</li>
+ *   <li>
+ *     Number of containers, of above specifications, which are required by the
+ *     application.
+ *   </li>
+ *   <li>Concurrency that indicates the gang size of the request.</li>
  * </ul>
- * </p>
- * 
  */
 @Public
 @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
index 1ee96c2..3b82626 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
@@ -33,14 +33,13 @@ public enum ReservationRequestInterpreter {
    * Requires that exactly ONE among the {@link ReservationRequest} submitted as
    * of a {@link ReservationDefinition} is satisfied to satisfy the overall
    * {@link ReservationDefinition}.
-   * 
+   * <p>
    * WHEN TO USE THIS: This is useful when the user have multiple equivalent
    * ways to run an application, and wants to expose to the ReservationAgent
-   * such flexibility. For example an application could use one <32GB,16core>
-   * container for 10min, or 16 <2GB,1core> containers for 15min, the
-   * ReservationAgent will decide which one of the two it is best for the system
-   * to place.
-   * 
+   * such flexibility. For example an application could use one
+   * {@literal <32GB,16core>} container for 10min, or 16 {@literal <2GB,1core>}
+   * containers for 15min, the ReservationAgent will decide which one of the
+   * two it is best for the system to place.
    */
   R_ANY,
 
@@ -49,16 +48,16 @@ public enum ReservationRequestInterpreter {
    * {@link ReservationDefinition} are satisfied for the overall
    * {@link ReservationDefinition} to be satisfied. No constraints are imposed
    * on the temporal ordering of the allocation used to satisfy the
-   * ResourceRequeusts.
-   * 
+   * ResourceRequests.
+   * <p>
    * WHEN TO USE THIS: This is useful to capture a scenario in which the user
    * cares for multiple ReservationDefinition to be all accepted, or none. For
-   * example, a user might want a reservation R1: with 10 x <8GB,4core> for
-   * 10min, and a reservation R2: with 2 <1GB,1core> for 1h, and only if both
-   * are satisfied the workflow run in this reservation succeeds. The key
-   * differentiator from ALL and ORDER, ORDER_NO_GAP, is that ALL imposes no
-   * restrictions on the relative allocations used to place R1 and R2 above.
-   * 
+   * example, a user might want a reservation R1: with 10 x
+   * {@literal <8GB,4core>} for 10min, and a reservation R2:
+   * with 2 {@literal <1GB,1core>} for 1h, and only if both are satisfied
+   * the workflow run in this reservation succeeds. The key differentiator
+   * from ALL and ORDER, ORDER_NO_GAP, is that ALL imposes no restrictions
+   * on the relative allocations used to place R1 and R2 above.
    */
   R_ALL,
 
@@ -73,15 +72,16 @@ public enum ReservationRequestInterpreter {
    * constraints are imposed on temporal gaps between subsequent allocations
    * (the last instant of the previous allocation can be an arbitrary long
    * period of time before the first instant of the subsequent allocation).
-   * 
+   * <p>
    * WHEN TO USE THIS: Like ALL this requires all ReservationDefinitions to be
    * placed, but it also imposes a time ordering on the allocations used. This
    * is important if the ReservationDefinition(s) are used to describe a
    * workflow with inherent inter-stage dependencies. For example, a first job
-   * runs in a ReservaitonDefinition R1 (10 x <1GB,1core> for 20min), and its
-   * output is consumed by a second job described by a ReservationDefinition R2
-   * (5 x <1GB,1core>) for 50min). R2 allocation cannot overlap R1, as R2 models
-   * a job depending on the output of the job modeled by R1.
+   * runs in a ReservaitonDefinition R1 (10 x {@literal <1GB,1core>}
+   * for 20min), and its output is consumed by a second job described by
+   * a ReservationDefinition R2 (5 x {@literal <1GB,1core>}) for 50min).
+   * R2 allocation cannot overlap R1, as R2 models a job depending on
+   * the output of the job modeled by R1.
    */
   R_ORDER,
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
index 2f17ac9..790120a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
@@ -27,31 +27,30 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>ResourceRequest</code> represents the request made by an
- * application to the <code>ResourceManager</code> to obtain various 
- * <code>Container</code> allocations.</p>
- * 
- * <p>It includes:
- *   <ul>
- *     <li>{@link Priority} of the request.</li>
- *     <li>
- *       The <em>name</em> of the machine or rack on which the allocation is 
- *       desired. A special value of <em>*</em> signifies that 
- *       <em>any</em> host/rack is acceptable to the application.
- *     </li>
- *     <li>{@link Resource} required for each request.</li>
- *     <li>
- *       Number of containers, of above specifications, which are required 
- *       by the application.
- *     </li>
- *     <li>
- *       A boolean <em>relaxLocality</em> flag, defaulting to <code>true</code>,
- *       which tells the <code>ResourceManager</code> if the application wants
- *       locality to be loose (i.e. allows fall-through to rack or <em>any</em>)
- *       or strict (i.e. specify hard constraint on resource allocation).
- *     </li>
- *   </ul>
- * </p>
+ * {@code ResourceRequest} represents the request made
+ * by an application to the {@code ResourceManager}
+ * to obtain various {@code Container} allocations.
+ * <p>
+ * It includes:
+ * <ul>
+ *   <li>{@link Priority} of the request.</li>
+ *   <li>
+ *     The <em>name</em> of the machine or rack on which the allocation is
+ *     desired. A special value of <em>*</em> signifies that
+ *     <em>any</em> host/rack is acceptable to the application.
+ *   </li>
+ *   <li>{@link Resource} required for each request.</li>
+ *   <li>
+ *     Number of containers, of above specifications, which are required
+ *     by the application.
+ *   </li>
+ *   <li>
+ *     A boolean <em>relaxLocality</em> flag, defaulting to {@code true},
+ *     which tells the {@code ResourceManager} if the application wants
+ *     locality to be loose (i.e. allows fall-through to rack or <em>any</em>)
+ *     or strict (i.e. specify hard constraint on resource allocation).
+ *   </li>
+ * </ul>
  * 
  * @see Resource
  * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
@@ -267,7 +266,7 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
   /**
    * Set node label expression of this resource request. Now only support
    * specifying a single node label. In the future we will support more complex
-   * node label expression specification like AND(&&), OR(||), etc.
+   * node label expression specification like {@code AND(&&), OR(||)}, etc.
    * 
    * Any please note that node label expression now can only take effect when
    * the resource request has resourceName = ANY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/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 8c83fea..f40c999 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
@@ -733,8 +733,9 @@ public class YarnConfiguration extends Configuration {
   
   /**
    * How long to wait between aggregated log retention checks. If set to
-   * a value <= 0 then the value is computed as one-tenth of the log retention
-   * setting. Be careful set this too small and you will spam the name node.
+   * a value {@literal <=} 0 then the value is computed as one-tenth of the
+   * log retention setting. Be careful set this too small and you will spam
+   * the name node.
    */
   public static final String LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS =
       YARN_PREFIX + "log-aggregation.retain-check-interval-seconds";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java
index d1ab781..d540cce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java
@@ -54,7 +54,7 @@ public abstract class UpdateNodeResourceRequest {
   
   /**
    * Get the map from <code>NodeId</code> to <code>ResourceOption</code>.
-   * @return the map of <NodeId, ResourceOption>
+   * @return the map of {@code <NodeId, ResourceOption>}
    */
   @Public
   @Evolving
@@ -62,7 +62,7 @@ public abstract class UpdateNodeResourceRequest {
   
   /**
    * Set the map from <code>NodeId</code> to <code>ResourceOption</code>.
-   * @param nodeResourceMap the map of <NodeId, ResourceOption>
+   * @param nodeResourceMap the map of {@code <NodeId, ResourceOption>}
    */
   @Public
   @Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java
index b3c5308..b590a51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java
@@ -56,28 +56,22 @@ public abstract class AHSClient extends AbstractService {
   }
 
   /**
-   * <p>
    * Get a report of the given Application.
-   * </p>
-   * 
    * <p>
    * In secure mode, <code>YARN</code> verifies access to the application, queue
    * etc. before accepting the request.
-   * </p>
-   * 
    * <p>
    * If the user does not have <code>VIEW_APP</code> access then the following
    * fields in the report will be set to stubbed values:
    * <ul>
-   * <li>host - set to "N/A"</li>
-   * <li>RPC port - set to -1</li>
-   * <li>client token - set to "N/A"</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL - set to "N/A"</li>
-   * <li>original tracking URL - set to "N/A"</li>
-   * <li>resource usage report - all values are -1</li>
+   *   <li>host - set to "N/A"</li>
+   *   <li>RPC port - set to -1</li>
+   *   <li>client token - set to "N/A"</li>
+   *   <li>diagnostics - set to "N/A"</li>
+   *   <li>tracking URL - set to "N/A"</li>
+   *   <li>original tracking URL - set to "N/A"</li>
+   *   <li>resource usage report - all values are -1</li>
    * </ul>
-   * </p>
    * 
    * @param appId
    *          {@link ApplicationId} of the application that needs a report
@@ -121,7 +115,7 @@ public abstract class AHSClient extends AbstractService {
    *          a report
    * @return application attempt report
    * @throws YarnException
-   * @throws {@link ApplicationAttemptNotFoundException} if application attempt
+   * @throws ApplicationAttemptNotFoundException if application attempt
    *         not found
    * @throws IOException
    */
@@ -157,7 +151,7 @@ public abstract class AHSClient extends AbstractService {
    *          {@link ContainerId} of the container that needs a report
    * @return container report
    * @throws YarnException
-   * @throws {@link ContainerNotFoundException} if container not found
+   * @throws ContainerNotFoundException if container not found
    * @throws IOException
    */
   public abstract ContainerReport getContainerReport(ContainerId containerId)


[26/50] hadoop git commit: MAPREDUCE-6105. nconsistent configuration in property mapreduce.reduce.shuffle.merge.percent. Contributed by Ray Chiang.

Posted by zj...@apache.org.
MAPREDUCE-6105. nconsistent configuration in property mapreduce.reduce.shuffle.merge.percent. Contributed by Ray Chiang.


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

Branch: refs/heads/YARN-2928
Commit: 685dbafbe2154e5bf4b638da0668ce32d8c879b0
Parents: ce5de93
Author: Harsh J <ha...@cloudera.com>
Authored: Tue Mar 17 01:17:34 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Tue Mar 17 02:28:09 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                            | 3 +++
 .../src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java  | 1 +
 .../apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java   | 5 +++--
 3 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/685dbafb/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index d02d725..52880f6 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -253,6 +253,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-6105. Inconsistent configuration in property
+    mapreduce.reduce.shuffle.merge.percent. (Ray Chiang via harsh)
+
     MAPREDUCE-4414. Add main methods to JobConf and YarnConfiguration,
     for debug purposes. (Plamen Jeliazkov via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/685dbafb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 3aa304a..f0a6ddf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -305,6 +305,7 @@ public interface MRJobConfig {
     = "mapreduce.reduce.shuffle.memory.limit.percent";
 
   public static final String SHUFFLE_MERGE_PERCENT = "mapreduce.reduce.shuffle.merge.percent";
+  public static final float DEFAULT_SHUFFLE_MERGE_PERCENT = 0.66f;
 
   public static final String REDUCE_FAILURES_MAXPERCENT = "mapreduce.reduce.failures.maxpercent";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/685dbafb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
index a4b1aa8..8bf17ef 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
@@ -191,8 +191,9 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
     this.memToMemMergeOutputsThreshold = 
             jobConf.getInt(MRJobConfig.REDUCE_MEMTOMEM_THRESHOLD, ioSortFactor);
     this.mergeThreshold = (long)(this.memoryLimit * 
-                          jobConf.getFloat(MRJobConfig.SHUFFLE_MERGE_PERCENT, 
-                                           0.90f));
+                          jobConf.getFloat(
+                            MRJobConfig.SHUFFLE_MERGE_PERCENT,
+                            MRJobConfig.DEFAULT_SHUFFLE_MERGE_PERCENT));
     LOG.info("MergerManager: memoryLimit=" + memoryLimit + ", " +
              "maxSingleShuffleLimit=" + maxSingleShuffleLimit + ", " +
              "mergeThreshold=" + mergeThreshold + ", " + 


[03/50] hadoop git commit: YARN-3267. Timelineserver applies the ACL rules after applying the limit on the number of records (Chang Li via jeagles)

Posted by zj...@apache.org.
YARN-3267. Timelineserver applies the ACL rules after applying the limit on the number of records (Chang Li via jeagles)


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

Branch: refs/heads/YARN-2928
Commit: 8180e676abb2bb500a48b3a0c0809d2a807ab235
Parents: 387f271
Author: Jonathan Eagles <je...@gmail.com>
Authored: Fri Mar 13 12:04:30 2015 -0500
Committer: Jonathan Eagles <je...@gmail.com>
Committed: Fri Mar 13 12:04:30 2015 -0500

----------------------------------------------------------------------
 .../jobhistory/TestJobHistoryEventHandler.java  | 14 +++---
 .../mapred/TestMRTimelineEventHandling.java     | 12 ++---
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../distributedshell/TestDistributedShell.java  |  4 +-
 .../server/timeline/LeveldbTimelineStore.java   | 18 +++++--
 .../server/timeline/MemoryTimelineStore.java    | 12 ++++-
 .../server/timeline/TimelineDataManager.java    | 50 +++++++++++---------
 .../yarn/server/timeline/TimelineReader.java    |  3 +-
 .../timeline/TestLeveldbTimelineStore.java      | 16 +++----
 .../timeline/TestTimelineDataManager.java       | 26 +++++++++-
 .../server/timeline/TimelineStoreTestUtils.java | 33 +++++++++----
 11 files changed, 126 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index de35d84..43e3dbe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
@@ -464,7 +464,7 @@ public class TestJobHistoryEventHandler {
               t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000),
               currentTime - 10));
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
-              null, null, null, null, null, null);
+              null, null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -480,7 +480,7 @@ public class TestJobHistoryEventHandler {
               new HashMap<JobACL, AccessControlList>(), "default"),
               currentTime + 10));
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -498,7 +498,7 @@ public class TestJobHistoryEventHandler {
               new JobQueueChangeEvent(TypeConverter.fromYarn(t.jobId), "q2"),
               currentTime - 20));
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -520,7 +520,7 @@ public class TestJobHistoryEventHandler {
               new JobFinishedEvent(TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0,
               0, new Counters(), new Counters(), new Counters()), currentTime));
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -546,7 +546,7 @@ public class TestJobHistoryEventHandler {
             new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId),
             0, 0, 0, JobStateInternal.KILLED.toString()), currentTime + 20));
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -575,7 +575,7 @@ public class TestJobHistoryEventHandler {
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
             new TaskStartedEvent(t.taskID, 0, TaskType.MAP, "")));
       entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());
@@ -588,7 +588,7 @@ public class TestJobHistoryEventHandler {
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
             new TaskStartedEvent(t.taskID, 0, TaskType.REDUCE, "")));
       entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
index 346953f..c2ef128 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
@@ -55,7 +55,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
               job.getJobStatus().getState().getValue());
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
-              null, null, null, null, null, null);
+              null, null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
       Assert.assertEquals(job.getID().toString(), tEntity.getEntityId());
@@ -70,7 +70,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.FAILED,
               job.getJobStatus().getState().getValue());
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null,
-              null, null, null);
+              null, null, null, null);
       Assert.assertEquals(2, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(job.getID().toString(), tEntity.getEntityId());
@@ -109,7 +109,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
-          null, null, null, null, null, null);
+          null, null, null, null, null, null, null);
       Assert.assertEquals(0, entities.getEntities().size());
 
       conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
@@ -117,7 +117,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null,
-          null, null, null);
+          null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
       Assert.assertEquals(job.getID().toString(), tEntity.getEntityId());
@@ -148,7 +148,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
-          null, null, null, null, null, null);
+          null, null, null, null, null, null, null);
       Assert.assertEquals(0, entities.getEntities().size());
 
       conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
@@ -156,7 +156,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null,
-          null, null, null);
+          null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
       Assert.assertEquals(job.getID().toString(), tEntity.getEntityId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 80d2697..94f992d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -760,6 +760,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3154. Added additional APIs in LogAggregationContext to avoid aggregating
     running logs of application when rolling is enabled. (Xuan Gong via vinodkv)
 
+    YARN-3267. Timelineserver applies the ACL rules after applying the limit on
+    the number of records (Chang Li via jeagles)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 46b5850..5e6fa46 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -243,7 +243,7 @@ public class TestDistributedShell {
         .getApplicationHistoryServer()
         .getTimelineStore()
         .getEntities(ApplicationMaster.DSEntity.DS_APP_ATTEMPT.toString(),
-            null, null, null, null, null, null, null, null);
+            null, null, null, null, null, null, null, null, null);
     Assert.assertNotNull(entitiesAttempts);
     Assert.assertEquals(1, entitiesAttempts.getEntities().size());
     Assert.assertEquals(2, entitiesAttempts.getEntities().get(0).getEvents()
@@ -261,7 +261,7 @@ public class TestDistributedShell {
         .getApplicationHistoryServer()
         .getTimelineStore()
         .getEntities(ApplicationMaster.DSEntity.DS_CONTAINER.toString(), null,
-            null, null, null, null, null, null, null);
+            null, null, null, null, null, null, null, null);
     Assert.assertNotNull(entities);
     Assert.assertEquals(2, entities.getEntities().size());
     Assert.assertEquals(entities.getEntities().get(0).getEntityType()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
index 9fd2cfc..d521f70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
 import org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils.KeyBuilder;
 import org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils.KeyParser;
 import org.apache.hadoop.yarn.server.utils.LeveldbIterator;
@@ -56,6 +57,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.yarn.server.timeline.GenericObjectMapper.readReverseOrderedLong;
 import static org.apache.hadoop.yarn.server.timeline.GenericObjectMapper.writeReverseOrderedLong;
+import static org.apache.hadoop.yarn.server.timeline.TimelineDataManager.DEFAULT_DOMAIN_ID;
 import static org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils.prefixMatches;
 import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 
@@ -549,12 +551,13 @@ public class LeveldbTimelineStore extends AbstractService
   public TimelineEntities getEntities(String entityType,
       Long limit, Long windowStart, Long windowEnd, String fromId, Long fromTs,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fields) throws IOException {
+      EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
     if (primaryFilter == null) {
       // if no primary filter is specified, prefix the lookup with
       // ENTITY_ENTRY_PREFIX
       return getEntityByTime(ENTITY_ENTRY_PREFIX, entityType, limit,
-          windowStart, windowEnd, fromId, fromTs, secondaryFilters, fields);
+          windowStart, windowEnd, fromId, fromTs, secondaryFilters, 
+          fields, checkAcl);
     } else {
       // if a primary filter is specified, prefix the lookup with
       // INDEXED_ENTRY_PREFIX + primaryFilterName + primaryFilterValue +
@@ -564,7 +567,7 @@ public class LeveldbTimelineStore extends AbstractService
           .add(GenericObjectMapper.write(primaryFilter.getValue()), true)
           .add(ENTITY_ENTRY_PREFIX).getBytesForLookup();
       return getEntityByTime(base, entityType, limit, windowStart, windowEnd,
-          fromId, fromTs, secondaryFilters, fields);
+          fromId, fromTs, secondaryFilters, fields, checkAcl);
     }
   }
 
@@ -586,7 +589,7 @@ public class LeveldbTimelineStore extends AbstractService
   private TimelineEntities getEntityByTime(byte[] base,
       String entityType, Long limit, Long starttime, Long endtime,
       String fromId, Long fromTs, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fields) throws IOException {
+      EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
     LeveldbIterator iterator = null;
     try {
       KeyBuilder kb = KeyBuilder.newInstance().add(base).add(entityType);
@@ -683,7 +686,12 @@ public class LeveldbTimelineStore extends AbstractService
           }
         }
         if (filterPassed) {
-          entities.addEntity(entity);
+          if (entity.getDomainId() == null) {
+            entity.setDomainId(DEFAULT_DOMAIN_ID);
+          }
+          if (checkAcl == null || checkAcl.check(entity)) {
+            entities.addEntity(entity);
+          }
         }
       }
       return entities;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
index 9c5419e..3489114 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
@@ -47,6 +47,9 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
+
+import static org.apache.hadoop.yarn.server.timeline.TimelineDataManager.DEFAULT_DOMAIN_ID;
 
 /**
  * In-memory implementation of {@link TimelineStore}. This
@@ -79,7 +82,7 @@ public class MemoryTimelineStore
   public synchronized TimelineEntities getEntities(String entityType, Long limit,
       Long windowStart, Long windowEnd, String fromId, Long fromTs,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fields) {
+      EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
     if (limit == null) {
       limit = DEFAULT_LIMIT;
     }
@@ -146,7 +149,12 @@ public class MemoryTimelineStore
           continue;
         }
       }
-      entitiesSelected.add(entity);
+      if (entity.getDomainId() == null) {
+        entity.setDomainId(DEFAULT_DOMAIN_ID);
+      }
+      if (checkAcl == null || checkAcl.check(entity)) {
+        entitiesSelected.add(entity);
+      }
     }
     List<TimelineEntity> entitiesToReturn = new ArrayList<TimelineEntity>();
     for (TimelineEntity entitySelected : entitiesSelected) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
index 888c283..8c6b83a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
@@ -90,6 +90,31 @@ public class TimelineDataManager extends AbstractService {
     super.serviceInit(conf);
   }
 
+  public interface CheckAcl {
+    boolean check(TimelineEntity entity) throws IOException;
+  }
+
+  class CheckAclImpl implements CheckAcl {
+    final UserGroupInformation ugi;
+
+    public CheckAclImpl(UserGroupInformation callerUGI) {
+      ugi = callerUGI;
+    }
+
+    public boolean check(TimelineEntity entity) throws IOException {
+      try{
+        return timelineACLsManager.checkAccess(
+          ugi, ApplicationAccessType.VIEW_APP, entity);
+      } catch (YarnException e) {
+        LOG.info("Error when verifying access for user " + ugi
+          + " on the events of the timeline entity "
+          + new EntityIdentifier(entity.getEntityId(),
+          entity.getEntityType()), e);
+        return false;
+      }
+    }
+  }
+
   /**
    * Get the timeline entities that the given user have access to. The meaning
    * of each argument has been documented with
@@ -118,28 +143,9 @@ public class TimelineDataManager extends AbstractService {
         fromTs,
         primaryFilter,
         secondaryFilter,
-        fields);
-    if (entities != null) {
-      Iterator<TimelineEntity> entitiesItr =
-          entities.getEntities().iterator();
-      while (entitiesItr.hasNext()) {
-        TimelineEntity entity = entitiesItr.next();
-        addDefaultDomainIdIfAbsent(entity);
-        try {
-          // check ACLs
-          if (!timelineACLsManager.checkAccess(
-              callerUGI, ApplicationAccessType.VIEW_APP, entity)) {
-            entitiesItr.remove();
-          }
-        } catch (YarnException e) {
-          LOG.error("Error when verifying access for user " + callerUGI
-              + " on the events of the timeline entity "
-              + new EntityIdentifier(entity.getEntityId(),
-                  entity.getEntityType()), e);
-          entitiesItr.remove();
-        }
-      }
-    }
+        fields,
+        new CheckAclImpl(callerUGI));
+
     if (entities == null) {
       return new TimelineEntities();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
index aba1ba2..012f4fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
 
 /**
  * This interface is for retrieving timeline information.
@@ -106,7 +107,7 @@ public interface TimelineReader {
   TimelineEntities getEntities(String entityType,
       Long limit, Long windowStart, Long windowEnd, String fromId, Long fromTs,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fieldsToRetrieve) throws IOException;
+      EnumSet<Field> fieldsToRetrieve, CheckAcl checkAcl) throws IOException;
 
   /**
    * This method retrieves the entity information for a given entity.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
index 15edecd..c5c0f93 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
@@ -164,13 +164,13 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
   @Test
   public void testGetEntityTypes() throws IOException {
     List<String> entityTypes = ((LeveldbTimelineStore)store).getEntityTypes();
-    assertEquals(6, entityTypes.size());
-    assertEquals("OLD_ENTITY_TYPE_1", entityTypes.get(0));
-    assertEquals(entityType1, entityTypes.get(1));
-    assertEquals(entityType2, entityTypes.get(2));
-    assertEquals(entityType4, entityTypes.get(3));
-    assertEquals(entityType5, entityTypes.get(4));
-    assertEquals(entityType7, entityTypes.get(5));
+    assertEquals(7, entityTypes.size());
+    assertEquals("ACL_ENTITY_TYPE_1", entityTypes.get(0));
+    assertEquals("OLD_ENTITY_TYPE_1", entityTypes.get(1));
+    assertEquals(entityType1, entityTypes.get(2));
+    assertEquals(entityType2, entityTypes.get(3));
+    assertEquals(entityType4, entityTypes.get(4));
+    assertEquals(entityType5, entityTypes.get(5));
   }
 
   @Test
@@ -201,7 +201,7 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     ((LeveldbTimelineStore)store).discardOldEntities(-123l);
     assertEquals(2, getEntities("type_1").size());
     assertEquals(0, getEntities("type_2").size());
-    assertEquals(5, ((LeveldbTimelineStore)store).getEntityTypes().size());
+    assertEquals(6, ((LeveldbTimelineStore)store).getEntityTypes().size());
 
     ((LeveldbTimelineStore)store).discardOldEntities(123l);
     assertEquals(0, getEntities("type_1").size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
index f749567..87c3b24 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.AdminACLsManager;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.junit.After;
 import org.junit.Assert;
@@ -40,7 +41,8 @@ public class TestTimelineDataManager extends TimelineStoreTestUtils {
   private FileContext fsContext;
   private File fsPath;
   private TimelineDataManager dataManaer;
-
+  private static TimelineACLsManager aclsManager;
+  private static AdminACLsManager adminACLsManager;
   @Before
   public void setup() throws Exception {
     fsPath = new File("target", this.getClass().getSimpleName() +
@@ -58,8 +60,12 @@ public class TestTimelineDataManager extends TimelineStoreTestUtils {
     loadVerificationEntityData();
     loadTestDomainData();
 
-    TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, false);
+    aclsManager = new TimelineACLsManager(conf);
     dataManaer = new TimelineDataManager(store, aclsManager);
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    conf.set(YarnConfiguration.YARN_ADMIN_ACL, "admin");
+    adminACLsManager = new AdminACLsManager(conf);
   }
 
   @After
@@ -85,6 +91,22 @@ public class TestTimelineDataManager extends TimelineStoreTestUtils {
   }
 
   @Test
+  public void testGetEntitiesAclEnabled() throws Exception {
+    AdminACLsManager oldAdminACLsManager =
+      aclsManager.setAdminACLsManager(adminACLsManager);
+    try {
+      TimelineEntities entities = dataManaer.getEntities(
+        "ACL_ENTITY_TYPE_1", null, null, null, null, null, null, 1l, null,
+        UserGroupInformation.createUserForTesting("owner_1", new String[] {"group1"}));
+      Assert.assertEquals(1, entities.getEntities().size());
+      Assert.assertEquals("ACL_ENTITY_ID_11",
+        entities.getEntities().get(0).getEntityId());
+    } finally {
+      aclsManager.setAdminACLsManager(oldAdminACLsManager);
+    }
+  }
+
+  @Test
   public void testGetOldEntitiesWithOutDomainId() throws Exception {
     TimelineEntities entities = dataManaer.getEntities(
         "OLD_ENTITY_TYPE_1", null, null, null, null, null, null, null, null,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
index c99786d..da71f46 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
@@ -353,6 +353,19 @@ public class TimelineStoreTestUtils {
     domain3.setReaders("reader_user_4 reader_group_4");
     domain3.setWriters("writer_user_4 writer_group_4");
     store.put(domain3);
+
+    TimelineEntities entities = new TimelineEntities();
+    if (store instanceof LeveldbTimelineStore) {
+      LeveldbTimelineStore leveldb = (LeveldbTimelineStore) store;
+      entities.setEntities(Collections.singletonList(createEntity(
+              "ACL_ENTITY_ID_11", "ACL_ENTITY_TYPE_1", 63l, null, null, null, null,
+              "domain_id_4")));
+      leveldb.put(entities);
+      entities.setEntities(Collections.singletonList(createEntity(
+              "ACL_ENTITY_ID_22", "ACL_ENTITY_TYPE_1", 64l, null, null, null, null,
+              "domain_id_2")));
+      leveldb.put(entities);
+    }
   }
 
   public void testGetSingleEntity() throws IOException {
@@ -419,66 +432,66 @@ public class TimelineStoreTestUtils {
   protected List<TimelineEntity> getEntities(String entityType)
       throws IOException {
     return store.getEntities(entityType, null, null, null, null, null,
-        null, null, null).getEntities();
+        null, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesWithPrimaryFilter(
       String entityType, NameValuePair primaryFilter) throws IOException {
     return store.getEntities(entityType, null, null, null, null, null,
-        primaryFilter, null, null).getEntities();
+        primaryFilter, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromId(String entityType,
       String fromId) throws IOException {
     return store.getEntities(entityType, null, null, null, fromId, null,
-        null, null, null).getEntities();
+        null, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromTs(String entityType,
       long fromTs) throws IOException {
     return store.getEntities(entityType, null, null, null, null, fromTs,
-        null, null, null).getEntities();
+        null, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromIdWithPrimaryFilter(
       String entityType, NameValuePair primaryFilter, String fromId)
       throws IOException {
     return store.getEntities(entityType, null, null, null, fromId, null,
-        primaryFilter, null, null).getEntities();
+        primaryFilter, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromTsWithPrimaryFilter(
       String entityType, NameValuePair primaryFilter, long fromTs)
       throws IOException {
     return store.getEntities(entityType, null, null, null, null, fromTs,
-        primaryFilter, null, null).getEntities();
+        primaryFilter, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromIdWithWindow(String entityType,
       Long windowEnd, String fromId) throws IOException {
     return store.getEntities(entityType, null, null, windowEnd, fromId, null,
-        null, null, null).getEntities();
+        null, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromIdWithPrimaryFilterAndWindow(
       String entityType, Long windowEnd, String fromId,
       NameValuePair primaryFilter) throws IOException {
     return store.getEntities(entityType, null, null, windowEnd, fromId, null,
-        primaryFilter, null, null).getEntities();
+        primaryFilter, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesWithFilters(String entityType,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters)
       throws IOException {
     return store.getEntities(entityType, null, null, null, null, null,
-        primaryFilter, secondaryFilters, null).getEntities();
+        primaryFilter, secondaryFilters, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntities(String entityType, Long limit,
       Long windowStart, Long windowEnd, NameValuePair primaryFilter,
       EnumSet<Field> fields) throws IOException {
     return store.getEntities(entityType, limit, windowStart, windowEnd, null,
-        null, primaryFilter, null, fields).getEntities();
+        null, primaryFilter, null, fields, null).getEntities();
   }
 
   public void testGetEntities() throws IOException {


[47/50] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into YARN-2928

Posted by zj...@apache.org.
Merge remote-tracking branch 'apache/trunk' into YARN-2928


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

Branch: refs/heads/YARN-2928
Commit: 5de4026d8aef6c3343d24aa3831da48ad7a1a87e
Parents: fb1b596 3bc72cc
Author: Zhijie Shen <zj...@apache.org>
Authored: Tue Mar 17 20:22:11 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Tue Mar 17 20:22:11 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.sh                       |   4 +
 hadoop-common-project/hadoop-common/CHANGES.txt |  36 +-
 .../org/apache/hadoop/crypto/CryptoCodec.java   |  10 +-
 .../hadoop/crypto/CryptoOutputStream.java       |  19 +-
 .../fs/CommonConfigurationKeysPublic.java       |  11 +
 .../ZKDelegationTokenSecretManager.java         |  21 +-
 .../apache/hadoop/tracing/SpanReceiverHost.java |  13 +-
 .../hadoop/crypto/random/OpensslSecureRandom.c  |  18 +-
 .../crypto/TestCryptoStreamsForLocalFS.java     |   8 +-
 ...stCryptoStreamsWithJceAesCtrCryptoCodec.java |  38 ++
 ...yptoStreamsWithOpensslAesCtrCryptoCodec.java |   7 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  29 +-
 .../src/contrib/libwebhdfs/src/hdfs_web.c       |   6 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  23 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   5 +
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |   9 +-
 .../hadoop/hdfs/protocol/BlockListAsLongs.java  | 660 +++++++++++--------
 .../datatransfer/DataTransferProtocol.java      |   5 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |   2 +-
 .../hdfs/protocol/datatransfer/Sender.java      |   4 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |  22 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |  14 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   6 +-
 .../BlockInfoContiguousUnderConstruction.java   |   1 +
 .../server/blockmanagement/BlockManager.java    |  16 +-
 .../hdfs/server/datanode/BPServiceActor.java    |  13 +-
 .../hdfs/server/datanode/DataXceiver.java       |  95 +--
 .../server/datanode/ShortCircuitRegistry.java   |  13 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  20 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  15 +
 .../hdfs/server/namenode/NameNodeRpcServer.java |   2 +-
 .../server/namenode/snapshot/FileDiffList.java  |  19 +-
 .../server/protocol/DatanodeRegistration.java   |   9 +
 .../hdfs/server/protocol/NamespaceInfo.java     |  52 ++
 .../server/protocol/StorageBlockReport.java     |   8 +-
 .../hadoop-hdfs/src/main/native/libhdfs/hdfs.c  |  37 ++
 .../hadoop-hdfs/src/main/native/libhdfs/hdfs.h  |  15 +
 .../src/main/proto/DatanodeProtocol.proto       |   2 +
 .../src/main/proto/datatransfer.proto           |  11 +
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |   1 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  76 ++-
 .../apache/hadoop/hdfs/TestFileCreation.java    |   4 +-
 .../hdfs/protocol/TestBlockListAsLongs.java     | 237 +++++++
 .../blockmanagement/TestBlockManager.java       |   8 +-
 .../server/datanode/BlockReportTestBase.java    |  27 +-
 .../server/datanode/SimulatedFSDataset.java     |  11 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |   9 +-
 .../datanode/TestDataNodeVolumeFailure.java     |   4 +-
 ...TestDnRespectsBlockReportSplitThreshold.java |   2 +-
 .../extdataset/ExternalDatasetImpl.java         |   2 +-
 .../server/namenode/NNThroughputBenchmark.java  |  23 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |   3 +-
 .../hdfs/server/namenode/TestFSImage.java       |   2 +
 .../hdfs/server/namenode/TestFileTruncate.java  |  50 +-
 .../snapshot/TestRenameWithSnapshots.java       |   4 +-
 .../shortcircuit/TestShortCircuitCache.java     |  63 ++
 .../TestOfflineEditsViewer.java                 |   9 +-
 hadoop-mapreduce-project/CHANGES.txt            |  23 +-
 .../v2/app/launcher/ContainerLauncherImpl.java  |  14 +-
 .../jobhistory/TestJobHistoryEventHandler.java  |  14 +-
 .../v2/app/launcher/TestContainerLauncher.java  |  21 +-
 .../java/org/apache/hadoop/mapred/JobConf.java  |   5 +
 .../java/org/apache/hadoop/mapred/MapTask.java  |   2 +
 .../apache/hadoop/mapreduce/JobSubmitter.java   |   2 +-
 .../apache/hadoop/mapreduce/MRJobConfig.java    |   9 +
 .../mapreduce/task/reduce/MergeManagerImpl.java |   5 +-
 .../src/main/resources/mapred-default.xml       |   8 +
 .../mapred/TestMRTimelineEventHandling.java     |  12 +-
 .../java/org/apache/hadoop/ant/DfsTask.java     |   6 +-
 .../org/apache/hadoop/fs/s3/S3FileSystem.java   |   4 +-
 .../hadoop/fs/s3a/S3AFastOutputStream.java      |   4 +-
 .../hadoop/fs/s3native/NativeS3FileSystem.java  |  24 +-
 .../fs/azure/AzureNativeFileSystemStore.java    |  22 +-
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |  16 +-
 .../hadoop/tools/CopyListingFileStatus.java     |   8 +-
 .../apache/hadoop/tools/SimpleCopyListing.java  |   2 +-
 .../apache/hadoop/tools/util/DistCpUtils.java   |   4 +-
 .../mapred/gridmix/TestRandomAlgorithm.java     |   2 -
 .../src/site/markdown/SchedulerLoadSimulator.md |   2 +-
 .../java/org/apache/hadoop/record/Buffer.java   |   8 +-
 .../java/org/apache/hadoop/record/Utils.java    |   8 +-
 .../src/site/markdown/HadoopStreaming.md.vm     |  14 +-
 hadoop-yarn-project/CHANGES.txt                 |  32 +-
 .../dev-support/findbugs-exclude.xml            |  27 +
 .../yarn/api/ApplicationBaseProtocol.java       |  44 +-
 .../yarn/api/ApplicationClientProtocol.java     |   3 -
 .../api/protocolrecords/AllocateRequest.java    |  25 +-
 .../api/protocolrecords/AllocateResponse.java   |  68 +-
 .../FinishApplicationMasterRequest.java         |  25 +-
 .../FinishApplicationMasterResponse.java        |   7 +-
 .../protocolrecords/GetApplicationsRequest.java |   1 -
 .../GetClusterMetricsResponse.java              |   4 +-
 .../GetContainerStatusesRequest.java            |   2 -
 .../GetContainerStatusesResponse.java           |   2 -
 .../protocolrecords/GetQueueInfoRequest.java    |   2 +-
 .../protocolrecords/GetQueueInfoResponse.java   |  11 +-
 .../KillApplicationResponse.java                |   9 +-
 .../RegisterApplicationMasterRequest.java       |  33 +-
 .../RegisterApplicationMasterResponse.java      |  11 +-
 .../protocolrecords/StartContainerRequest.java  |   9 +-
 .../api/records/ApplicationAttemptReport.java   |  23 +-
 .../yarn/api/records/ApplicationReport.java     |  47 +-
 .../records/ApplicationSubmissionContext.java   |  50 +-
 .../hadoop/yarn/api/records/Container.java      |  49 +-
 .../api/records/ContainerLaunchContext.java     |  35 +-
 .../yarn/api/records/ContainerReport.java       |  29 +-
 .../yarn/api/records/ContainerStatus.java       |  21 +-
 .../yarn/api/records/LocalResourceType.java     |  32 +-
 .../api/records/LocalResourceVisibility.java    |  31 +-
 .../yarn/api/records/LogAggregationContext.java |  39 +-
 .../hadoop/yarn/api/records/NodeReport.java     |  25 +-
 .../yarn/api/records/PreemptionMessage.java     |  32 +-
 .../hadoop/yarn/api/records/QueueACL.java       |  13 +-
 .../hadoop/yarn/api/records/QueueInfo.java      |  25 +-
 .../hadoop/yarn/api/records/QueueState.java     |  15 +-
 .../yarn/api/records/ReservationRequest.java    |  17 +-
 .../records/ReservationRequestInterpreter.java  |  38 +-
 .../yarn/api/records/ResourceRequest.java       |  51 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +-
 .../UpdateNodeResourceRequest.java              |   4 +-
 .../distributedshell/TestDistributedShell.java  |   4 +-
 .../hadoop/yarn/client/api/AHSClient.java       |  24 +-
 .../hadoop/yarn/client/api/AMRMClient.java      |   4 +-
 .../apache/hadoop/yarn/client/api/NMClient.java |   4 +-
 .../hadoop/yarn/client/api/NMTokenCache.java    |  58 +-
 .../hadoop/yarn/client/api/YarnClient.java      |  23 +-
 .../nodelabels/CommonNodeLabelsManager.java     |   6 +-
 .../hadoop/yarn/nodelabels/NodeLabelsStore.java |   3 +-
 .../server/security/ApplicationACLsManager.java |   1 -
 .../apache/hadoop/yarn/util/StringHelper.java   |   6 +-
 .../org/apache/hadoop/yarn/webapp/WebApps.java  |   4 +-
 .../registry/client/binding/RegistryUtils.java  |   8 +-
 .../client/impl/RegistryOperationsClient.java   |   2 +-
 .../client/impl/zk/ZookeeperConfigOptions.java  |   3 +-
 .../server/services/MicroZookeeperService.java  |  10 +-
 .../registry/server/services/package-info.java  |   9 +-
 .../server/timeline/LeveldbTimelineStore.java   |  18 +-
 .../server/timeline/MemoryTimelineStore.java    |  12 +-
 .../server/timeline/TimelineDataManager.java    |  50 +-
 .../yarn/server/timeline/TimelineReader.java    |   3 +-
 ...TimelineAuthenticationFilterInitializer.java |  13 +-
 .../timeline/TestLeveldbTimelineStore.java      |  16 +-
 .../timeline/TestTimelineDataManager.java       |  26 +-
 .../server/timeline/TimelineStoreTestUtils.java |  33 +-
 .../org/apache/hadoop/yarn/lib/ZKClient.java    |   2 +-
 .../RegisterNodeManagerRequest.java             |   3 +-
 .../server/api/records/NodeHealthStatus.java    |  24 +-
 .../hadoop/yarn/server/webapp/WebPageUtils.java |   6 +-
 .../server/nodemanager/ContainerExecutor.java   |   8 +-
 .../util/NodeManagerHardwareUtils.java          |   8 +-
 .../TestDockerContainerExecutor.java            |   2 +-
 .../server/resourcemanager/RMAppManager.java    |   6 +-
 .../recovery/FileSystemRMStateStore.java        |  22 +-
 .../rmapp/attempt/RMAppAttempt.java             |  11 +-
 .../scheduler/SchedulerNode.java                |   2 +-
 .../scheduler/SchedulerUtils.java               |   3 +-
 .../scheduler/capacity/AbstractCSQueue.java     | 112 +++-
 .../scheduler/capacity/CSQueue.java             |   4 +-
 .../scheduler/capacity/CapacityScheduler.java   |  38 +-
 .../scheduler/capacity/LeafQueue.java           | 292 +++-----
 .../scheduler/capacity/ParentQueue.java         | 140 +---
 .../scheduler/common/fica/FiCaSchedulerApp.java |  16 +-
 .../scheduler/fair/AllocationConfiguration.java |  13 +-
 .../fair/AllocationFileLoaderService.java       |   2 +-
 .../scheduler/fair/FSOpDurations.java           |   3 -
 .../fair/policies/ComputeFairShares.java        |  19 +-
 .../security/DelegationTokenRenewer.java        |   2 -
 .../server/resourcemanager/TestAppManager.java  |   5 +
 .../resourcemanager/TestClientRMService.java    |   5 +
 .../recovery/TestFSRMStateStore.java            |  16 +-
 .../capacity/TestApplicationLimits.java         |   8 +-
 .../capacity/TestCapacityScheduler.java         |  86 +++
 .../scheduler/capacity/TestChildQueueOrder.java |  25 +-
 .../scheduler/capacity/TestLeafQueue.java       | 142 ++--
 .../scheduler/capacity/TestParentQueue.java     |  97 +--
 .../scheduler/capacity/TestReservations.java    | 147 +++--
 .../yarn/server/webproxy/ProxyUriUtils.java     |   2 +-
 .../src/site/markdown/TimelineServer.md         | 318 ++++-----
 .../resources/images/timeline_structure.jpg     | Bin 0 -> 23070 bytes
 179 files changed, 3007 insertions(+), 1895 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5de4026d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5de4026d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5de4026d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------


[43/50] hadoop git commit: Revert "YARN-3181. FairScheduler: Fix up outdated findbugs issues. (kasha)"

Posted by zj...@apache.org.
Revert "YARN-3181. FairScheduler: Fix up outdated findbugs issues. (kasha)"

This reverts commit c2b185def846f5577a130003a533b9c377b58fab.


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

Branch: refs/heads/YARN-2928
Commit: 32b43304563c2430c00bc3e142a962d2bc5f4d58
Parents: d884670
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Mar 17 12:31:15 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Tue Mar 17 12:31:15 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  2 --
 .../dev-support/findbugs-exclude.xml            | 27 ++++++++++++++++++++
 .../scheduler/fair/AllocationConfiguration.java | 13 +++-------
 .../fair/AllocationFileLoaderService.java       |  2 +-
 .../scheduler/fair/FSOpDurations.java           |  3 ---
 5 files changed, 31 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/32b43304/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f5b72d7..fee0ce0 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -320,8 +320,6 @@ Release 2.7.0 - UNRELEASED
     YARN-2079. Recover NonAggregatingLogHandler state upon nodemanager
     restart. (Jason Lowe via junping_du) 
 
-    YARN-3181. FairScheduler: Fix up outdated findbugs issues. (kasha)
-
     YARN-3124. Fixed CS LeafQueue/ParentQueue to use QueueCapacities to track
     capacities-by-label. (Wangda Tan via jianhe)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32b43304/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 a89884a..943ecb0 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -152,12 +152,22 @@
     <Class name="org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationFileLoaderService" />
+    <Field name="allocFile" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
   <!-- Inconsistent sync warning - minimumAllocation is only initialized once and never changed -->
   <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler" />
     <Field name="minimumAllocation" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSSchedulerNode" />
+    <Method name="reserveResource" />
+    <Bug pattern="BC_UNCONFIRMED_CAST" /> 
+  </Match>
   <!-- Inconsistent sync warning - reinitialize read from other queue does not need sync-->
   <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue" />
@@ -215,6 +225,18 @@
     <Field name="scheduleAsynchronously" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+  <!-- Inconsistent sync warning - updateInterval is only initialized once and never changed -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler" />
+    <Field name="updateInterval" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
+  <!-- Inconsistent sync warning - callDurationMetrics is only initialized once and never changed -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler" />
+    <Field name="fsOpDurations" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
 
   <!-- Inconsistent sync warning - numRetries is only initialized once and never changed -->
   <Match>
@@ -415,6 +437,11 @@
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
   <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler" />
+    <Field name="allocConf" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
+  <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode" />
     <Field name="numContainers" />
     <Bug pattern="VO_VOLATILE_INCREMENT" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32b43304/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.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/AllocationConfiguration.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/AllocationConfiguration.java
index 9cb767d..0ea7314 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/AllocationConfiguration.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/AllocationConfiguration.java
@@ -33,9 +33,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import javax.annotation.concurrent.ThreadSafe;
-
-@ThreadSafe
 public class AllocationConfiguration extends ReservationSchedulerConfiguration {
   private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
   private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
@@ -207,16 +204,12 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
   }
 
   public ResourceWeights getQueueWeight(String queue) {
-    synchronized (queueWeights) {
-      ResourceWeights weight = queueWeights.get(queue);
-      return (weight == null) ? ResourceWeights.NEUTRAL : weight;
-    }
+    ResourceWeights weight = queueWeights.get(queue);
+    return (weight == null) ? ResourceWeights.NEUTRAL : weight;
   }
 
   public void setQueueWeight(String queue, ResourceWeights weight) {
-    synchronized (queueWeights) {
-      queueWeights.put(queue, weight);
-    }
+    queueWeights.put(queue, weight);
   }
   
   public int getUserMaxApps(String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32b43304/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.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/AllocationFileLoaderService.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/AllocationFileLoaderService.java
index c19aa51..76fa588 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/AllocationFileLoaderService.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/AllocationFileLoaderService.java
@@ -201,7 +201,7 @@ public class AllocationFileLoaderService extends AbstractService {
    * @throws ParserConfigurationException if XML parser is misconfigured.
    * @throws SAXException if config file is malformed.
    */
-  public void reloadAllocations() throws IOException,
+  public synchronized void reloadAllocations() throws IOException,
       ParserConfigurationException, SAXException, AllocationConfigurationException {
     if (allocFile == null) {
       return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32b43304/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSOpDurations.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/FSOpDurations.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/FSOpDurations.java
index c50f281..c2282fd 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/FSOpDurations.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/FSOpDurations.java
@@ -31,8 +31,6 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import static org.apache.hadoop.metrics2.lib.Interns.info;
 import org.apache.hadoop.metrics2.lib.MutableRate;
 
-import javax.annotation.concurrent.ThreadSafe;
-
 /**
  * Class to capture the performance metrics of FairScheduler.
  * This should be a singleton.
@@ -40,7 +38,6 @@ import javax.annotation.concurrent.ThreadSafe;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 @Metrics(context="fairscheduler-op-durations")
-@ThreadSafe
 public class FSOpDurations implements MetricsSource {
 
   @Metric("Duration for a continuous scheduling run")


[22/50] hadoop git commit: YARN-2854. Addendum patch to fix the minor issue in the timeline service documentation.

Posted by zj...@apache.org.
YARN-2854. Addendum patch to fix the minor issue in the timeline service documentation.


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

Branch: refs/heads/YARN-2928
Commit: ed4e72a20b75ffbd22deb0607dd8b94f6e437a84
Parents: d1eebd9
Author: Zhijie Shen <zj...@apache.org>
Authored: Mon Mar 16 10:39:10 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon Mar 16 10:39:10 2015 -0700

----------------------------------------------------------------------
 .../src/site/markdown/TimelineServer.md         | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed4e72a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
index 31fe4ac..cb8a5d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
@@ -43,7 +43,7 @@ Overview
 
 ### Current Status
 
-  Current version of Timeline sever has been completed. Essential functionality of the timeline server can work in both secure and non secure modes. The generic history service is also ridden on the timeline store. In subsequent releases we will be rolling out next generation timeline service which is scalable and reliable. Finally, the Application specific information is only available via RESTful APIs, using JSON type content - ability to install framework specific UIs in YARN isn't supported yet.
+  The essential functionality of the timeline server have been completed and it can work in both secure and non secure modes. The generic history service is also built on timeline store. In subsequent releases we will be rolling out next generation timeline service which is scalable and reliable. Currently, Application specific information is only available via RESTful APIs using JSON type content. The ability to install framework specific UIs in YARN is not supported yet.
 
 ### Timeline Structure
 
@@ -72,7 +72,7 @@ Deployment
 |:---- |:---- |
 | `yarn.timeline-service.enabled` | Indicate to clients whether Timeline service is enabled or not. If enabled, the TimelineClient library used by end-users will post entities and events to the Timeline server. Defaults to false. |
 | `yarn.resourcemanager.system-metrics-publisher.enabled` | The setting that controls whether yarn system metrics is published on the timeline server or not by RM. Defaults to false. |
-| `yarn.timeline-service.generic-application-history.enabled` | Indicate to clients whether to query generic application data from timeline history-service. If not enabled then application data is only queried from Resource Manager. Defaults to false. |
+| `yarn.timeline-service.generic-application-history.enabled` | Indicate to clients whether to query generic application data from timeline history-service or not. If not enabled then application data is queried only from Resource Manager. Defaults to false. |
 
 #### Advanced configuration
 
@@ -141,16 +141,16 @@ Deployment
 </property>
 
 <property>
-    <description>The setting that controls whether yarn system metrics is
-    published on the timeline server or not by RM.</description>
-    <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
-    <value>true</value>
+  <description>The setting that controls whether yarn system metrics is
+  published on the timeline server or not by RM.</description>
+  <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
+  <value>true</value>
 </property>
 
 <property>
-  <description>Indicate to clients whether to query generic application data from
-  timeline history-service. If not enabled then application data is only queried 
-  from Resource Manager</description>
+  <description>Indicate to clients whether to query generic application
+  data from timeline history-service or not. If not enabled then application
+  data is queried only from Resource Manager.</description>
   <name>yarn.timeline-service.generic-application-history.enabled</name>
   <value>true</value>
 </property>
@@ -167,7 +167,7 @@ Deployment
   Or users can start the Timeline server / history service as a daemon:
 
 ```
-  $ yarn --daemon start timelineserver
+  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh start timelineserver
 ```
 
 ### Accessing generic-data via command-line


[14/50] hadoop git commit: Revert "HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Masatake Iwasaki."

Posted by zj...@apache.org.
Revert "HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Masatake Iwasaki."

This reverts commit 7da136ecca4dafc83ef69b5d9980fa5b67ada084.


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

Branch: refs/heads/YARN-2928
Commit: b308a8d181416b5fe6bf77756e5f2c7b8fbd793c
Parents: 7da136e
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sun Mar 15 14:27:38 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sun Mar 15 14:27:38 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt       |  3 ---
 .../src/site/markdown/SchedulerLoadSimulator.md       |  2 +-
 .../src/site/markdown/HadoopStreaming.md.vm           | 14 +++++++-------
 3 files changed, 8 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b308a8d1/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e386723..55028cb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1100,9 +1100,6 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt
     synchronization. (Sean Busbey via yliu)
 
-    HADOOP-11558. Fix dead links to doc of hadoop-tools. (Masatake Iwasaki
-    via ozawa)
-
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b308a8d1/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
index 2cffc86..ca179ee 100644
--- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
+++ b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
@@ -43,7 +43,7 @@ The Yarn Scheduler Load Simulator (SLS) is such a tool, which can simulate large
 o
 The simulator will exercise the real Yarn `ResourceManager` removing the network factor by simulating `NodeManagers` and `ApplicationMasters` via handling and dispatching `NM`/`AMs` heartbeat events from within the same JVM. To keep tracking of scheduler behavior and performance, a scheduler wrapper will wrap the real scheduler.
 
-The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](../hadoop-rumen/Rumen.html).
+The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](https://hadoop.apache.org/docs/stable/rumen.html).
 
 The simulator will produce real time metrics while executing, including:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b308a8d1/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
index b4c5e38..0b64586 100644
--- a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
+++ b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
@@ -201,7 +201,7 @@ To specify additional local temp directories use:
      -D mapred.system.dir=/tmp/system
      -D mapred.temp.dir=/tmp/temp
 
-**Note:** For more details on job configuration parameters see: [mapred-default.xml](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml)
+**Note:** For more details on job configuration parameters see: [mapred-default.xml](./mapred-default.xml)
 
 $H4 Specifying Map-Only Jobs
 
@@ -322,7 +322,7 @@ More Usage Examples
 
 $H3 Hadoop Partitioner Class
 
-Hadoop has a library class, [KeyFieldBasedPartitioner](../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
+Hadoop has a library class, [KeyFieldBasedPartitioner](../../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D stream.map.output.field.separator=. \
@@ -372,7 +372,7 @@ Sorting within each partition for the reducer(all 4 fields used for sorting)
 
 $H3 Hadoop Comparator Class
 
-Hadoop has a library class, [KeyFieldBasedComparator](../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
+Hadoop has a library class, [KeyFieldBasedComparator](../../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.job.output.key.comparator.class=org.apache.hadoop.mapreduce.lib.partition.KeyFieldBasedComparator \
@@ -406,7 +406,7 @@ Sorting output for the reducer (where second field used for sorting)
 
 $H3 Hadoop Aggregate Package
 
-Hadoop has a library package called [Aggregate](../api/org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
+Hadoop has a library package called [Aggregate](../../org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
 
 To use Aggregate, simply specify "-reducer aggregate":
 
@@ -441,7 +441,7 @@ The python program myAggregatorForKeyCount.py looks like:
 
 $H3 Hadoop Field Selection Class
 
-Hadoop has a library class, [FieldSelectionMapReduce](../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
+Hadoop has a library class, [FieldSelectionMapReduce](../../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.map.output.key.field.separator=. \
@@ -480,7 +480,7 @@ As an example, consider the problem of zipping (compressing) a set of files acro
 
 $H3 How many reducers should I use?
 
-See MapReduce Tutorial for details: [Reducer](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Reducer)
+See MapReduce Tutorial for details: [Reducer](./MapReduceTutorial.html#Reducer)
 
 $H3 If I set up an alias in my shell script, will that work after -mapper?
 
@@ -556,4 +556,4 @@ A streaming process can use the stderr to emit status information. To set a stat
 
 $H3 How do I get the Job variables in a streaming job's mapper/reducer?
 
-See [Configured Parameters](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.
+See [Configured Parameters](./MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.


[38/50] hadoop git commit: YARN-3197. Confusing log generated by CapacityScheduler. Contributed by Varun Saxena.

Posted by zj...@apache.org.
YARN-3197. Confusing log generated by CapacityScheduler. Contributed by
Varun Saxena.


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

Branch: refs/heads/YARN-2928
Commit: 7179f94f9d000fc52bd9ce5aa9741aba97ec3ee8
Parents: 018893e
Author: Devaraj K <de...@apache.org>
Authored: Tue Mar 17 15:57:57 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Tue Mar 17 15:57:57 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                 | 3 +++
 .../resourcemanager/scheduler/capacity/CapacityScheduler.java   | 5 +++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7179f94f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index cb68480..82934ad 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -63,6 +63,9 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+     YARN-3197. Confusing log generated by CapacityScheduler. (Varun Saxena 
+     via devaraj)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7179f94f/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 28ce264..756e537 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
@@ -1279,7 +1279,8 @@ public class CapacityScheduler extends
   protected synchronized void completedContainer(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event) {
     if (rmContainer == null) {
-      LOG.info("Null container completed...");
+      LOG.info("Container " + containerStatus.getContainerId() +
+          " completed with event " + event);
       return;
     }
     
@@ -1291,7 +1292,7 @@ public class CapacityScheduler extends
     ApplicationId appId =
         container.getId().getApplicationAttemptId().getApplicationId();
     if (application == null) {
-      LOG.info("Container " + container + " of" + " unknown application "
+      LOG.info("Container " + container + " of" + " finished application "
           + appId + " completed with event " + event);
       return;
     }


[36/50] hadoop git commit: MAPREDUCE-4653. TestRandomAlgorithm has an unused import statement. Contributed by Amir Sanjar.

Posted by zj...@apache.org.
MAPREDUCE-4653. TestRandomAlgorithm has an unused import statement. Contributed by Amir Sanjar.


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

Branch: refs/heads/YARN-2928
Commit: e5370477c2d00745e695507ecfdf86de59c5f5b9
Parents: 48c2db3
Author: Harsh J <ha...@cloudera.com>
Authored: Tue Mar 17 14:01:15 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Tue Mar 17 14:11:54 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../org/apache/hadoop/mapred/gridmix/TestRandomAlgorithm.java     | 2 --
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5370477/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index b5baf51..3936c9b 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -253,6 +253,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-4653. TestRandomAlgorithm has an unused "import" statement.
+    (Amir Sanjar via harsh)
+
     MAPREDUCE-6100. replace "mapreduce.job.credentials.binary" with
     MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY for better readability.
     (Zhihai Xu via harsh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5370477/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomAlgorithm.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomAlgorithm.java b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomAlgorithm.java
index cd55483..4e85ce2 100644
--- a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomAlgorithm.java
+++ b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestRandomAlgorithm.java
@@ -30,8 +30,6 @@ import java.util.Set;
 
 import org.junit.Test;
 
-import com.sun.tools.javac.code.Attribute.Array;
-
 public class TestRandomAlgorithm {
   private static final int[][] parameters = new int[][] {
     {5, 1, 1}, 


[06/50] hadoop git commit: HDFS-7903. Cannot recover block after truncate and delete snapshot. Contributed by Plamen Jeliazkov.

Posted by zj...@apache.org.
HDFS-7903. Cannot recover block after truncate and delete snapshot. Contributed by Plamen Jeliazkov.

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

Branch: refs/heads/YARN-2928
Commit: 6acb7f2110897264241df44d564db2f85260348f
Parents: d324164
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Fri Mar 13 12:39:01 2015 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Mar 13 13:12:51 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../server/namenode/snapshot/FileDiffList.java  | 19 +++++++++++--
 .../hdfs/server/namenode/TestFileTruncate.java  | 30 ++++++++++++++++++++
 3 files changed, 49 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6acb7f21/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ac7e096..a149f18 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1148,6 +1148,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not 
     idempotent (Tsz Wo Nicholas Sze via brandonli)
 
+    HDFS-7903. Cannot recover block after truncate and delete snapshot.
+    (Plamen Jeliazkov via shv)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6acb7f21/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 0c94554..5c9e121 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -20,8 +20,11 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -125,9 +128,19 @@ public class FileDiffList extends
         continue;
       break;
     }
-    // Collect the remaining blocks of the file
-    while(i < removedBlocks.length) {
-      collectedBlocks.addDeleteBlock(removedBlocks[i++]);
+    // Check if last block is part of truncate recovery
+    BlockInfoContiguous lastBlock = file.getLastBlock();
+    Block dontRemoveBlock = null;
+    if(lastBlock != null && lastBlock.getBlockUCState().equals(
+        HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
+      dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
+          .getTruncateBlock();
+    }
+    // Collect the remaining blocks of the file, ignoring truncate block
+    for(;i < removedBlocks.length; i++) {
+      if(dontRemoveBlock == null || !removedBlocks[i].equals(dontRemoveBlock)) {
+        collectedBlocks.addDeleteBlock(removedBlocks[i]);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6acb7f21/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 260d8bb..3b6e107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -178,6 +178,36 @@ public class TestFileTruncate {
     fs.delete(dir, true);
   }
 
+  /** Truncate the same file multiple times until its size is zero. */
+  @Test
+  public void testSnapshotTruncateThenDeleteSnapshot() throws IOException {
+    Path dir = new Path("/testSnapshotTruncateThenDeleteSnapshot");
+    fs.mkdirs(dir);
+    fs.allowSnapshot(dir);
+    final Path p = new Path(dir, "file");
+    final byte[] data = new byte[BLOCK_SIZE];
+    DFSUtil.getRandom().nextBytes(data);
+    writeContents(data, data.length, p);
+    final String snapshot = "s0";
+    fs.createSnapshot(dir, snapshot);
+    Block lastBlock = getLocatedBlocks(p).getLastLocatedBlock()
+        .getBlock().getLocalBlock();
+    final int newLength = data.length - 1;
+    assert newLength % BLOCK_SIZE != 0 :
+        " newLength must not be multiple of BLOCK_SIZE";
+    final boolean isReady = fs.truncate(p, newLength);
+    LOG.info("newLength=" + newLength + ", isReady=" + isReady);
+    assertEquals("File must be closed for truncating at the block boundary",
+        isReady, newLength % BLOCK_SIZE == 0);
+    fs.deleteSnapshot(dir, snapshot);
+    if (!isReady) {
+      checkBlockRecovery(p);
+    }
+    checkFullFile(p, newLength, data);
+    assertBlockNotPresent(lastBlock);
+    fs.delete(dir, true);
+  }
+
   /**
    * Truncate files and then run other operations such as
    * rename, set replication, set permission, etc.


[05/50] hadoop git commit: HDFS-7435. PB encoding of block reports is very inefficient. Contributed by Daryn Sharp.

Posted by zj...@apache.org.
HDFS-7435. PB encoding of block reports is very inefficient. Contributed by Daryn Sharp.


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

Branch: refs/heads/YARN-2928
Commit: d324164a51a43d72c02567248bd9f0f12b244a40
Parents: f446669
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Mar 13 14:13:55 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Mar 13 14:23:37 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop/hdfs/protocol/BlockListAsLongs.java  | 660 +++++++++++--------
 .../DatanodeProtocolClientSideTranslatorPB.java |  22 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |  14 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   6 +-
 .../server/blockmanagement/BlockManager.java    |  16 +-
 .../hdfs/server/datanode/BPServiceActor.java    |  13 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  20 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   2 +-
 .../server/protocol/DatanodeRegistration.java   |   9 +
 .../hdfs/server/protocol/NamespaceInfo.java     |  52 ++
 .../server/protocol/StorageBlockReport.java     |   8 +-
 .../src/main/proto/DatanodeProtocol.proto       |   2 +
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |   1 +
 .../hdfs/protocol/TestBlockListAsLongs.java     | 237 +++++++
 .../blockmanagement/TestBlockManager.java       |   8 +-
 .../server/datanode/BlockReportTestBase.java    |  27 +-
 .../server/datanode/SimulatedFSDataset.java     |  11 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |   9 +-
 .../datanode/TestDataNodeVolumeFailure.java     |   4 +-
 ...TestDnRespectsBlockReportSplitThreshold.java |   2 +-
 .../extdataset/ExternalDatasetImpl.java         |   2 +-
 .../server/namenode/NNThroughputBenchmark.java  |  23 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |   3 +-
 .../hdfs/server/namenode/TestFSImage.java       |   2 +
 .../TestOfflineEditsViewer.java                 |   9 +-
 26 files changed, 811 insertions(+), 354 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 909182b..ac7e096 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -743,6 +743,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7491. Add incremental blockreport latency to DN metrics.
     (Ming Ma via cnauroth)
 
+    HDFS-7435. PB encoding of block reports is very inefficient.
+    (Daryn Sharp via kihwal)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
index 4389714..1c89ee4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
@@ -17,342 +17,458 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Random;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.CodedOutputStream;
 
-/**
- * This class provides an interface for accessing list of blocks that
- * has been implemented as long[].
- * This class is useful for block report. Rather than send block reports
- * as a Block[] we can send it as a long[].
- *
- * The structure of the array is as follows:
- * 0: the length of the finalized replica list;
- * 1: the length of the under-construction replica list;
- * - followed by finalized replica list where each replica is represented by
- *   3 longs: one for the blockId, one for the block length, and one for
- *   the generation stamp;
- * - followed by the invalid replica represented with three -1s;
- * - followed by the under-construction replica list where each replica is
- *   represented by 4 longs: three for the block id, length, generation 
- *   stamp, and the fourth for the replica state.
- */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class BlockListAsLongs implements Iterable<Block> {
+public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
+  private final static int CHUNK_SIZE = 64*1024; // 64K
+  private static long[] EMPTY_LONGS = new long[]{0, 0};
+
+  public static BlockListAsLongs EMPTY = new BlockListAsLongs() {
+    @Override
+    public int getNumberOfBlocks() {
+      return 0;
+    }
+    @Override
+    public ByteString getBlocksBuffer() {
+      return ByteString.EMPTY;
+    }
+    @Override
+    public long[] getBlockListAsLongs() {
+      return EMPTY_LONGS;
+    }
+    @Override
+    public Iterator<BlockReportReplica> iterator() {
+      return Collections.emptyIterator();
+    }
+  };
+
   /**
-   * A finalized block as 3 longs
-   *   block-id and block length and generation stamp
+   * Prepare an instance to in-place decode the given ByteString buffer
+   * @param numBlocks - blocks in the buffer
+   * @param blocksBuf - ByteString encoded varints
+   * @return BlockListAsLongs
    */
-  private static final int LONGS_PER_FINALIZED_BLOCK = 3;
+  public static BlockListAsLongs decodeBuffer(final int numBlocks,
+      final ByteString blocksBuf) {
+    return new BufferDecoder(numBlocks, blocksBuf);
+  }
 
   /**
-   * An under-construction block as 4 longs
-   *   block-id and block length, generation stamp and replica state
+   * Prepare an instance to in-place decode the given ByteString buffers
+   * @param numBlocks - blocks in the buffers
+   * @param blocksBufs - list of ByteString encoded varints
+   * @return BlockListAsLongs
    */
-  private static final int LONGS_PER_UC_BLOCK = 4;
-
-  /** Number of longs in the header */
-  private static final int HEADER_SIZE = 2;
+  public static BlockListAsLongs decodeBuffers(final int numBlocks,
+      final List<ByteString> blocksBufs) {
+    // this doesn't actually copy the data
+    return decodeBuffer(numBlocks, ByteString.copyFrom(blocksBufs));
+  }
 
   /**
-   * Returns the index of the first long in blockList
-   * belonging to the specified block.
-   * The first long contains the block id.
+   * Prepare an instance to in-place decode the given list of Longs.  Note
+   * it's much more efficient to decode ByteString buffers and only exists
+   * for compatibility.
+   * @param blocksList - list of longs
+   * @return BlockListAsLongs
    */
-  private int index2BlockId(int blockIndex) {
-    if(blockIndex < 0 || blockIndex > getNumberOfBlocks())
-      return -1;
-    int finalizedSize = getNumberOfFinalizedReplicas();
-    if(blockIndex < finalizedSize)
-      return HEADER_SIZE + blockIndex * LONGS_PER_FINALIZED_BLOCK;
-    return HEADER_SIZE + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK
-            + (blockIndex - finalizedSize) * LONGS_PER_UC_BLOCK;
+  public static BlockListAsLongs decodeLongs(List<Long> blocksList) {
+    return blocksList.isEmpty() ? EMPTY : new LongsDecoder(blocksList);
   }
 
-  private final long[] blockList;
-  
   /**
-   * Create block report from finalized and under construction lists of blocks.
-   * 
-   * @param finalized - list of finalized blocks
-   * @param uc - list of under construction blocks
+   * Prepare an instance to encode the collection of replicas into an
+   * efficient ByteString.
+   * @param replicas - replicas to encode
+   * @return BlockListAsLongs
    */
-  public BlockListAsLongs(final List<? extends Replica> finalized,
-                          final List<? extends Replica> uc) {
-    int finalizedSize = finalized == null ? 0 : finalized.size();
-    int ucSize = uc == null ? 0 : uc.size();
-    int len = HEADER_SIZE
-              + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK
-              + ucSize * LONGS_PER_UC_BLOCK;
-
-    blockList = new long[len];
+  public static BlockListAsLongs encode(
+      final Collection<? extends Replica> replicas) {
+    BlockListAsLongs.Builder builder = builder();
+    for (Replica replica : replicas) {
+      builder.add(replica);
+    }
+    return builder.build();
+  }
 
-    // set the header
-    blockList[0] = finalizedSize;
-    blockList[1] = ucSize;
+  public static Builder builder() {
+    return new BlockListAsLongs.Builder();
+  }
 
-    // set finalized blocks
-    for (int i = 0; i < finalizedSize; i++) {
-      setBlock(i, finalized.get(i));
-    }
+  /**
+   * The number of blocks
+   * @return - the number of blocks
+   */
+  abstract public int getNumberOfBlocks();
 
-    // set invalid delimiting block
-    setDelimitingBlock(finalizedSize);
+  /**
+   * Very efficient encoding of the block report into a ByteString to avoid
+   * the overhead of protobuf repeating fields.  Primitive repeating fields
+   * require re-allocs of an ArrayList<Long> and the associated (un)boxing
+   * overhead which puts pressure on GC.
+   * 
+   * The structure of the buffer is as follows:
+   * - each replica is represented by 4 longs:
+   *   blockId, block length, genstamp, replica state
+   *
+   * @return ByteString encoded block report
+   */
+  abstract public ByteString getBlocksBuffer();
 
-    // set under construction blocks
-    for (int i = 0; i < ucSize; i++) {
-      setBlock(finalizedSize + i, uc.get(i));
+  /**
+   * List of ByteStrings that encode this block report
+   *
+   * @return ByteStrings
+   */
+  public List<ByteString> getBlocksBuffers() {
+    final ByteString blocksBuf = getBlocksBuffer();
+    final List<ByteString> buffers;
+    final int size = blocksBuf.size();
+    if (size <= CHUNK_SIZE) {
+      buffers = Collections.singletonList(blocksBuf);
+    } else {
+      buffers = new ArrayList<ByteString>();
+      for (int pos=0; pos < size; pos += CHUNK_SIZE) {
+        // this doesn't actually copy the data
+        buffers.add(blocksBuf.substring(pos, Math.min(pos+CHUNK_SIZE, size)));
+      }
     }
+    return buffers;
   }
 
   /**
-   * Create block report from a list of finalized blocks.  Used by
-   * NNThroughputBenchmark.
-   *
-   * @param blocks - list of finalized blocks
+   * Convert block report to old-style list of longs.  Only used to
+   * re-encode the block report when the DN detects an older NN. This is
+   * inefficient, but in practice a DN is unlikely to be upgraded first
+   * 
+   * The structure of the array is as follows:
+   * 0: the length of the finalized replica list;
+   * 1: the length of the under-construction replica list;
+   * - followed by finalized replica list where each replica is represented by
+   *   3 longs: one for the blockId, one for the block length, and one for
+   *   the generation stamp;
+   * - followed by the invalid replica represented with three -1s;
+   * - followed by the under-construction replica list where each replica is
+   *   represented by 4 longs: three for the block id, length, generation 
+   *   stamp, and the fourth for the replica state.
+   * @return list of longs
    */
-  public BlockListAsLongs(final List<? extends Block> blocks) {
-    int finalizedSize = blocks == null ? 0 : blocks.size();
-    int len = HEADER_SIZE
-              + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK;
+  abstract public long[] getBlockListAsLongs();
 
-    blockList = new long[len];
+  /**
+   * Returns a singleton iterator over blocks in the block report.  Do not
+   * add the returned blocks to a collection.
+   * @return Iterator
+   */
+  abstract public Iterator<BlockReportReplica> iterator();
 
-    // set the header
-    blockList[0] = finalizedSize;
-    blockList[1] = 0;
+  public static class Builder {
+    private final ByteString.Output out;
+    private final CodedOutputStream cos;
+    private int numBlocks = 0;
+    private int numFinalized = 0;
 
-    // set finalized blocks
-    for (int i = 0; i < finalizedSize; i++) {
-      setBlock(i, blocks.get(i));
+    Builder() {
+      out = ByteString.newOutput(64*1024);
+      cos = CodedOutputStream.newInstance(out);
     }
 
-    // set invalid delimiting block
-    setDelimitingBlock(finalizedSize);
-  }
-
-  public BlockListAsLongs() {
-    this((long[])null);
-  }
+    public void add(Replica replica) {
+      try {
+        // zig-zag to reduce size of legacy blocks
+        cos.writeSInt64NoTag(replica.getBlockId());
+        cos.writeRawVarint64(replica.getBytesOnDisk());
+        cos.writeRawVarint64(replica.getGenerationStamp());
+        ReplicaState state = replica.getState();
+        // although state is not a 64-bit value, using a long varint to
+        // allow for future use of the upper bits
+        cos.writeRawVarint64(state.getValue());
+        if (state == ReplicaState.FINALIZED) {
+          numFinalized++;
+        }
+        numBlocks++;
+      } catch (IOException ioe) {
+        // shouldn't happen, ByteString.Output doesn't throw IOE
+        throw new IllegalStateException(ioe);
+      }
+    }
 
-  /**
-   * Constructor
-   * @param iBlockList - BlockListALongs create from this long[] parameter
-   */
-  public BlockListAsLongs(final long[] iBlockList) {
-    if (iBlockList == null) {
-      blockList = new long[HEADER_SIZE];
-      return;
+    public int getNumberOfBlocks() {
+      return numBlocks;
+    }
+    
+    public BlockListAsLongs build() {
+      try {
+        cos.flush();
+      } catch (IOException ioe) {
+        // shouldn't happen, ByteString.Output doesn't throw IOE
+        throw new IllegalStateException(ioe);
+      }
+      return new BufferDecoder(numBlocks, numFinalized, out.toByteString());
     }
-    blockList = iBlockList;
   }
 
-  public long[] getBlockListAsLongs() {
-    return blockList;
-  }
+  // decode new-style ByteString buffer based block report
+  private static class BufferDecoder extends BlockListAsLongs {
+    // reserve upper bits for future use.  decoding masks off these bits to
+    // allow compatibility for the current through future release that may
+    // start using the bits
+    private static long NUM_BYTES_MASK = (-1L) >>> (64 - 48);
+    private static long REPLICA_STATE_MASK = (-1L) >>> (64 - 4);
 
-  /**
-   * Iterates over blocks in the block report.
-   * Avoids object allocation on each iteration.
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  public class BlockReportIterator implements Iterator<Block> {
-    private int currentBlockIndex;
-    private final Block block;
-    private ReplicaState currentReplicaState;
-
-    BlockReportIterator() {
-      this.currentBlockIndex = 0;
-      this.block = new Block();
-      this.currentReplicaState = null;
+    private final ByteString buffer;
+    private final int numBlocks;
+    private int numFinalized;
+
+    BufferDecoder(final int numBlocks, final ByteString buf) {
+      this(numBlocks, -1, buf);
     }
 
-    @Override
-    public boolean hasNext() {
-      return currentBlockIndex < getNumberOfBlocks();
+    BufferDecoder(final int numBlocks, final int numFinalized,
+        final ByteString buf) {
+      this.numBlocks = numBlocks;
+      this.numFinalized = numFinalized;
+      this.buffer = buf;
     }
 
     @Override
-    public Block next() {
-      block.set(blockId(currentBlockIndex),
-                blockLength(currentBlockIndex),
-                blockGenerationStamp(currentBlockIndex));
-      currentReplicaState = blockReplicaState(currentBlockIndex);
-      currentBlockIndex++;
-      return block;
+    public int getNumberOfBlocks() {
+      return numBlocks;
     }
 
     @Override
-    public void remove() {
-      throw new UnsupportedOperationException("Sorry. can't remove.");
+    public ByteString getBlocksBuffer() {
+      return buffer;
     }
 
-    /**
-     * Get the state of the current replica.
-     * The state corresponds to the replica returned
-     * by the latest {@link #next()}. 
-     */
-    public ReplicaState getCurrentReplicaState() {
-      return currentReplicaState;
+    @Override
+    public long[] getBlockListAsLongs() {
+      // terribly inefficient but only occurs if server tries to transcode
+      // an undecoded buffer into longs - ie. it will never happen but let's
+      // handle it anyway
+      if (numFinalized == -1) {
+        int n = 0;
+        for (Replica replica : this) {
+          if (replica.getState() == ReplicaState.FINALIZED) {
+            n++;
+          }
+        }
+        numFinalized = n;
+      }
+      int numUc = numBlocks - numFinalized;
+      int size = 2 + 3*(numFinalized+1) + 4*(numUc);
+      long[] longs = new long[size];
+      longs[0] = numFinalized;
+      longs[1] = numUc;
+
+      int idx = 2;
+      int ucIdx = idx + 3*numFinalized;
+      // delimiter block
+      longs[ucIdx++] = -1;
+      longs[ucIdx++] = -1;
+      longs[ucIdx++] = -1;
+
+      for (BlockReportReplica block : this) {
+        switch (block.getState()) {
+          case FINALIZED: {
+            longs[idx++] = block.getBlockId();
+            longs[idx++] = block.getNumBytes();
+            longs[idx++] = block.getGenerationStamp();
+            break;
+          }
+          default: {
+            longs[ucIdx++] = block.getBlockId();
+            longs[ucIdx++] = block.getNumBytes();
+            longs[ucIdx++] = block.getGenerationStamp();
+            longs[ucIdx++] = block.getState().getValue();
+            break;
+          }
+        }
+      }
+      return longs;
     }
-  }
 
-  /**
-   * Returns an iterator over blocks in the block report. 
-   */
-  @Override
-  public Iterator<Block> iterator() {
-    return getBlockReportIterator();
-  }
-
-  /**
-   * Returns {@link BlockReportIterator}. 
-   */
-  public BlockReportIterator getBlockReportIterator() {
-    return new BlockReportIterator();
-  }
-
-  /**
-   * The number of blocks
-   * @return - the number of blocks
-   */
-  public int getNumberOfBlocks() {
-    assert blockList.length == HEADER_SIZE + 
-            (blockList[0] + 1) * LONGS_PER_FINALIZED_BLOCK +
-            blockList[1] * LONGS_PER_UC_BLOCK :
-              "Number of blocks is inconcistent with the array length";
-    return getNumberOfFinalizedReplicas() + getNumberOfUCReplicas();
-  }
-
-  /**
-   * Returns the number of finalized replicas in the block report.
-   */
-  private int getNumberOfFinalizedReplicas() {
-    return (int)blockList[0];
-  }
-
-  /**
-   * Returns the number of under construction replicas in the block report.
-   */
-  private int getNumberOfUCReplicas() {
-    return (int)blockList[1];
+    @Override
+    public Iterator<BlockReportReplica> iterator() {
+      return new Iterator<BlockReportReplica>() {
+        final BlockReportReplica block = new BlockReportReplica();
+        final CodedInputStream cis = buffer.newCodedInput();
+        private int currentBlockIndex = 0;
+
+        @Override
+        public boolean hasNext() {
+          return currentBlockIndex < numBlocks;
+        }
+
+        @Override
+        public BlockReportReplica next() {
+          currentBlockIndex++;
+          try {
+            // zig-zag to reduce size of legacy blocks and mask off bits
+            // we don't (yet) understand
+            block.setBlockId(cis.readSInt64());
+            block.setNumBytes(cis.readRawVarint64() & NUM_BYTES_MASK);
+            block.setGenerationStamp(cis.readRawVarint64());
+            long state = cis.readRawVarint64() & REPLICA_STATE_MASK;
+            block.setState(ReplicaState.getState((int)state));
+          } catch (IOException e) {
+            throw new IllegalStateException(e);
+          }
+          return block;
+        }
+
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
   }
 
-  /**
-   * Returns the id of the specified replica of the block report.
-   */
-  private long blockId(int index) {
-    return blockList[index2BlockId(index)];
-  }
+  // decode old style block report of longs
+  private static class LongsDecoder extends BlockListAsLongs {
+    private final List<Long> values;
+    private final int finalizedBlocks;
+    private final int numBlocks;
 
-  /**
-   * Returns the length of the specified replica of the block report.
-   */
-  private long blockLength(int index) {
-    return blockList[index2BlockId(index) + 1];
-  }
+    // set the header
+    LongsDecoder(List<Long> values) {
+      this.values = values.subList(2, values.size());
+      this.finalizedBlocks = values.get(0).intValue();
+      this.numBlocks = finalizedBlocks + values.get(1).intValue();
+    }
 
-  /**
-   * Returns the generation stamp of the specified replica of the block report.
-   */
-  private long blockGenerationStamp(int index) {
-    return blockList[index2BlockId(index) + 2];
-  }
+    @Override
+    public int getNumberOfBlocks() {
+      return numBlocks;
+    }
 
-  /**
-   * Returns the state of the specified replica of the block report.
-   */
-  private ReplicaState blockReplicaState(int index) {
-    if(index < getNumberOfFinalizedReplicas())
-      return ReplicaState.FINALIZED;
-    return ReplicaState.getState((int)blockList[index2BlockId(index) + 3]);
-  }
+    @Override
+    public ByteString getBlocksBuffer() {
+      Builder builder = builder();
+      for (Replica replica : this) {
+        builder.add(replica);
+      }
+      return builder.build().getBlocksBuffer();
+    }
 
-  /**
-   * Corrupt the generation stamp of the block with the given index.
-   * Not meant to be used outside of tests.
-   */
-  @VisibleForTesting
-  public long corruptBlockGSForTesting(final int blockIndex, Random rand) {
-    long oldGS = blockList[index2BlockId(blockIndex) + 2];
-    while (blockList[index2BlockId(blockIndex) + 2] == oldGS) {
-      blockList[index2BlockId(blockIndex) + 2] = rand.nextInt();
+    @Override
+    public long[] getBlockListAsLongs() {
+      long[] longs = new long[2+values.size()];
+      longs[0] = finalizedBlocks;
+      longs[1] = numBlocks - finalizedBlocks;
+      for (int i=0; i < longs.length; i++) {
+        longs[i] = values.get(i);
+      }
+      return longs;
     }
-    return oldGS;
-  }
 
-  /**
-   * Corrupt the length of the block with the given index by truncation.
-   * Not meant to be used outside of tests.
-   */
-  @VisibleForTesting
-  public long corruptBlockLengthForTesting(final int blockIndex, Random rand) {
-    long oldLength = blockList[index2BlockId(blockIndex) + 1];
-    blockList[index2BlockId(blockIndex) + 1] =
-        rand.nextInt((int) oldLength - 1);
-    return oldLength;
+    @Override
+    public Iterator<BlockReportReplica> iterator() {
+      return new Iterator<BlockReportReplica>() {
+        private final BlockReportReplica block = new BlockReportReplica();
+        final Iterator<Long> iter = values.iterator();
+        private int currentBlockIndex = 0;
+
+        @Override
+        public boolean hasNext() {
+          return currentBlockIndex < numBlocks;
+        }
+
+        @Override
+        public BlockReportReplica next() {
+          if (currentBlockIndex == finalizedBlocks) {
+            // verify the presence of the delimiter block
+            readBlock();
+            Preconditions.checkArgument(block.getBlockId() == -1 &&
+                                        block.getNumBytes() == -1 &&
+                                        block.getGenerationStamp() == -1,
+                                        "Invalid delimiter block");
+          }
+
+          readBlock();
+          if (currentBlockIndex++ < finalizedBlocks) {
+            block.setState(ReplicaState.FINALIZED);
+          } else {
+            block.setState(ReplicaState.getState(iter.next().intValue()));
+          }
+          return block;
+        }
+
+        private void readBlock() {
+          block.setBlockId(iter.next());
+          block.setNumBytes(iter.next());
+          block.setGenerationStamp(iter.next());
+        }
+
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
   }
   
-  /**
-   * Set the indexTh block
-   * @param index - the index of the block to set
-   * @param r - the block is set to the value of the this Replica
-   */
-  private void setBlock(final int index, final Replica r) {
-    int pos = index2BlockId(index);
-    blockList[pos] = r.getBlockId();
-    blockList[pos + 1] = r.getNumBytes();
-    blockList[pos + 2] = r.getGenerationStamp();
-    if(index < getNumberOfFinalizedReplicas())
-      return;
-    assert r.getState() != ReplicaState.FINALIZED :
-      "Must be under-construction replica.";
-    blockList[pos + 3] = r.getState().getValue();
-  }
-
-  /**
-   * Set the indexTh block
-   * @param index - the index of the block to set
-   * @param b - the block is set to the value of the this Block
-   */
-  private void setBlock(final int index, final Block b) {
-    int pos = index2BlockId(index);
-    blockList[pos] = b.getBlockId();
-    blockList[pos + 1] = b.getNumBytes();
-    blockList[pos + 2] = b.getGenerationStamp();
-  }
-
-  /**
-   * Set the invalid delimiting block between the finalized and
-   * the under-construction lists.
-   * The invalid block has all three fields set to -1.
-   * @param finalizedSzie - the size of the finalized list
-   */
-  private void setDelimitingBlock(final int finalizedSzie) {
-    int idx = HEADER_SIZE + finalizedSzie * LONGS_PER_FINALIZED_BLOCK;
-    blockList[idx] = -1;
-    blockList[idx+1] = -1;
-    blockList[idx+2] = -1;
-  }
-
-  public long getMaxGsInBlockList() {
-    long maxGs = -1;
-    Iterator<Block> iter = getBlockReportIterator();
-    while (iter.hasNext()) {
-      Block b = iter.next();
-      if (b.getGenerationStamp() > maxGs) {
-        maxGs = b.getGenerationStamp();
+  @InterfaceAudience.Private
+  public static class BlockReportReplica extends Block implements Replica {
+    private ReplicaState state;
+    private BlockReportReplica() {
+    }
+    public BlockReportReplica(Block block) {
+      super(block);
+      if (block instanceof BlockReportReplica) {
+        this.state = ((BlockReportReplica)block).getState();
+      } else {
+        this.state = ReplicaState.FINALIZED;
       }
     }
-    return maxGs;
+    public void setState(ReplicaState state) {
+      this.state = state;
+    }
+    @Override
+    public ReplicaState getState() {
+      return state;
+    }
+    @Override
+    public long getBytesOnDisk() {
+      return getNumBytes();
+    }
+    @Override
+    public long getVisibleLength() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public String getStorageUuid() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public boolean isOnTransientStorage() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public boolean equals(Object o) {
+      return super.equals(o);
+    }
+    @Override
+    public int hashCode() {
+      return super.hashCode();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 192916f..c4003f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -51,6 +52,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo.Capability;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
@@ -64,6 +66,7 @@ import org.apache.hadoop.ipc.RpcClientUtil;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
@@ -83,6 +86,11 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       VersionRequestProto.newBuilder().build();
   private final static RpcController NULL_CONTROLLER = null;
   
+  @VisibleForTesting
+  public DatanodeProtocolClientSideTranslatorPB(DatanodeProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
+  }
+
   public DatanodeProtocolClientSideTranslatorPB(InetSocketAddress nameNodeAddr,
       Configuration conf) throws IOException {
     RPC.setProtocolEngine(conf, DatanodeProtocolPB.class,
@@ -166,12 +174,20 @@ public class DatanodeProtocolClientSideTranslatorPB implements
         .newBuilder().setRegistration(PBHelper.convert(registration))
         .setBlockPoolId(poolId);
     
+    boolean useBlocksBuffer = registration.getNamespaceInfo()
+        .isCapabilitySupported(Capability.STORAGE_BLOCK_REPORT_BUFFERS);
+
     for (StorageBlockReport r : reports) {
       StorageBlockReportProto.Builder reportBuilder = StorageBlockReportProto
           .newBuilder().setStorage(PBHelper.convert(r.getStorage()));
-      long[] blocks = r.getBlocks();
-      for (int i = 0; i < blocks.length; i++) {
-        reportBuilder.addBlocks(blocks[i]);
+      BlockListAsLongs blocks = r.getBlocks();
+      if (useBlocksBuffer) {
+        reportBuilder.setNumberOfBlocks(blocks.getNumberOfBlocks());
+        reportBuilder.addAllBlocksBuffers(blocks.getBlocksBuffers());
+      } else {
+        for (long value : blocks.getBlockListAsLongs()) {
+          reportBuilder.addBlocks(value);
+        }
       }
       builder.addReports(reportBuilder.build());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 1a89090..e18081f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.protocolPB;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 
+import com.google.common.base.Preconditions;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
@@ -145,10 +147,14 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     
     int index = 0;
     for (StorageBlockReportProto s : request.getReportsList()) {
-      List<Long> blockIds = s.getBlocksList();
-      long[] blocks = new long[blockIds.size()];
-      for (int i = 0; i < blockIds.size(); i++) {
-        blocks[i] = blockIds.get(i);
+      final BlockListAsLongs blocks;
+      if (s.hasNumberOfBlocks()) { // new style buffer based reports
+        int num = (int)s.getNumberOfBlocks();
+        Preconditions.checkState(s.getBlocksCount() == 0,
+            "cannot send both blocks list and buffers");
+        blocks = BlockListAsLongs.decodeBuffers(num, s.getBlocksBuffersList());
+      } else {
+        blocks = BlockListAsLongs.decodeLongs(s.getBlocksList());
       }
       report[index++] = new StorageBlockReport(PBHelper.convert(s.getStorage()),
           blocks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index ee1603c..c428c2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -573,7 +573,7 @@ public class PBHelper {
     StorageInfoProto storage = info.getStorageInfo();
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
         info.getBlockPoolID(), storage.getCTime(), info.getBuildVersion(),
-        info.getSoftwareVersion());
+        info.getSoftwareVersion(), info.getCapabilities());
   }
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
@@ -1233,7 +1233,9 @@ public class PBHelper {
         .setBuildVersion(info.getBuildVersion())
         .setUnused(0)
         .setStorageInfo(PBHelper.convert((StorageInfo)info))
-        .setSoftwareVersion(info.getSoftwareVersion()).build();
+        .setSoftwareVersion(info.getSoftwareVersion())
+        .setCapabilities(info.getCapabilities())
+        .build();
   }
   
   // Located Block Arrays and Lists

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index c1a3e05..f155375 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -1968,11 +1968,9 @@ public class BlockManager {
     if (report == null) return;
     assert (namesystem.hasWriteLock());
     assert (storageInfo.numBlocks() == 0);
-    BlockReportIterator itBR = report.getBlockReportIterator();
 
-    while(itBR.hasNext()) {
-      Block iblk = itBR.next();
-      ReplicaState reportedState = itBR.getCurrentReplicaState();
+    for (BlockReportReplica iblk : report) {
+      ReplicaState reportedState = iblk.getState();
       
       if (shouldPostponeBlocksFromFuture &&
           namesystem.isGenStampInFuture(iblk)) {
@@ -2042,13 +2040,11 @@ public class BlockManager {
     int curIndex;
 
     if (newReport == null) {
-      newReport = new BlockListAsLongs();
+      newReport = BlockListAsLongs.EMPTY;
     }
     // scan the report and process newly reported blocks
-    BlockReportIterator itBR = newReport.getBlockReportIterator();
-    while(itBR.hasNext()) {
-      Block iblk = itBR.next();
-      ReplicaState iState = itBR.getCurrentReplicaState();
+    for (BlockReportReplica iblk : newReport) {
+      ReplicaState iState = iblk.getState();
       BlockInfoContiguous storedBlock = processReportedBlock(storageInfo,
           iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index 3ba2f54..3c20f6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -228,7 +228,7 @@ class BPServiceActor implements Runnable {
     bpos.verifyAndSetNamespaceInfo(nsInfo);
     
     // Second phase of the handshake with the NN.
-    register();
+    register(nsInfo);
   }
 
   // This is useful to make sure NN gets Heartbeat before Blockreport
@@ -468,8 +468,7 @@ class BPServiceActor implements Runnable {
 
     for(Map.Entry<DatanodeStorage, BlockListAsLongs> kvPair : perVolumeBlockLists.entrySet()) {
       BlockListAsLongs blockList = kvPair.getValue();
-      reports[i++] = new StorageBlockReport(
-          kvPair.getKey(), blockList.getBlockListAsLongs());
+      reports[i++] = new StorageBlockReport(kvPair.getKey(), blockList);
       totalBlockCount += blockList.getNumberOfBlocks();
     }
 
@@ -774,10 +773,11 @@ class BPServiceActor implements Runnable {
    *  
    * issued by the namenode to recognize registered datanodes.
    * 
+   * @param nsInfo current NamespaceInfo
    * @see FSNamesystem#registerDatanode(DatanodeRegistration)
    * @throws IOException
    */
-  void register() throws IOException {
+  void register(NamespaceInfo nsInfo) throws IOException {
     // The handshake() phase loaded the block pool storage
     // off disk - so update the bpRegistration object from that info
     bpRegistration = bpos.createRegistration();
@@ -788,6 +788,7 @@ class BPServiceActor implements Runnable {
       try {
         // Use returned registration from namenode with updated fields
         bpRegistration = bpNamenode.registerDatanode(bpRegistration);
+        bpRegistration.setNamespaceInfo(nsInfo);
         break;
       } catch(EOFException e) {  // namenode might have just restarted
         LOG.info("Problem connecting to server: " + nnAddr + " :"
@@ -915,9 +916,9 @@ class BPServiceActor implements Runnable {
     if (shouldRun()) {
       // re-retrieve namespace info to make sure that, if the NN
       // was restarted, we still match its version (HDFS-2120)
-      retrieveNamespaceInfo();
+      NamespaceInfo nsInfo = retrieveNamespaceInfo();
       // and re-register
-      register();
+      register(nsInfo);
       scheduleHeartbeat();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 486acbc..d42c00c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -1575,30 +1575,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     Map<DatanodeStorage, BlockListAsLongs> blockReportsMap =
         new HashMap<DatanodeStorage, BlockListAsLongs>();
 
-    Map<String, ArrayList<ReplicaInfo>> finalized =
-        new HashMap<String, ArrayList<ReplicaInfo>>();
-    Map<String, ArrayList<ReplicaInfo>> uc =
-        new HashMap<String, ArrayList<ReplicaInfo>>();
+    Map<String, BlockListAsLongs.Builder> builders =
+        new HashMap<String, BlockListAsLongs.Builder>();
 
     List<FsVolumeImpl> curVolumes = getVolumes();
     for (FsVolumeSpi v : curVolumes) {
-      finalized.put(v.getStorageID(), new ArrayList<ReplicaInfo>());
-      uc.put(v.getStorageID(), new ArrayList<ReplicaInfo>());
+      builders.put(v.getStorageID(), BlockListAsLongs.builder());
     }
 
     synchronized(this) {
       for (ReplicaInfo b : volumeMap.replicas(bpid)) {
         switch(b.getState()) {
           case FINALIZED:
-            finalized.get(b.getVolume().getStorageID()).add(b);
-            break;
           case RBW:
           case RWR:
-            uc.get(b.getVolume().getStorageID()).add(b);
+            builders.get(b.getVolume().getStorageID()).add(b);
             break;
           case RUR:
             ReplicaUnderRecovery rur = (ReplicaUnderRecovery)b;
-            uc.get(rur.getVolume().getStorageID()).add(rur.getOriginalReplica());
+            builders.get(rur.getVolume().getStorageID())
+                .add(rur.getOriginalReplica());
             break;
           case TEMPORARY:
             break;
@@ -1609,10 +1605,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
 
     for (FsVolumeImpl v : curVolumes) {
-      ArrayList<ReplicaInfo> finalizedList = finalized.get(v.getStorageID());
-      ArrayList<ReplicaInfo> ucList = uc.get(v.getStorageID());
       blockReportsMap.put(v.toDatanodeStorage(),
-                          new BlockListAsLongs(finalizedList, ucList));
+                          builders.get(v.getStorageID()).build());
     }
 
     return blockReportsMap;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index f20fb35..059bd28 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1302,7 +1302,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     final BlockManager bm = namesystem.getBlockManager(); 
     boolean noStaleStorages = false;
     for(StorageBlockReport r : reports) {
-      final BlockListAsLongs blocks = new BlockListAsLongs(r.getBlocks());
+      final BlockListAsLongs blocks = r.getBlocks();
       //
       // BlockManager.processReport accumulates information of prior calls
       // for the same node and storage, so the value returned by the last

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
index 9db2fca..e788137 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
@@ -40,6 +40,7 @@ public class DatanodeRegistration extends DatanodeID
   private final StorageInfo storageInfo;
   private ExportedBlockKeys exportedKeys;
   private final String softwareVersion;
+  private NamespaceInfo nsInfo;
 
   @VisibleForTesting
   public DatanodeRegistration(String uuid, DatanodeRegistration dnr) {
@@ -77,6 +78,14 @@ public class DatanodeRegistration extends DatanodeID
   public int getVersion() {
     return storageInfo.getLayoutVersion();
   }
+
+  public void setNamespaceInfo(NamespaceInfo nsInfo) {
+    this.nsInfo = nsInfo;
+  }
+
+  public NamespaceInfo getNamespaceInfo() {
+    return nsInfo;
+  }
   
   @Override // NodeRegistration
   public String getRegistrationID() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
index 0733743..a7439a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
@@ -29,6 +29,9 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.util.VersionInfo;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
 /**
  * NamespaceInfo is returned by the name-node in reply 
  * to a data-node handshake.
@@ -40,19 +43,52 @@ public class NamespaceInfo extends StorageInfo {
   final String  buildVersion;
   String blockPoolID = "";    // id of the block pool
   String softwareVersion;
+  long capabilities;
+
+  // only authoritative on the server-side to determine advertisement to
+  // clients.  enum will update the supported values
+  private static long CAPABILITIES_SUPPORTED = 0;
+
+  public enum Capability {
+    UNKNOWN(false),
+    STORAGE_BLOCK_REPORT_BUFFERS(true); // use optimized ByteString buffers
+    private final long mask;
+    Capability(boolean isSupported) {
+      int bits = ordinal() - 1;
+      mask = (bits < 0) ? 0 : (1L << bits);
+      if (isSupported) {
+        CAPABILITIES_SUPPORTED |= mask;
+      }
+    }
+    public long getMask() {
+      return mask;
+    }
+  }
 
+  // defaults to enabled capabilites since this ctor is for server
   public NamespaceInfo() {
     super(NodeType.NAME_NODE);
     buildVersion = null;
+    capabilities = CAPABILITIES_SUPPORTED;
   }
 
+  // defaults to enabled capabilites since this ctor is for server
   public NamespaceInfo(int nsID, String clusterID, String bpID,
       long cT, String buildVersion, String softwareVersion) {
+    this(nsID, clusterID, bpID, cT, buildVersion, softwareVersion,
+        CAPABILITIES_SUPPORTED);
+  }
+
+  // for use by server and/or client
+  public NamespaceInfo(int nsID, String clusterID, String bpID,
+      long cT, String buildVersion, String softwareVersion,
+      long capabilities) {
     super(HdfsConstants.NAMENODE_LAYOUT_VERSION, nsID, clusterID, cT,
         NodeType.NAME_NODE);
     blockPoolID = bpID;
     this.buildVersion = buildVersion;
     this.softwareVersion = softwareVersion;
+    this.capabilities = capabilities;
   }
 
   public NamespaceInfo(int nsID, String clusterID, String bpID, 
@@ -61,6 +97,22 @@ public class NamespaceInfo extends StorageInfo {
         VersionInfo.getVersion());
   }
   
+  public long getCapabilities() {
+    return capabilities;
+  }
+
+  @VisibleForTesting
+  public void setCapabilities(long capabilities) {
+    this.capabilities = capabilities;
+  }
+
+  public boolean isCapabilitySupported(Capability capability) {
+    Preconditions.checkArgument(capability != Capability.UNKNOWN,
+        "cannot test for unknown capability");
+    long mask = capability.getMask();
+    return (capabilities & mask) == mask;
+  }
+
   public String getBuildVersion() {
     return buildVersion;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
index 1521693..4ef5ebc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
@@ -18,14 +18,16 @@
 
 package org.apache.hadoop.hdfs.server.protocol;
 
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+
 /**
  * Block report for a Datanode storage
  */
 public class StorageBlockReport {
   private final DatanodeStorage storage;
-  private final long[] blocks;
+  private final BlockListAsLongs blocks;
   
-  public StorageBlockReport(DatanodeStorage storage, long[] blocks) {
+  public StorageBlockReport(DatanodeStorage storage, BlockListAsLongs blocks) {
     this.storage = storage;
     this.blocks = blocks;
   }
@@ -34,7 +36,7 @@ public class StorageBlockReport {
     return storage;
   }
 
-  public long[] getBlocks() {
+  public BlockListAsLongs getBlocks() {
     return blocks;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 348f346..7b3a4a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -237,6 +237,8 @@ message BlockReportRequestProto {
 message StorageBlockReportProto {
   required DatanodeStorageProto storage = 1;    // Storage
   repeated uint64 blocks = 2 [packed=true];
+  optional uint64 numberOfBlocks = 3;
+  repeated bytes blocksBuffers = 4;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index 97906b1..31e5585 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -517,6 +517,7 @@ message NamespaceInfoProto {
   required string blockPoolID = 3;          // block pool used by the namespace
   required StorageInfoProto storageInfo = 4;// Node information
   required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
+  optional uint64 capabilities = 6 [default = 0]; // feature flags
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
new file mode 100644
index 0000000..bebde18
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
@@ -0,0 +1,237 @@
+/**
+ * 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.protocol;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.Replica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo.Capability;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class TestBlockListAsLongs {
+  static Block b1 = new Block(1, 11, 111);
+  static Block b2 = new Block(2, 22, 222);
+  static Block b3 = new Block(3, 33, 333);
+  static Block b4 = new Block(4, 44, 444);
+
+  @Test
+  public void testEmptyReport() {
+    BlockListAsLongs blocks = checkReport();
+    assertArrayEquals(
+        new long[] {
+            0, 0,
+            -1, -1, -1 }, 
+        blocks.getBlockListAsLongs());
+  }
+
+  @Test
+  public void testFinalized() {
+    BlockListAsLongs blocks = checkReport(
+        new FinalizedReplica(b1, null, null));
+    assertArrayEquals(
+        new long[] {
+            1, 0,
+            1, 11, 111,
+            -1, -1, -1 }, 
+        blocks.getBlockListAsLongs());
+  }
+
+  @Test
+  public void testUc() {
+    BlockListAsLongs blocks = checkReport(
+      new ReplicaBeingWritten(b1, null, null, null));
+    assertArrayEquals(
+        new long[] {
+            0, 1,
+            -1, -1, -1,
+            1, 11, 111, ReplicaState.RBW.getValue() }, 
+        blocks.getBlockListAsLongs());
+  }
+  
+  @Test
+  public void testMix() {
+    BlockListAsLongs blocks = checkReport(
+        new FinalizedReplica(b1, null, null),
+        new FinalizedReplica(b2, null, null),
+        new ReplicaBeingWritten(b3, null, null, null),
+        new ReplicaWaitingToBeRecovered(b4, null, null));
+    assertArrayEquals(
+        new long[] {
+            2, 2,
+            1, 11, 111,
+            2, 22, 222,
+            -1, -1, -1,
+            3, 33, 333, ReplicaState.RBW.getValue(),
+            4, 44, 444, ReplicaState.RWR.getValue() },
+        blocks.getBlockListAsLongs());
+  }
+
+  @Test
+  public void testFuzz() throws InterruptedException {
+    Replica[] replicas = new Replica[100000];
+    Random rand = new Random(0);
+    for (int i=0; i<replicas.length; i++) {
+      Block b = new Block(rand.nextLong(), i, i<<4);
+      switch (rand.nextInt(2)) {
+        case 0:
+          replicas[i] = new FinalizedReplica(b, null, null);
+          break;
+        case 1:
+          replicas[i] = new ReplicaBeingWritten(b, null, null, null);
+          break;
+        case 2:
+          replicas[i] = new ReplicaWaitingToBeRecovered(b, null, null);
+          break;
+      }
+    }
+    checkReport(replicas);
+  }
+
+  private BlockListAsLongs checkReport(Replica...replicas) {
+    Map<Long, Replica> expectedReplicas = new HashMap<>();
+    for (Replica replica : replicas) {
+      expectedReplicas.put(replica.getBlockId(), replica);
+    }
+    expectedReplicas = Collections.unmodifiableMap(expectedReplicas);
+    
+    // encode the blocks and extract the buffers
+    BlockListAsLongs blocks =
+        BlockListAsLongs.encode(expectedReplicas.values());
+    List<ByteString> buffers = blocks.getBlocksBuffers();
+    
+    // convert to old-style list of longs
+    List<Long> longs = new ArrayList<Long>();
+    for (long value : blocks.getBlockListAsLongs()) {
+      longs.add(value);
+    }
+
+    // decode the buffers and verify its contents
+    BlockListAsLongs decodedBlocks =
+        BlockListAsLongs.decodeBuffers(expectedReplicas.size(), buffers);
+    checkReplicas(expectedReplicas, decodedBlocks);
+
+    // decode the long and verify its contents
+    BlockListAsLongs decodedList = BlockListAsLongs.decodeLongs(longs);
+    checkReplicas(expectedReplicas, decodedList);
+    return blocks;
+  }
+  
+  private void checkReplicas(Map<Long,Replica> expectedReplicas,
+                             BlockListAsLongs decodedBlocks) {
+    assertEquals(expectedReplicas.size(), decodedBlocks.getNumberOfBlocks());
+
+    Map<Long, Replica> reportReplicas = new HashMap<>(expectedReplicas);
+    for (BlockReportReplica replica : decodedBlocks) {
+      assertNotNull(replica);
+      Replica expected = reportReplicas.remove(replica.getBlockId());
+      assertNotNull(expected);
+      assertEquals("wrong bytes",
+          expected.getNumBytes(), replica.getNumBytes());
+      assertEquals("wrong genstamp",
+          expected.getGenerationStamp(), replica.getGenerationStamp());
+      assertEquals("wrong replica state",
+          expected.getState(), replica.getState());
+    }
+    assertTrue(reportReplicas.isEmpty());
+  }
+  
+  @Test
+  public void testDatanodeDetect() throws ServiceException, IOException {
+    final AtomicReference<BlockReportRequestProto> request =
+        new AtomicReference<>();
+
+    // just capture the outgoing PB
+    DatanodeProtocolPB mockProxy = mock(DatanodeProtocolPB.class);
+    doAnswer(new Answer<BlockReportResponseProto>() {
+      public BlockReportResponseProto answer(InvocationOnMock invocation) {
+        Object[] args = invocation.getArguments();
+        request.set((BlockReportRequestProto) args[1]);
+        return BlockReportResponseProto.newBuilder().build();
+      }
+    }).when(mockProxy).blockReport(any(RpcController.class),
+                                   any(BlockReportRequestProto.class));
+    
+    @SuppressWarnings("resource")
+    DatanodeProtocolClientSideTranslatorPB nn =
+        new DatanodeProtocolClientSideTranslatorPB(mockProxy);
+
+    DatanodeRegistration reg = DFSTestUtil.getLocalDatanodeRegistration();
+    NamespaceInfo nsInfo = new NamespaceInfo(1, "cluster", "bp", 1);
+    reg.setNamespaceInfo(nsInfo);
+
+    Replica r = new FinalizedReplica(new Block(1, 2, 3), null, null);
+    BlockListAsLongs bbl = BlockListAsLongs.encode(Collections.singleton(r));
+    DatanodeStorage storage = new DatanodeStorage("s1");
+    StorageBlockReport[] sbr = { new StorageBlockReport(storage, bbl) };    
+
+    // check DN sends new-style BR
+    request.set(null);
+    nsInfo.setCapabilities(Capability.STORAGE_BLOCK_REPORT_BUFFERS.getMask());
+    nn.blockReport(reg, "pool", sbr);
+    BlockReportRequestProto proto = request.get();
+    assertNotNull(proto);
+    assertTrue(proto.getReports(0).getBlocksList().isEmpty());
+    assertFalse(proto.getReports(0).getBlocksBuffersList().isEmpty());
+    
+    // back up to prior version and check DN sends old-style BR
+    request.set(null);
+    nsInfo.setCapabilities(Capability.UNKNOWN.getMask());
+    nn.blockReport(reg, "pool", sbr);
+    proto = request.get();
+    assertNotNull(proto);
+    assertFalse(proto.getReports(0).getBlocksList().isEmpty());
+    assertTrue(proto.getReports(0).getBlocksBuffersList().isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 6d67c7d..d9ac9e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -555,12 +555,12 @@ public class TestBlockManager {
     reset(node);
     
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        new BlockListAsLongs(null, null));
+        BlockListAsLongs.EMPTY);
     assertEquals(1, ds.getBlockReportCount());
     // send block report again, should NOT be processed
     reset(node);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        new BlockListAsLongs(null, null));
+        BlockListAsLongs.EMPTY);
     assertEquals(1, ds.getBlockReportCount());
 
     // re-register as if node restarted, should update existing node
@@ -571,7 +571,7 @@ public class TestBlockManager {
     // send block report, should be processed after restart
     reset(node);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-                     new BlockListAsLongs(null, null));
+                     BlockListAsLongs.EMPTY);
     // Reinitialize as registration with empty storage list pruned
     // node.storageMap.
     ds = node.getStorageInfos()[0];
@@ -600,7 +600,7 @@ public class TestBlockManager {
     reset(node);
     doReturn(1).when(node).numBlocks();
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        new BlockListAsLongs(null, null));
+        BlockListAsLongs.EMPTY);
     assertEquals(1, ds.getBlockReportCount());
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
index 8d9de7b..37c503c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
@@ -146,22 +147,32 @@ public abstract class BlockReportTestBase {
 
       // Walk the list of blocks until we find one each to corrupt the
       // generation stamp and length, if so requested.
-      for (int i = 0; i < blockList.getNumberOfBlocks(); ++i) {
+      BlockListAsLongs.Builder builder = BlockListAsLongs.builder();
+      for (BlockReportReplica block : blockList) {
         if (corruptOneBlockGs && !corruptedGs) {
-          blockList.corruptBlockGSForTesting(i, rand);
-          LOG.info("Corrupted the GS for block ID " + i);
+          long gsOld = block.getGenerationStamp();
+          long gsNew;
+          do {
+            gsNew = rand.nextInt();
+          } while (gsNew == gsOld);
+          block.setGenerationStamp(gsNew);
+          LOG.info("Corrupted the GS for block ID " + block);
           corruptedGs = true;
         } else if (corruptOneBlockLen && !corruptedLen) {
-          blockList.corruptBlockLengthForTesting(i, rand);
-          LOG.info("Corrupted the length for block ID " + i);
+          long lenOld = block.getNumBytes();
+          long lenNew;
+          do {
+            lenNew = rand.nextInt((int)lenOld - 1);
+          } while (lenNew == lenOld);
+          block.setNumBytes(lenNew);
+          LOG.info("Corrupted the length for block ID " + block);
           corruptedLen = true;
-        } else {
-          break;
         }
+        builder.add(new BlockReportReplica(block));
       }
 
       reports[reportIndex++] =
-          new StorageBlockReport(dnStorage, blockList.getBlockListAsLongs());
+          new StorageBlockReport(dnStorage, builder.build());
     }
 
     return reports;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/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 a4ec8d5..5c7b4ac 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
@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -271,7 +270,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
     @Override
     public ReplicaState getState() {
-      return null;
+      return finalized ? ReplicaState.FINALIZED : ReplicaState.RBW;
     }
 
     @Override
@@ -529,7 +528,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   public synchronized void injectBlocks(String bpid,
-      Iterable<Block> injectBlocks) throws IOException {
+      Iterable<? extends Block> injectBlocks) throws IOException {
     ExtendedBlock blk = new ExtendedBlock();
     if (injectBlocks != null) {
       for (Block b: injectBlocks) { // if any blocks in list is bad, reject list
@@ -582,16 +581,16 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   synchronized BlockListAsLongs getBlockReport(String bpid) {
-    final List<Replica> blocks = new ArrayList<Replica>();
+    BlockListAsLongs.Builder report = BlockListAsLongs.builder();
     final Map<Block, BInfo> map = blockMap.get(bpid);
     if (map != null) {
       for (BInfo b : map.values()) {
         if (b.isFinalized()) {
-          blocks.add(b);
+          report.add(b);
         }
       }
     }
-    return new BlockListAsLongs(blocks, null);
+    return report.build();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
index 1152c74..3238d6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
@@ -107,17 +107,18 @@ public class TestBlockHasMultipleReplicasOnSameDN {
     StorageBlockReport reports[] =
         new StorageBlockReport[cluster.getStoragesPerDatanode()];
 
-    ArrayList<Block> blocks = new ArrayList<Block>();
+    ArrayList<Replica> blocks = new ArrayList<Replica>();
 
     for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
-      blocks.add(locatedBlock.getBlock().getLocalBlock());
+      Block localBlock = locatedBlock.getBlock().getLocalBlock();
+      blocks.add(new FinalizedReplica(localBlock, null, null));
     }
 
+    BlockListAsLongs bll = BlockListAsLongs.encode(blocks);
     for (int i = 0; i < cluster.getStoragesPerDatanode(); ++i) {
-      BlockListAsLongs bll = new BlockListAsLongs(blocks);
       FsVolumeSpi v = dn.getFSDataset().getVolumes().get(i);
       DatanodeStorage dns = new DatanodeStorage(v.getStorageID());
-      reports[i] = new StorageBlockReport(dns, bll.getBlockListAsLongs());
+      reports[i] = new StorageBlockReport(dns, bll);
     }
 
     // Should not assert!

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index ba786d1..9cbad6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -25,11 +25,9 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
-import java.io.FilenameFilter;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -188,7 +186,7 @@ public class TestDataNodeVolumeFailure {
         DatanodeStorage dnStorage = kvPair.getKey();
         BlockListAsLongs blockList = kvPair.getValue();
         reports[reportIndex++] =
-            new StorageBlockReport(dnStorage, blockList.getBlockListAsLongs());
+            new StorageBlockReport(dnStorage, blockList);
     }
     
     cluster.getNameNodeRpc().blockReport(dnR, bpid, reports);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
index 7058d71..a5e4d4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
@@ -98,7 +98,7 @@ public class TestDnRespectsBlockReportSplitThreshold {
       assertThat(reports.length, is(expectedReportsPerCall));
 
       for (StorageBlockReport report : reports) {
-        BlockListAsLongs blockList = new BlockListAsLongs(report.getBlocks());
+        BlockListAsLongs blockList = report.getBlocks();
         numBlocksReported += blockList.getNumberOfBlocks();
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 5a440c4..0865e11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -195,7 +195,7 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
     final Map<DatanodeStorage, BlockListAsLongs> result =
 	new HashMap<DatanodeStorage, BlockListAsLongs>();
 
-    result.put(storage, new BlockListAsLongs(null, null));
+    result.put(storage, BlockListAsLongs.EMPTY);
     return result;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index c11abfc..bc3c6b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -26,6 +26,7 @@ import java.util.EnumSet;
 import java.util.List;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,6 +37,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -891,9 +893,9 @@ public class NNThroughputBenchmark implements Tool {
     NamespaceInfo nsInfo;
     DatanodeRegistration dnRegistration;
     DatanodeStorage storage; //only one storage 
-    final ArrayList<Block> blocks;
+    final ArrayList<BlockReportReplica> blocks;
     int nrBlocks; // actual number of blocks
-    long[] blockReportList;
+    BlockListAsLongs blockReportList;
     final int dnIdx;
 
     private static int getNodePort(int num) throws IOException {
@@ -904,7 +906,7 @@ public class NNThroughputBenchmark implements Tool {
 
     TinyDatanode(int dnIdx, int blockCapacity) throws IOException {
       this.dnIdx = dnIdx;
-      this.blocks = new ArrayList<Block>(blockCapacity);
+      this.blocks = new ArrayList<BlockReportReplica>(blockCapacity);
       this.nrBlocks = 0;
     }
 
@@ -934,8 +936,7 @@ public class NNThroughputBenchmark implements Tool {
       //first block reports
       storage = new DatanodeStorage(DatanodeStorage.generateUuid());
       final StorageBlockReport[] reports = {
-          new StorageBlockReport(storage,
-              new BlockListAsLongs(null, null).getBlockListAsLongs())
+          new StorageBlockReport(storage, BlockListAsLongs.EMPTY)
       };
       nameNodeProto.blockReport(dnRegistration, 
           nameNode.getNamesystem().getBlockPoolId(), reports);
@@ -968,19 +969,21 @@ public class NNThroughputBenchmark implements Tool {
         }
         return false;
       }
-      blocks.set(nrBlocks, blk);
+      blocks.set(nrBlocks, new BlockReportReplica(blk));
       nrBlocks++;
       return true;
     }
 
     void formBlockReport() {
       // fill remaining slots with blocks that do not exist
-      for(int idx = blocks.size()-1; idx >= nrBlocks; idx--)
-        blocks.set(idx, new Block(blocks.size() - idx, 0, 0));
-      blockReportList = new BlockListAsLongs(blocks).getBlockListAsLongs();
+      for (int idx = blocks.size()-1; idx >= nrBlocks; idx--) {
+        Block block = new Block(blocks.size() - idx, 0, 0);
+        blocks.set(idx, new BlockReportReplica(block));
+      }
+      blockReportList = BlockListAsLongs.EMPTY;
     }
 
-    long[] getBlockReportList() {
+    BlockListAsLongs getBlockReportList() {
       return blockReportList;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index fb1418a..ee80b33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
@@ -104,7 +105,7 @@ public class TestDeadDatanode {
     // Ensure blockReport from dead datanode is rejected with IOException
     StorageBlockReport[] report = { new StorageBlockReport(
         new DatanodeStorage(reg.getDatanodeUuid()),
-        new long[] { 0L, 0L, 0L }) };
+        BlockListAsLongs.EMPTY) };
     try {
       dnp.blockReport(reg, poolId, report);
       fail("Expected IOException is not thrown");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index f7dad18..7b9ea93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -219,6 +220,7 @@ public class TestFSImage {
         .manageDataDfsDirs(false)
         .manageNameDfsDirs(false)
         .waitSafeMode(false)
+        .startupOption(StartupOption.UPGRADE)
         .build();
     try {
       FileSystem fs = cluster.getFileSystem();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
index 0e605ac..2ad7b60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
@@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.OfflineEditsViewerHelper;
 import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer.Flags;
 import org.apache.hadoop.test.PathUtils;
@@ -140,8 +141,8 @@ public class TestOfflineEditsViewer {
     assertEquals(0, runOev(editsReparsed, editsParsedXml2, "xml", false));
 
     // judgment time
-    assertTrue("Test round trip",
-        filesEqualIgnoreTrailingZeros(editsParsedXml, editsParsedXml2));
+    assertTrue("Test round trip", FileUtils.contentEqualsIgnoreEOL(
+        new File(editsParsedXml), new File(editsParsedXml2), "UTF-8"));
 
     os.close();
   }
@@ -238,6 +239,10 @@ public class TestOfflineEditsViewer {
 
     ByteBuffer small = ByteBuffer.wrap(DFSTestUtil.loadFile(filenameSmall));
     ByteBuffer large = ByteBuffer.wrap(DFSTestUtil.loadFile(filenameLarge));
+    // OEV outputs with the latest layout version, so tweak the old file's
+    // contents to have latest version so checkedin binary files don't
+    // require frequent updates
+    small.put(3, (byte)NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
 
     // now correct if it's otherwise
     if (small.capacity() > large.capacity()) {


[31/50] hadoop git commit: HDFS-7838. Expose truncate API for libhdfs. (yliu)

Posted by zj...@apache.org.
HDFS-7838. Expose truncate API for libhdfs. (yliu)


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

Branch: refs/heads/YARN-2928
Commit: 48c2db34eff376c0f3a72587a5540b1e3dffafd2
Parents: ef9946c
Author: yliu <yl...@apache.org>
Authored: Tue Mar 17 07:22:17 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Tue Mar 17 07:22:17 2015 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 ++
 .../src/contrib/libwebhdfs/src/hdfs_web.c       |  6 ++++
 .../hadoop-hdfs/src/main/native/libhdfs/hdfs.c  | 37 ++++++++++++++++++++
 .../hadoop-hdfs/src/main/native/libhdfs/hdfs.h  | 15 ++++++++
 4 files changed, 60 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/48c2db34/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 9339b97..ad3e880 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -364,6 +364,8 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-6488. Support HDFS superuser in NFS gateway. (brandonli)
 
+    HDFS-7838. Expose truncate API for libhdfs. (yliu)
+
   IMPROVEMENTS
 
     HDFS-7752. Improve description for

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48c2db34/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c
index deb11ef..86b4faf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c
@@ -1124,6 +1124,12 @@ done:
     return file;
 }
 
+int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength)
+{
+    errno = ENOTSUP;
+    return -1;
+}
+
 tSize hdfsWrite(hdfsFS fs, hdfsFile file, const void* buffer, tSize length)
 {
     if (length == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48c2db34/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
index 34acccc..504d47e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
@@ -1037,6 +1037,43 @@ done:
     return file;
 }
 
+int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength)
+{
+    jobject jFS = (jobject)fs;
+    jthrowable jthr;
+    jvalue jVal;
+    jobject jPath = NULL;
+
+    JNIEnv *env = getJNIEnv();
+
+    if (!env) {
+        errno = EINTERNAL;
+        return -1;
+    }
+
+    /* Create an object of org.apache.hadoop.fs.Path */
+    jthr = constructNewObjectOfPath(env, path, &jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsTruncateFile(%s): constructNewObjectOfPath", path);
+        return -1;
+    }
+
+    jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
+                        "truncate", JMETHOD2(JPARAM(HADOOP_PATH), "J", "Z"),
+                        jPath, newlength);
+    destroyLocalReference(env, jPath);
+    if (jthr) {
+        errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
+            "hdfsTruncateFile(%s): FileSystem#truncate", path);
+        return -1;
+    }
+    if (jVal.z == JNI_TRUE) {
+        return 1;
+    }
+    return 0;
+}
+
 int hdfsUnbufferFile(hdfsFile file)
 {
     int ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48c2db34/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h
index 64889ed..5b7bc1e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.h
@@ -396,6 +396,21 @@ extern  "C" {
                           int bufferSize, short replication, tSize blocksize);
 
     /**
+     * hdfsTruncateFile - Truncate a hdfs file to given lenght.
+     * @param fs The configured filesystem handle.
+     * @param path The full path to the file.
+     * @param newlength The size the file is to be truncated to
+     * @return 1 if the file has been truncated to the desired newlength 
+     *         and is immediately available to be reused for write operations 
+     *         such as append.
+     *         0 if a background process of adjusting the length of the last 
+     *         block has been started, and clients should wait for it to
+     *         complete before proceeding with further file updates.
+     *         -1 on error.
+     */
+    int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength);
+
+    /**
      * hdfsUnbufferFile - Reduce the buffering done on a file.
      *
      * @param file  The file to unbuffer.


[15/50] hadoop git commit: HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Jean-Pierre Matsumoto.

Posted by zj...@apache.org.
HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Jean-Pierre Matsumoto.


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

Branch: refs/heads/YARN-2928
Commit: 79426f3334ade5850fbf169764f540ede00fe366
Parents: b308a8d
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sun Mar 15 14:17:35 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sun Mar 15 14:29:49 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt       |  3 +++
 .../src/site/markdown/SchedulerLoadSimulator.md       |  2 +-
 .../src/site/markdown/HadoopStreaming.md.vm           | 14 +++++++-------
 3 files changed, 11 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79426f33/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 55028cb..bb08cfe 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1100,6 +1100,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt
     synchronization. (Sean Busbey via yliu)
 
+    HADOOP-11558. Fix dead links to doc of hadoop-tools. (Jean-Pierre 
+    Matsumoto via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79426f33/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
index ca179ee..2cffc86 100644
--- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
+++ b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
@@ -43,7 +43,7 @@ The Yarn Scheduler Load Simulator (SLS) is such a tool, which can simulate large
 o
 The simulator will exercise the real Yarn `ResourceManager` removing the network factor by simulating `NodeManagers` and `ApplicationMasters` via handling and dispatching `NM`/`AMs` heartbeat events from within the same JVM. To keep tracking of scheduler behavior and performance, a scheduler wrapper will wrap the real scheduler.
 
-The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](https://hadoop.apache.org/docs/stable/rumen.html).
+The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](../hadoop-rumen/Rumen.html).
 
 The simulator will produce real time metrics while executing, including:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79426f33/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
index 0b64586..b4c5e38 100644
--- a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
+++ b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
@@ -201,7 +201,7 @@ To specify additional local temp directories use:
      -D mapred.system.dir=/tmp/system
      -D mapred.temp.dir=/tmp/temp
 
-**Note:** For more details on job configuration parameters see: [mapred-default.xml](./mapred-default.xml)
+**Note:** For more details on job configuration parameters see: [mapred-default.xml](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml)
 
 $H4 Specifying Map-Only Jobs
 
@@ -322,7 +322,7 @@ More Usage Examples
 
 $H3 Hadoop Partitioner Class
 
-Hadoop has a library class, [KeyFieldBasedPartitioner](../../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
+Hadoop has a library class, [KeyFieldBasedPartitioner](../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D stream.map.output.field.separator=. \
@@ -372,7 +372,7 @@ Sorting within each partition for the reducer(all 4 fields used for sorting)
 
 $H3 Hadoop Comparator Class
 
-Hadoop has a library class, [KeyFieldBasedComparator](../../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
+Hadoop has a library class, [KeyFieldBasedComparator](../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.job.output.key.comparator.class=org.apache.hadoop.mapreduce.lib.partition.KeyFieldBasedComparator \
@@ -406,7 +406,7 @@ Sorting output for the reducer (where second field used for sorting)
 
 $H3 Hadoop Aggregate Package
 
-Hadoop has a library package called [Aggregate](../../org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
+Hadoop has a library package called [Aggregate](../api/org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
 
 To use Aggregate, simply specify "-reducer aggregate":
 
@@ -441,7 +441,7 @@ The python program myAggregatorForKeyCount.py looks like:
 
 $H3 Hadoop Field Selection Class
 
-Hadoop has a library class, [FieldSelectionMapReduce](../../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
+Hadoop has a library class, [FieldSelectionMapReduce](../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.map.output.key.field.separator=. \
@@ -480,7 +480,7 @@ As an example, consider the problem of zipping (compressing) a set of files acro
 
 $H3 How many reducers should I use?
 
-See MapReduce Tutorial for details: [Reducer](./MapReduceTutorial.html#Reducer)
+See MapReduce Tutorial for details: [Reducer](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Reducer)
 
 $H3 If I set up an alias in my shell script, will that work after -mapper?
 
@@ -556,4 +556,4 @@ A streaming process can use the stderr to emit status information. To set a stat
 
 $H3 How do I get the Job variables in a streaming job's mapper/reducer?
 
-See [Configured Parameters](./MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.
+See [Configured Parameters](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.


[16/50] hadoop git commit: HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)

Posted by zj...@apache.org.
HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)


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

Branch: refs/heads/YARN-2928
Commit: bc9cb3e271b22069a15ca110cd60c860250aaab2
Parents: 79426f3
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Sat Mar 14 22:36:46 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Sat Mar 14 22:36:46 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 23 ++++-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  2 +
 .../datatransfer/DataTransferProtocol.java      |  5 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |  2 +-
 .../hdfs/protocol/datatransfer/Sender.java      |  4 +-
 .../hdfs/server/datanode/DataXceiver.java       | 95 ++++++++++++--------
 .../server/datanode/ShortCircuitRegistry.java   | 13 ++-
 .../src/main/proto/datatransfer.proto           | 11 +++
 .../shortcircuit/TestShortCircuitCache.java     | 63 +++++++++++++
 10 files changed, 178 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c3f9367..93237af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1154,6 +1154,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7903. Cannot recover block after truncate and delete snapshot.
     (Plamen Jeliazkov via shv)
 
+    HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and
+    fail to tell the DFSClient about it because of a network error (cmccabe)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
index ba48c79..1e915b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -69,6 +71,12 @@ import com.google.common.base.Preconditions;
 public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
 
+  public static class FailureInjector {
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      // do nothing
+    }
+  }
+
   @VisibleForTesting
   static ShortCircuitReplicaCreator
       createShortCircuitReplicaInfoCallback = null;
@@ -76,6 +84,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private final DFSClient.Conf conf;
 
   /**
+   * Injects failures into specific operations during unit tests.
+   */
+  private final FailureInjector failureInjector;
+
+  /**
    * The file name, for logging and debugging purposes.
    */
   private String fileName;
@@ -169,6 +182,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
 
   public BlockReaderFactory(DFSClient.Conf conf) {
     this.conf = conf;
+    this.failureInjector = conf.brfFailureInjector;
     this.remainingCacheTries = conf.nCachedConnRetry;
   }
 
@@ -518,11 +532,12 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     SlotId slotId = slot == null ? null : slot.getSlotId();
-    new Sender(out).requestShortCircuitFds(block, token, slotId, 1);
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1, true);
     DataInputStream in = new DataInputStream(peer.getInputStream());
     BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
     DomainSocket sock = peer.getDomainSocket();
+    failureInjector.injectRequestFileDescriptorsFailure();
     switch (resp.getStatus()) {
     case SUCCESS:
       byte buf[] = new byte[1];
@@ -532,8 +547,13 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       try {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+        if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
+          LOG.trace("Sending receipt verification byte for slot " + slot);
+          sock.getOutputStream().write(0);
+        }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
             Time.monotonicNow(), slot);
+        return new ShortCircuitReplicaInfo(replica);
       } catch (IOException e) {
         // This indicates an error reading from disk, or a format error.  Since
         // it's not a socket communication problem, we return null rather than
@@ -545,7 +565,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
         }
       }
-      return new ShortCircuitReplicaInfo(replica);
     case ERROR_UNSUPPORTED:
       if (!resp.hasShortCircuitAccessVersion()) {
         LOG.warn("short-circuit read access is disabled for " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index aac7b51..f970fef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -337,6 +337,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     final long shortCircuitCacheStaleThresholdMs;
 
     final long keyProviderCacheExpiryMs;
+    public BlockReaderFactory.FailureInjector brfFailureInjector =
+      new BlockReaderFactory.FailureInjector();
 
     public Conf(Configuration conf) {
       // The hdfsTimeout is currently the same as the ipc timeout 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
index 4be42a8..48e931d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
@@ -138,10 +138,13 @@ public interface DataTransferProtocol {
    *                          to use no slot id.
    * @param maxVersion      Maximum version of the block data the client 
    *                          can understand.
+   * @param supportsReceiptVerification  True if the client supports
+   *                          receipt verification.
    */
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion) throws IOException;
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException;
 
   /**
    * Release a pair of short-circuit FDs requested earlier.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
index 7994027..31bdc5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
@@ -186,7 +186,7 @@ public abstract class Receiver implements DataTransferProtocol {
     try {
       requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
           PBHelper.convert(proto.getHeader().getToken()),
-          slotId, proto.getMaxVersion());
+          slotId, proto.getMaxVersion(), true);
     } finally {
       if (traceScope != null) traceScope.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index 7fea33e..df69125 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -181,7 +181,8 @@ public class Sender implements DataTransferProtocol {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException {
     OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
           .setHeader(DataTransferProtoUtil.buildBaseHeader(
@@ -189,6 +190,7 @@ public class Sender implements DataTransferProtocol {
     if (slotId != null) {
       builder.setSlotId(PBHelper.convert(slotId));
     }
+    builder.setSupportsReceiptVerification(supportsReceiptVerification);
     OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index e9547a8..84504fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -22,6 +22,8 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ER
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.DO_NOT_USE_RECEIPT_VERIFICATION;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 import static org.apache.hadoop.util.Time.now;
 
@@ -291,64 +293,83 @@ class DataXceiver extends Receiver implements Runnable {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> token,
-      SlotId slotId, int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException {
     updateCurrentThreadName("Passing file descriptors for block " + blk);
     BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
     FileInputStream fis[] = null;
+    SlotId registeredSlotId = null;
+    boolean success = false;
     try {
-      if (peer.getDomainSocket() == null) {
-        throw new IOException("You cannot pass file descriptors over " +
-            "anything but a UNIX domain socket.");
-      }
-      if (slotId != null) {
-        boolean isCached = datanode.data.
-            isCached(blk.getBlockPoolId(), blk.getBlockId());
-        datanode.shortCircuitRegistry.registerSlot(
-            ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
-      }
       try {
-        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
-      } finally {
-        if ((fis == null) && (slotId != null)) {
-          datanode.shortCircuitRegistry.unregisterSlot(slotId);
+        if (peer.getDomainSocket() == null) {
+          throw new IOException("You cannot pass file descriptors over " +
+              "anything but a UNIX domain socket.");
         }
+        if (slotId != null) {
+          boolean isCached = datanode.data.
+              isCached(blk.getBlockPoolId(), blk.getBlockId());
+          datanode.shortCircuitRegistry.registerSlot(
+              ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
+          registeredSlotId = slotId;
+        }
+        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
+        Preconditions.checkState(fis != null);
+        bld.setStatus(SUCCESS);
+        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+      } catch (ShortCircuitFdsVersionException e) {
+        bld.setStatus(ERROR_UNSUPPORTED);
+        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+        bld.setMessage(e.getMessage());
+      } catch (ShortCircuitFdsUnsupportedException e) {
+        bld.setStatus(ERROR_UNSUPPORTED);
+        bld.setMessage(e.getMessage());
+      } catch (InvalidToken e) {
+        bld.setStatus(ERROR_ACCESS_TOKEN);
+        bld.setMessage(e.getMessage());
+      } catch (IOException e) {
+        bld.setStatus(ERROR);
+        bld.setMessage(e.getMessage());
       }
-      bld.setStatus(SUCCESS);
-      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-    } catch (ShortCircuitFdsVersionException e) {
-      bld.setStatus(ERROR_UNSUPPORTED);
-      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-      bld.setMessage(e.getMessage());
-    } catch (ShortCircuitFdsUnsupportedException e) {
-      bld.setStatus(ERROR_UNSUPPORTED);
-      bld.setMessage(e.getMessage());
-    } catch (InvalidToken e) {
-      bld.setStatus(ERROR_ACCESS_TOKEN);
-      bld.setMessage(e.getMessage());
-    } catch (IOException e) {
-      bld.setStatus(ERROR);
-      bld.setMessage(e.getMessage());
-    }
-    try {
       bld.build().writeDelimitedTo(socketOut);
       if (fis != null) {
         FileDescriptor fds[] = new FileDescriptor[fis.length];
         for (int i = 0; i < fds.length; i++) {
           fds[i] = fis[i].getFD();
         }
-        byte buf[] = new byte[] { (byte)0 };
-        peer.getDomainSocket().
-          sendFileDescriptors(fds, buf, 0, buf.length);
+        byte buf[] = new byte[1];
+        if (supportsReceiptVerification) {
+          buf[0] = (byte)USE_RECEIPT_VERIFICATION.getNumber();
+        } else {
+          buf[0] = (byte)DO_NOT_USE_RECEIPT_VERIFICATION.getNumber();
+        }
+        DomainSocket sock = peer.getDomainSocket();
+        sock.sendFileDescriptors(fds, buf, 0, buf.length);
+        if (supportsReceiptVerification) {
+          LOG.trace("Reading receipt verification byte for " + slotId);
+          int val = sock.getInputStream().read();
+          if (val < 0) {
+            throw new EOFException();
+          }
+        } else {
+          LOG.trace("Receipt verification is not enabled on the DataNode.  " +
+                    "Not verifying " + slotId);
+        }
+        success = true;
       }
     } finally {
+      if ((!success) && (registeredSlotId != null)) {
+        LOG.info("Unregistering " + registeredSlotId + " because the " +
+            "requestShortCircuitFdsForRead operation failed.");
+        datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
+      }
       if (ClientTraceLog.isInfoEnabled()) {
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
             .getBlockPoolId());
         BlockSender.ClientTraceLog.info(String.format(
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             " blockid: %s, srvID: %s, success: %b",
-            blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
-          ));
+            blk.getBlockId(), dnR.getDatanodeUuid(), success));
       }
       if (fis != null) {
         IOUtils.cleanup(LOG, fis);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
index 32906f4..b32c0d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
@@ -30,6 +30,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -83,7 +84,7 @@ public class ShortCircuitRegistry {
 
   private static final int SHM_LENGTH = 8192;
 
-  private static class RegisteredShm extends ShortCircuitShm
+  public static class RegisteredShm extends ShortCircuitShm
       implements DomainSocketWatcher.Handler {
     private final String clientName;
     private final ShortCircuitRegistry registry;
@@ -383,4 +384,14 @@ public class ShortCircuitRegistry {
     }
     IOUtils.closeQuietly(watcher);
   }
+
+  public static interface Visitor {
+    void accept(HashMap<ShmId, RegisteredShm> segments,
+                HashMultimap<ExtendedBlockId, Slot> slots);
+  }
+
+  @VisibleForTesting
+  public synchronized void visit(Visitor visitor) {
+    visitor.accept(segments, slots);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
index d72bb5e1..8426198 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
@@ -179,6 +179,12 @@ message OpRequestShortCircuitAccessProto {
    * The shared memory slot to use, if we are using one.
    */
   optional ShortCircuitShmSlotProto slotId = 3;
+
+  /**
+   * True if the client supports verifying that the file descriptor has been
+   * sent successfully.
+   */
+  optional bool supportsReceiptVerification = 4 [default = false];
 }
 
 message ReleaseShortCircuitAccessRequestProto {
@@ -230,6 +236,11 @@ enum Status {
   IN_PROGRESS = 12;
 }
 
+enum ShortCircuitFdResponse {
+  DO_NOT_USE_RECEIPT_VERIFICATION = 0;
+  USE_RECEIPT_VERIFICATION = 1;
+}
+
 message PipelineAckProto {
   required sint64 seqno = 1;
   repeated uint32 reply = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index bfa871c..7daabd0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -36,13 +36,16 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
+import com.google.common.collect.HashMultimap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -52,11 +55,14 @@ import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.RegisteredShm;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.PerDatanodeVisitorInfo;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -615,4 +621,61 @@ public class TestShortCircuitCache {
     cluster.shutdown();
     sockDir.close();
   }
+
+  public static class TestCleanupFailureInjector
+        extends BlockReaderFactory.FailureInjector {
+    @Override
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      throw new IOException("injected I/O error");
+    }
+  }
+
+  // Regression test for HDFS-7915
+  @Test(timeout=60000)
+  public void testDataXceiverCleansUpSlotsOnFailure() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration conf = createShortCircuitConf(
+        "testDataXceiverCleansUpSlotsOnFailure", sockDir);
+    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+        1000000000L);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    final Path TEST_PATH1 = new Path("/test_file1");
+    final Path TEST_PATH2 = new Path("/test_file2");
+    final int TEST_FILE_LEN = 4096;
+    final int SEED = 0xFADE1;
+    DFSTestUtil.createFile(fs, TEST_PATH1, TEST_FILE_LEN,
+        (short)1, SEED);
+    DFSTestUtil.createFile(fs, TEST_PATH2, TEST_FILE_LEN,
+        (short)1, SEED);
+
+    // The first read should allocate one shared memory segment and slot.
+    DFSTestUtil.readFileBuffer(fs, TEST_PATH1);
+
+    // The second read should fail, and we should only have 1 segment and 1 slot
+    // left.
+    fs.getClient().getConf().brfFailureInjector =
+        new TestCleanupFailureInjector();
+    try {
+      DFSTestUtil.readFileBuffer(fs, TEST_PATH2);
+    } catch (Throwable t) {
+      GenericTestUtils.assertExceptionContains("TCP reads were disabled for " +
+          "testing, but we failed to do a non-TCP read.", t);
+    }
+    ShortCircuitRegistry registry =
+      cluster.getDataNodes().get(0).getShortCircuitRegistry();
+    registry.visit(new ShortCircuitRegistry.Visitor() {
+      @Override
+      public void accept(HashMap<ShmId, RegisteredShm> segments,
+                         HashMultimap<ExtendedBlockId, Slot> slots) {
+        Assert.assertEquals(1, segments.size());
+        Assert.assertEquals(1, slots.size());
+      }
+    });
+    cluster.shutdown();
+    sockDir.close();
+  }
 }


[48/50] hadoop git commit: YARN-3039. Implemented the app-level timeline aggregator discovery service. Contributed by Junping Du.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a637914/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregatorsCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregatorsCollection.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregatorsCollection.java
index cec1d71..dd64629 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregatorsCollection.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/aggregator/TestTimelineAggregatorsCollection.java
@@ -32,6 +32,7 @@ import java.util.concurrent.Future;
 
 import com.sun.jersey.core.impl.provider.entity.XMLJAXBElementProvider;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.junit.Test;
 
 public class TestTimelineAggregatorsCollection {
@@ -45,11 +46,11 @@ public class TestTimelineAggregatorsCollection {
     final int NUM_APPS = 5;
     List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>();
     for (int i = 0; i < NUM_APPS; i++) {
-      final String appId = String.valueOf(i);
+      final ApplicationId appId = ApplicationId.newInstance(0L, i);
       Callable<Boolean> task = new Callable<Boolean>() {
         public Boolean call() {
           AppLevelTimelineAggregator aggregator =
-              new AppLevelTimelineAggregator(appId);
+              new AppLevelTimelineAggregator(appId.toString());
           return (aggregatorCollection.putIfAbsent(appId, aggregator) == aggregator);
         }
       };
@@ -79,14 +80,14 @@ public class TestTimelineAggregatorsCollection {
     final int NUM_APPS = 5;
     List<Callable<Boolean>> tasks = new ArrayList<Callable<Boolean>>();
     for (int i = 0; i < NUM_APPS; i++) {
-      final String appId = String.valueOf(i);
+      final ApplicationId appId = ApplicationId.newInstance(0L, i);
       Callable<Boolean> task = new Callable<Boolean>() {
         public Boolean call() {
           AppLevelTimelineAggregator aggregator =
-              new AppLevelTimelineAggregator(appId);
+              new AppLevelTimelineAggregator(appId.toString());
           boolean successPut =
               (aggregatorCollection.putIfAbsent(appId, aggregator) == aggregator);
-          return successPut && aggregatorCollection.remove(appId);
+          return successPut && aggregatorCollection.remove(appId.toString());
         }
       };
       tasks.add(task);


[37/50] hadoop git commit: HDFS-5356. MiniDFSCluster should close all open FileSystems when shutdown() (Contributed by Rakesh R)

Posted by zj...@apache.org.
HDFS-5356. MiniDFSCluster should close all open FileSystems when shutdown() (Contributed by Rakesh R)


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

Branch: refs/heads/YARN-2928
Commit: 018893e81ec1c43e6c79c77adec92c2edfb20cab
Parents: e537047
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Mar 17 15:32:34 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Mar 17 15:32:34 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  | 32 +++++++++++++++++---
 .../apache/hadoop/hdfs/TestFileCreation.java    |  4 +--
 .../snapshot/TestRenameWithSnapshots.java       |  4 +--
 4 files changed, 35 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/018893e8/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ad3e880..bbe1f02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -327,6 +327,9 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+    HDFS-5356. MiniDFSCluster should close all open FileSystems when shutdown()
+    (Rakesh R via vinayakumarb)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/018893e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 9208ed2..a6cc71f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -60,6 +60,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -118,6 +119,7 @@ import org.apache.hadoop.util.ToolRunner;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 /**
  * This class creates a single-process DFS cluster for junit testing.
@@ -523,7 +525,8 @@ public class MiniDFSCluster {
   private boolean federation;
   private boolean checkExitOnShutdown = true;
   protected final int storagesPerDatanode;
-  
+  private Set<FileSystem> fileSystems = Sets.newHashSet();
+
   /**
    * A unique instance identifier for the cluster. This
    * is used to disambiguate HA filesystems in the case where
@@ -1705,6 +1708,13 @@ public class MiniDFSCluster {
    * Shutdown all the nodes in the cluster.
    */
   public void shutdown(boolean deleteDfsDir) {
+    shutdown(deleteDfsDir, true);
+  }
+
+  /**
+   * Shutdown all the nodes in the cluster.
+   */
+  public void shutdown(boolean deleteDfsDir, boolean closeFileSystem) {
     LOG.info("Shutting down the Mini HDFS Cluster");
     if (checkExitOnShutdown)  {
       if (ExitUtil.terminateCalled()) {
@@ -1714,6 +1724,16 @@ public class MiniDFSCluster {
         throw new AssertionError("Test resulted in an unexpected exit");
       }
     }
+    if (closeFileSystem) {
+      for (FileSystem fs : fileSystems) {
+        try {
+          fs.close();
+        } catch (IOException ioe) {
+          LOG.warn("Exception while closing file system", ioe);
+        }
+      }
+      fileSystems.clear();
+    }
     shutdownDataNodes();
     for (NameNodeInfo nnInfo : nameNodes) {
       if (nnInfo == null) continue;
@@ -2144,8 +2164,10 @@ public class MiniDFSCluster {
    * Get a client handle to the DFS cluster for the namenode at given index.
    */
   public DistributedFileSystem getFileSystem(int nnIndex) throws IOException {
-    return (DistributedFileSystem)FileSystem.get(getURI(nnIndex),
-        nameNodes[nnIndex].conf);
+    DistributedFileSystem dfs = (DistributedFileSystem) FileSystem.get(
+        getURI(nnIndex), nameNodes[nnIndex].conf);
+    fileSystems.add(dfs);
+    return dfs;
   }
 
   /**
@@ -2153,7 +2175,9 @@ public class MiniDFSCluster {
    * This simulating different threads working on different FileSystem instances.
    */
   public FileSystem getNewFileSystemInstance(int nnIndex) throws IOException {
-    return FileSystem.newInstance(getURI(nnIndex), nameNodes[nnIndex].conf);
+    FileSystem dfs = FileSystem.newInstance(getURI(nnIndex), nameNodes[nnIndex].conf);
+    fileSystems.add(dfs);
+    return dfs;
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/018893e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index a0129da..e1c547b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -675,7 +675,7 @@ public class TestFileCreation {
 
       // restart cluster with the same namenode port as before.
       // This ensures that leases are persisted in fsimage.
-      cluster.shutdown();
+      cluster.shutdown(false, false);
       try {
         Thread.sleep(2*MAX_IDLE_TIME);
       } catch (InterruptedException e) {
@@ -687,7 +687,7 @@ public class TestFileCreation {
 
       // restart cluster yet again. This triggers the code to read in
       // persistent leases from fsimage.
-      cluster.shutdown();
+      cluster.shutdown(false, false);
       try {
         Thread.sleep(5000);
       } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/018893e8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
index a215bee..09bd2dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
@@ -519,8 +519,8 @@ public class TestRenameWithSnapshots {
     File fsnAfter = new File(testDir, "dumptree_after");
     
     SnapshotTestHelper.dumpTree2File(fsdir, fsnBefore);
-    
-    cluster.shutdown();
+
+    cluster.shutdown(false, false);
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(REPL).build();
     cluster.waitActive();


[35/50] hadoop git commit: HADOOP-11720. [JDK8] Fix javadoc errors caused by incorrect or illegal tags in hadoop-tools. Contributed by Akira AJISAKA.

Posted by zj...@apache.org.
HADOOP-11720. [JDK8] Fix javadoc errors caused by incorrect or illegal tags in hadoop-tools. Contributed by Akira AJISAKA.


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

Branch: refs/heads/YARN-2928
Commit: ef9946cd52d54200c658987c1dbc3e6fce133f77
Parents: bb243ce
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Mar 17 16:09:21 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Mar 17 16:09:21 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 .../java/org/apache/hadoop/ant/DfsTask.java     |  6 ++---
 .../org/apache/hadoop/fs/s3/S3FileSystem.java   |  4 +---
 .../hadoop/fs/s3a/S3AFastOutputStream.java      |  4 ++--
 .../hadoop/fs/s3native/NativeS3FileSystem.java  | 24 ++++++++++++--------
 .../fs/azure/AzureNativeFileSystemStore.java    | 22 ++++++------------
 .../hadoop/fs/azure/NativeAzureFileSystem.java  | 16 +++++--------
 .../hadoop/tools/CopyListingFileStatus.java     |  8 +++----
 .../apache/hadoop/tools/SimpleCopyListing.java  |  2 +-
 .../apache/hadoop/tools/util/DistCpUtils.java   |  4 ++--
 .../java/org/apache/hadoop/record/Buffer.java   |  8 +++----
 .../java/org/apache/hadoop/record/Utils.java    |  8 +++----
 12 files changed, 51 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 2e04cc1..3817054 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1108,6 +1108,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11638. OpensslSecureRandom.c pthreads_thread_id should support FreeBSD
     and Solaris in addition to Linux. (Kiran Kumar M R via cnauroth)
 
+    HADOOP-11720. [JDK8] Fix javadoc errors caused by incorrect or illegal
+    tags in hadoop-tools. (Akira AJISAKA via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/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
index 78cb360..9d0b3a4 100644
--- 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
@@ -41,8 +41,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 public class DfsTask extends Task {
 
   /**
-   * Default sink for {@link java.lang.System.out System.out}
-   * and {@link java.lang.System.err System.err}.
+   * 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 */ }
@@ -171,7 +171,7 @@ public class DfsTask extends Task {
   }
 
   /**
-   * Invoke {@link org.apache.hadoop.fs.FsShell#doMain FsShell.doMain} after a
+   * Invoke {@link org.apache.hadoop.fs.FsShell#main} after a
    * few cursory checks of the configuration.
    */
   public void execute() throws BuildException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
index dda3cf6..8bdfe9a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
@@ -44,10 +44,9 @@ import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.util.Progressable;
 
 /**
- * <p>
  * A block-based {@link FileSystem} backed by
  * <a href="http://aws.amazon.com/s3">Amazon S3</a>.
- * </p>
+ *
  * @see NativeS3FileSystem
  */
 @InterfaceAudience.Public
@@ -70,7 +69,6 @@ public class S3FileSystem extends FileSystem {
 
   /**
    * Return the protocol scheme for the FileSystem.
-   * <p/>
    *
    * @return <code>s3</code>
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
index a29c47b..6819581 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java
@@ -57,11 +57,11 @@ import java.util.concurrent.ThreadPoolExecutor;
 /**
  * Upload files/parts asap directly from a memory buffer (instead of buffering
  * to a file).
- * <p/>
+ * <p>
  * Uploads are managed low-level rather than through the AWS TransferManager.
  * This allows for uploading each part of a multi-part upload as soon as
  * the bytes are in memory, rather than waiting until the file is closed.
- * <p/>
+ * <p>
  * Unstable: statistics and error handling might evolve
  */
 @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
index acc5500..a2f9805 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
@@ -62,24 +62,29 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * <p>
  * A {@link FileSystem} for reading and writing files stored on
  * <a href="http://aws.amazon.com/s3">Amazon S3</a>.
  * Unlike {@link org.apache.hadoop.fs.s3.S3FileSystem} this implementation
  * stores files on S3 in their
  * native form so they can be read by other S3 tools.
- *
+ * <p>
  * A note about directories. S3 of course has no "native" support for them.
  * The idiom we choose then is: for any directory created by this class,
  * we use an empty object "#{dirpath}_$folder$" as a marker.
  * Further, to interoperate with other S3 tools, we also accept the following:
- *  - an object "#{dirpath}/' denoting a directory marker
- *  - if there exists any objects with the prefix "#{dirpath}/", then the
- *    directory is said to exist
- *  - if both a file with the name of a directory and a marker for that
- *    directory exists, then the *file masks the directory*, and the directory
- *    is never returned.
- * </p>
+ * <ul>
+ *   <li>an object "#{dirpath}/' denoting a directory marker</li>
+ *   <li>
+ *     if there exists any objects with the prefix "#{dirpath}/", then the
+ *     directory is said to exist
+ *   </li>
+ *   <li>
+ *     if both a file with the name of a directory and a marker for that
+ *     directory exists, then the *file masks the directory*, and the directory
+ *     is never returned.
+ *   </li>
+ * </ul>
+ *
  * @see org.apache.hadoop.fs.s3.S3FileSystem
  */
 @InterfaceAudience.Public
@@ -308,7 +313,6 @@ public class NativeS3FileSystem extends FileSystem {
 
   /**
    * Return the protocol scheme for the FileSystem.
-   * <p/>
    *
    * @return <code>s3n</code>
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index c6ba84f..5dc0963 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -600,8 +600,6 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * Azure.
    * 
    * @throws AzureException
-   * @throws ConfigurationException
-   * 
    */
   private void configureAzureStorageSession() throws AzureException {
 
@@ -705,7 +703,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    *           raised on errors communicating with Azure storage.
    * @throws IOException
    *           raised on errors performing I/O or setting up the session.
-   * @throws URISyntaxExceptions
+   * @throws URISyntaxException
    *           raised on creating mal-formed URI's.
    */
   private void connectUsingAnonymousCredentials(final URI uri)
@@ -1036,7 +1034,6 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   /**
    * Checks if the given key in Azure Storage should be stored as a page
    * blob instead of block blob.
-   * @throws URISyntaxException
    */
   public boolean isPageBlobKey(String key) {
     return isKeyForDirectorySet(key, pageBlobDirs);
@@ -1755,7 +1752,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * the path and returns a path relative to the root directory of the
    * container.
    * 
-   * @param blob
+   * @param directory
    *          - adjust the key to this directory to a path relative to the root
    *          directory
    * 
@@ -2142,14 +2139,10 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * uses a in-order first traversal of blob directory structures to maintain
    * the sorted order of the blob names.
    * 
-   * @param dir
-   *          -- Azure blob directory
-   * 
-   * @param list
-   *          -- a list of file metadata objects for each non-directory blob.
-   * 
-   * @param maxListingLength
-   *          -- maximum length of the built up list.
+   * @param aCloudBlobDirectory Azure blob directory
+   * @param aFileMetadataList a list of file metadata objects for each
+   *                          non-directory blob.
+   * @param maxListingCount maximum length of the built up list.
    */
   private void buildUpList(CloudBlobDirectoryWrapper aCloudBlobDirectory,
       ArrayList<FileMetadata> aFileMetadataList, final int maxListingCount,
@@ -2320,8 +2313,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * swallow the error since what most probably happened is that
    * the first operation succeeded on the server.
    * @param blob The blob to delete.
-   * @param leaseID A string identifying the lease, or null if no
-   *        lease is to be used.
+   * @param lease Azure blob lease, or null if no lease is to be used.
    * @throws StorageException
    */
   private void safeDelete(CloudBlobWrapper blob, SelfRenewingLease lease) throws StorageException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index e39b37d..623645a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -81,12 +81,10 @@ import com.microsoft.azure.storage.blob.CloudBlob;
 import com.microsoft.azure.storage.core.*;
 
 /**
- * <p>
  * A {@link FileSystem} for reading and writing files stored on <a
  * href="http://store.azure.com/">Windows Azure</a>. This implementation is
  * blob-based and stores files on Azure in their native form so they can be read
  * by other Azure tools.
- * </p>
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -218,9 +216,11 @@ public class NativeAzureFileSystem extends FileSystem {
     }
 
     /**
-     * Write to disk the information needed to redo folder rename, in JSON format.
-     * The file name will be wasb://<sourceFolderPrefix>/folderName-RenamePending.json
+     * Write to disk the information needed to redo folder rename,
+     * in JSON format. The file name will be
+     * {@code wasb://<sourceFolderPrefix>/folderName-RenamePending.json}
      * The file format will be:
+     * <pre>{@code
      * {
      *   FormatVersion: "1.0",
      *   OperationTime: "<YYYY-MM-DD HH:MM:SS.MMM>",
@@ -239,7 +239,7 @@ public class NativeAzureFileSystem extends FileSystem {
      *    "innerFile",
      *    "innerFile2"
      *  ]
-     * }
+     * } }</pre>
      * @throws IOException
      */
     public void writeFile(FileSystem fs) throws IOException {
@@ -913,9 +913,6 @@ public class NativeAzureFileSystem extends FileSystem {
      * The create also includes the name of the original key value which is
      * stored in the m_key member variable. This method should only be called
      * when the stream is closed.
-     * 
-     * @param anEncodedKey
-     *          Encoding of the original key stored in m_key member.
      */
     private void restoreKey() throws IOException {
       store.rename(getEncodedKey(), getKey());
@@ -1796,7 +1793,7 @@ public class NativeAzureFileSystem extends FileSystem {
    * 
    * @param permission
    *          The permission to mask.
-   * @param applyDefaultUmask
+   * @param applyMode
    *          Whether to also apply the default umask.
    * @return The masked persmission.
    */
@@ -2409,7 +2406,6 @@ public class NativeAzureFileSystem extends FileSystem {
    * recover the original key.
    * 
    * @param aKey
-   * @param numBuckets
    * @return Encoded version of the original key.
    */
   private static String encodeKey(String aKey) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
index 9b31dfb..8af799a 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
@@ -79,7 +79,7 @@ public final class CopyListingFileStatus extends FileStatus {
   /**
    * Returns the full logical ACL.
    *
-   * @return List<AclEntry> containing full logical ACL
+   * @return List containing full logical ACL
    */
   public List<AclEntry> getAclEntries() {
     return AclUtil.getAclFromPermAndEntries(getPermission(),
@@ -89,7 +89,7 @@ public final class CopyListingFileStatus extends FileStatus {
   /**
    * Sets optional ACL entries.
    *
-   * @param aclEntries List<AclEntry> containing all ACL entries
+   * @param aclEntries List containing all ACL entries
    */
   public void setAclEntries(List<AclEntry> aclEntries) {
     this.aclEntries = aclEntries;
@@ -98,7 +98,7 @@ public final class CopyListingFileStatus extends FileStatus {
   /**
    * Returns all xAttrs.
    * 
-   * @return Map<String, byte[]> containing all xAttrs
+   * @return Map containing all xAttrs
    */
   public Map<String, byte[]> getXAttrs() {
     return xAttrs != null ? xAttrs : Collections.<String, byte[]>emptyMap();
@@ -107,7 +107,7 @@ public final class CopyListingFileStatus extends FileStatus {
   /**
    * Sets optional xAttrs.
    * 
-   * @param xAttrs Map<String, byte[]> containing all xAttrs
+   * @param xAttrs Map containing all xAttrs
    */
   public void setXAttrs(Map<String, byte[]> xAttrs) {
     this.xAttrs = xAttrs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
index f9cfc86..6dc827a 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
@@ -141,7 +141,7 @@ public class SimpleCopyListing extends CopyListing {
   }
   /**
    * Collect the list of 
-   *   <sourceRelativePath, sourceFileStatus>
+   *   {@literal <sourceRelativePath, sourceFileStatus>}
    * to be copied and write to the sequence file. In essence, any file or
    * directory that need to be copied or sync-ed is written as an entry to the
    * sequence file, with the possible exception of the source root:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
index 20fdf11..d34faaf 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
@@ -270,7 +270,7 @@ public class DistCpUtils {
    *
    * @param fileSystem FileSystem containing the file
    * @param fileStatus FileStatus of file
-   * @return List<AclEntry> containing full logical ACL
+   * @return List containing full logical ACL
    * @throws IOException if there is an I/O error
    */
   public static List<AclEntry> getAcl(FileSystem fileSystem,
@@ -285,7 +285,7 @@ public class DistCpUtils {
    * 
    * @param fileSystem FileSystem containing the file
    * @param path file path
-   * @return Map<String, byte[]> containing all xAttrs
+   * @return Map containing all xAttrs
    * @throws IOException if there is an I/O error
    */
   public static Map<String, byte[]> getXAttrs(FileSystem fileSystem,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Buffer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Buffer.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Buffer.java
index 50cc1a1..737d63d 100644
--- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Buffer.java
+++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Buffer.java
@@ -25,10 +25,10 @@ import org.apache.hadoop.classification.InterfaceStability;
 
 /**
  * A byte sequence that is used as a Java native type for buffer.
- * It is resizable and distinguishes between the count of the seqeunce and
+ * It is resizable and distinguishes between the count of the sequence and
  * the current capacity.
  * 
- * @deprecated Replaced by <a href="http://hadoop.apache.org/avro/">Avro</a>.
+ * @deprecated Replaced by <a href="http://avro.apache.org/">Avro</a>.
  */
 @Deprecated
 @InterfaceAudience.Public
@@ -124,7 +124,7 @@ public class Buffer implements Comparable, Cloneable {
   
   /**
    * Change the capacity of the backing storage.
-   * The data is preserved if newCapacity >= getCount().
+   * The data is preserved if newCapacity {@literal >=} getCount().
    * @param newCapacity The new capacity in bytes.
    */
   public void setCapacity(int newCapacity) {
@@ -162,7 +162,7 @@ public class Buffer implements Comparable, Cloneable {
   public void truncate() {
     setCapacity(count);
   }
-  
+
   /**
    * Append specified bytes to the buffer.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef9946cd/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Utils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Utils.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Utils.java
index d5be59c..59e2080 100644
--- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Utils.java
+++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/record/Utils.java
@@ -28,9 +28,9 @@ import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableUtils;
 
 /**
- * Various utility functions for Hadooop record I/O runtime.
+ * Various utility functions for Hadoop record I/O runtime.
  * 
- * @deprecated Replaced by <a href="http://hadoop.apache.org/avro/">Avro</a>.
+ * @deprecated Replaced by <a href="http://avro.apache.org/">Avro</a>.
  */
 @Deprecated
 @InterfaceAudience.Public
@@ -462,8 +462,8 @@ public class Utils {
   
   /**
    * Serializes a long to a binary stream with zero-compressed encoding.
-   * For -112 <= i <= 127, only one byte is used with the actual value.
-   * For other values of i, the first byte value indicates whether the
+   * For {@literal -112 <= i <= 127}, only one byte is used with the actual
+   * value. For other values of i, the first byte value indicates whether the
    * long is positive or negative, and the number of bytes that follow.
    * If the first byte value v is between -113 and -120, the following long
    * is positive, with number of bytes that follow are -(v+112).


[18/50] hadoop git commit: YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA, Andrew Purtell, and Allen Wittenauer.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 9923806..bfe10d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -349,7 +349,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
    * Set the NM token cache for the <code>AMRMClient</code>. This cache must
    * be shared with the {@link NMClient} used to manage containers for the
    * <code>AMRMClient</code>
-   * <p/>
+   * <p>
    * If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
    * singleton instance will be used.
    *
@@ -363,7 +363,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
    * Get the NM token cache of the <code>AMRMClient</code>. This cache must be
    * shared with the {@link NMClient} used to manage containers for the
    * <code>AMRMClient</code>.
-   * <p/>
+   * <p>
    * If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
    * singleton instance will be used.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
index 721728e..08b911b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
@@ -125,7 +125,7 @@ public abstract class NMClient extends AbstractService {
    * Set the NM Token cache of the <code>NMClient</code>. This cache must be
    * shared with the {@link AMRMClient} that requested the containers managed
    * by this <code>NMClient</code>
-   * <p/>
+   * <p>
    * If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
    * singleton instance will be used.
    *
@@ -139,7 +139,7 @@ public abstract class NMClient extends AbstractService {
    * Get the NM token cache of the <code>NMClient</code>. This cache must be
    * shared with the {@link AMRMClient} that requested the containers managed
    * by this <code>NMClient</code>
-   * <p/>
+   * <p>
    * If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
    * singleton instance will be used.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java
index 0e7356f..0c349cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java
@@ -34,26 +34,26 @@ import com.google.common.annotations.VisibleForTesting;
 /**
  * NMTokenCache manages NMTokens required for an Application Master
  * communicating with individual NodeManagers.
- * <p/>
+ * <p>
  * By default Yarn client libraries {@link AMRMClient} and {@link NMClient} use
  * {@link #getSingleton()} instance of the cache.
  * <ul>
- * <li>Using the singleton instance of the cache is appropriate when running a
- * single ApplicationMaster in the same JVM.</li>
- * <li>When using the singleton, users don't need to do anything special,
- * {@link AMRMClient} and {@link NMClient} are already set up to use the default
- * singleton {@link NMTokenCache}</li>
+ *   <li>
+ *     Using the singleton instance of the cache is appropriate when running a
+ *     single ApplicationMaster in the same JVM.
+ *   </li>
+ *   <li>
+ *     When using the singleton, users don't need to do anything special,
+ *     {@link AMRMClient} and {@link NMClient} are already set up to use the
+ *     default singleton {@link NMTokenCache}
+ *     </li>
  * </ul>
- * <p/>
  * If running multiple Application Masters in the same JVM, a different cache
  * instance should be used for each Application Master.
- * <p/>
  * <ul>
- * <li>
- * If using the {@link AMRMClient} and the {@link NMClient}, setting up and using
- * an instance cache is as follows:
- * <p/>
- * 
+ *   <li>
+ *     If using the {@link AMRMClient} and the {@link NMClient}, setting up
+ *     and using an instance cache is as follows:
  * <pre>
  *   NMTokenCache nmTokenCache = new NMTokenCache();
  *   AMRMClient rmClient = AMRMClient.createAMRMClient();
@@ -61,12 +61,10 @@ import com.google.common.annotations.VisibleForTesting;
  *   nmClient.setNMTokenCache(nmTokenCache);
  *   ...
  * </pre>
- * </li>
- * <li>
- * If using the {@link AMRMClientAsync} and the {@link NMClientAsync}, setting up
- * and using an instance cache is as follows:
- * <p/>
- * 
+ *   </li>
+ *   <li>
+ *     If using the {@link AMRMClientAsync} and the {@link NMClientAsync},
+ *     setting up and using an instance cache is as follows:
  * <pre>
  *   NMTokenCache nmTokenCache = new NMTokenCache();
  *   AMRMClient rmClient = AMRMClient.createAMRMClient();
@@ -76,13 +74,11 @@ import com.google.common.annotations.VisibleForTesting;
  *   NMClientAsync nmClientAsync = new NMClientAsync("nmClient", nmClient, [NM_CALLBACK]);
  *   ...
  * </pre>
- * </li>
- * <li>
- * If using {@link ApplicationMasterProtocol} and
- * {@link ContainerManagementProtocol} directly, setting up and using an
- * instance cache is as follows:
- * <p/>
- * 
+ *   </li>
+ *   <li>
+ *     If using {@link ApplicationMasterProtocol} and
+ *     {@link ContainerManagementProtocol} directly, setting up and using an
+ *     instance cache is as follows:
  * <pre>
  *   NMTokenCache nmTokenCache = new NMTokenCache();
  *   ...
@@ -100,12 +96,12 @@ import com.google.common.annotations.VisibleForTesting;
  *   nmPro.startContainer(container, containerContext);
  *   ...
  * </pre>
- * </li>
+ *   </li>
  * </ul>
- * It is also possible to mix the usage of a client (<code>AMRMClient</code> or
- * <code>NMClient</code>, or the async versions of them) with a protocol proxy (
- * <code>ContainerManagementProtocolProxy</code> or
- * <code>ApplicationMasterProtocol</code>).
+ * It is also possible to mix the usage of a client ({@code AMRMClient} or
+ * {@code NMClient}, or the async versions of them) with a protocol proxy
+ * ({@code ContainerManagementProtocolProxy} or
+ * {@code ApplicationMasterProtocol}).
  */
 @Public
 @Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/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 d96761a..f617731 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
@@ -32,14 +32,12 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
 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.ReservationUpdateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -58,8 +56,10 @@ import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationIdNotProvidedException;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
+import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 
@@ -171,7 +171,6 @@ public abstract class YarnClient extends AbstractService {
    * <li>original tracking URL - set to "N/A"</li>
    * <li>resource usage report - all values are -1</li>
    * </ul>
-   * </p>
    * 
    * @param appId
    *          {@link ApplicationId} of the application that needs a report
@@ -184,20 +183,20 @@ public abstract class YarnClient extends AbstractService {
 
   /**
    * Get the AMRM token of the application.
-   * <p/>
+   * <p>
    * The AMRM token is required for AM to RM scheduling operations. For 
    * managed Application Masters Yarn takes care of injecting it. For unmanaged
    * Applications Masters, the token must be obtained via this method and set
    * in the {@link org.apache.hadoop.security.UserGroupInformation} of the
    * current user.
-   * <p/>
+   * <p>
    * The AMRM token will be returned only if all the following conditions are
    * met:
-   * <li>
-   *   <ul>the requester is the owner of the ApplicationMaster</ul>
-   *   <ul>the application master is an unmanaged ApplicationMaster</ul>
-   *   <ul>the application master is in ACCEPTED state</ul>
-   * </li>
+   * <ul>
+   *   <li>the requester is the owner of the ApplicationMaster</li>
+   *   <li>the application master is an unmanaged ApplicationMaster</li>
+   *   <li>the application master is in ACCEPTED state</li>
+   * </ul>
    * Else this method returns NULL.
    *
    * @param appId {@link ApplicationId} of the application to get the AMRM token
@@ -415,7 +414,7 @@ public abstract class YarnClient extends AbstractService {
    *          a report
    * @return application attempt report
    * @throws YarnException
-   * @throws {@link ApplicationAttemptNotFoundException} if application attempt
+   * @throws ApplicationAttemptNotFoundException if application attempt
    *         not found
    * @throws IOException
    */
@@ -450,7 +449,7 @@ public abstract class YarnClient extends AbstractService {
    *          {@link ContainerId} of the container that needs a report
    * @return container report
    * @throws YarnException
-   * @throws {@link ContainerNotFoundException} if container not found.
+   * @throws ContainerNotFoundException if container not found.
    * @throws IOException
    */
   public abstract ContainerReport getContainerReport(ContainerId containerId)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index e2da664..67a5494 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -344,7 +344,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   /**
    * add more labels to nodes
    * 
-   * @param addedLabelsToNode node -> labels map
+   * @param addedLabelsToNode node {@literal ->} labels map
    */
   public void addLabelsToNode(Map<NodeId, Set<String>> addedLabelsToNode)
       throws IOException {
@@ -614,7 +614,7 @@ public class CommonNodeLabelsManager extends AbstractService {
    * remove labels from nodes, labels being removed most be contained by these
    * nodes
    * 
-   * @param removeLabelsFromNode node -> labels map
+   * @param removeLabelsFromNode node {@literal ->} labels map
    */
   public void
       removeLabelsFromNode(Map<NodeId, Set<String>> removeLabelsFromNode)
@@ -668,7 +668,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   /**
    * replace labels to nodes
    * 
-   * @param replaceLabelsToNode node -> labels map
+   * @param replaceLabelsToNode node {@literal ->} labels map
    */
   public void replaceLabelsOnNode(Map<NodeId, Set<String>> replaceLabelsToNode)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
index 857d81b..4a34a09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
@@ -35,7 +35,7 @@ public abstract class NodeLabelsStore implements Closeable {
   }
   
   /**
-   * Store node -> label
+   * Store node {@literal ->} label
    */
   public abstract void updateNodeToLabelsMappings(
       Map<NodeId, Set<String>> nodeToLabels) throws IOException;
@@ -54,7 +54,6 @@ public abstract class NodeLabelsStore implements Closeable {
   
   /**
    * Recover labels and node to labels mappings from store
-   * @param conf
    */
   public abstract void recover() throws IOException;
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
index 4cacfca..4daaa68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
@@ -93,7 +93,6 @@ public class ApplicationACLsManager {
    * @param applicationAccessType
    * @param applicationOwner
    * @param applicationId
-   * @throws AccessControlException
    */
   public boolean checkAccess(UserGroupInformation callerUGI,
       ApplicationAccessType applicationAccessType, String applicationOwner,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
index e6f23f0..3343a17 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
@@ -88,7 +88,7 @@ public final class StringHelper {
   }
 
   /**
-   * Join on slash & colon (e.g., path args in routing spec)
+   * Join on slash and colon (e.g., path args in routing spec)
    * @param args to join
    * @return args joined with /:
    */
@@ -116,7 +116,7 @@ public final class StringHelper {
   }
 
   /**
-   * Split on space & trim results.
+   * Split on space and trim results.
    * @param s the string to split
    * @return an iterable of strings
    */
@@ -125,7 +125,7 @@ public final class StringHelper {
   }
 
   /**
-   * Split on _ & trim results
+   * Split on _ and trim results
    * @param s the string to split
    * @return an iterable of strings
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
index 1b5840f..bda24aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
@@ -52,13 +52,13 @@ import com.google.inject.servlet.GuiceFilter;
 /**
  * Helpers to create an embedded webapp.
  *
- * <h4>Quick start:</h4>
+ * <b>Quick start:</b>
  * <pre>
  *   WebApp wa = WebApps.$for(myApp).start();</pre>
  * Starts a webapp with default routes binds to 0.0.0.0 (all network interfaces)
  * on an ephemeral port, which can be obtained with:<pre>
  *   int port = wa.port();</pre>
- * <h4>With more options:</h4>
+ * <b>With more options:</b>
  * <pre>
  *   WebApp wa = WebApps.$for(myApp).at(address, port).
  *                        with(configuration).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
index 06a56d8..858b6b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
@@ -116,10 +116,10 @@ public class RegistryUtils {
   }
 
   /**
-   * Create a path to a service under a user & service class
+   * Create a path to a service under a user and service class
    * @param user username or ""
    * @param serviceClass service name
-   * @param serviceName service name unique for that user & service class
+   * @param serviceName service name unique for that user and service class
    * @return a full path
    */
   public static String servicePath(String user,
@@ -135,7 +135,7 @@ public class RegistryUtils {
    * Create a path for listing components under a service
    * @param user username or ""
    * @param serviceClass service name
-   * @param serviceName service name unique for that user & service class
+   * @param serviceName service name unique for that user and service class
    * @return a full path
    */
   public static String componentListPath(String user,
@@ -149,7 +149,7 @@ public class RegistryUtils {
    * Create the path to a service record for a component
    * @param user username or ""
    * @param serviceClass service name
-   * @param serviceName service name unique for that user & service class
+   * @param serviceName service name unique for that user and service class
    * @param componentName unique name/ID of the component
    * @return a full path
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
index db03936..44cefed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.registry.client.impl.zk.RegistryOperationsService;
  *
  * For SASL, the client must be operating in the context of an authed user.
  *
- * For id:pass the client must have the relevant id & password, SASL is
+ * For id:pass the client must have the relevant id and password, SASL is
  * not used even if the client has credentials.
  *
  * For anonymous, nothing is used.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
index d81f24b..edcf085 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
@@ -71,14 +71,13 @@ public interface ZookeeperConfigOptions {
    * The SASL client username: {@value}.
    * <p>
    * Set this to the <i>short</i> name of the client, e.g, "user",
-   * not <code>user/host</code>, or <code>user/host@REALM</code>
+   * not {@code user/host}, or {@code user/host@REALM}
    */
   String PROP_ZK_SASL_CLIENT_USERNAME = "zookeeper.sasl.client.username";
 
   /**
    * The SASL Server context, referring to a context in the JVM's
    * JAAS context file: {@value}
-   * <p>
    */
   String PROP_ZK_SERVER_SASL_CONTEXT =
       ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
index 3fa0c19..88e9d67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
@@ -47,16 +47,16 @@ import java.net.UnknownHostException;
 /**
  * This is a small, localhost Zookeeper service instance that is contained
  * in a YARN service...it's been derived from Apache Twill.
- *
+ * <p>
  * It implements {@link RegistryBindingSource} and provides binding information,
- * <i>once started</i>. Until <code>start()</code> is called, the hostname &
+ * <i>once started</i>. Until {@link #start()} is called, the hostname and
  * port may be undefined. Accordingly, the service raises an exception in this
  * condition.
- *
+ * <p>
  * If you wish to chain together a registry service with this one under
- * the same <code>CompositeService</code>, this service must be added
+ * the same {@code CompositeService}, this service must be added
  * as a child first.
- *
+ * <p>
  * It also sets the configuration parameter
  * {@link RegistryConstants#KEY_REGISTRY_ZK_QUORUM}
  * to its connection string. Any code with access to the service configuration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
index 85d24b3..fe2a0a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
@@ -19,9 +19,10 @@
 /**
  * Basic services for the YARN registry
  * <ul>
- *   <li>The {@link org.apache.hadoop.registry.server.services.RegistryAdminService}</ol>
- *   extends the shared Yarn Registry client with registry setup and
- *   (potentially asynchronous) administrative actions.
+ *   <li>
+ *     The {@link org.apache.hadoop.registry.server.services.RegistryAdminService}
+ *     extends the shared Yarn Registry client with registry setup and
+ *     (potentially asynchronous) administrative actions.
  *   </li>
  *   <li>
  *     The {@link org.apache.hadoop.registry.server.services.MicroZookeeperService}
@@ -33,8 +34,6 @@
  *     extends the standard YARN composite service by making its add and remove
  *     methods public. It is a utility service used in parts of the codebase
  *   </li>
- *
  * </ul>
- *
  */
 package org.apache.hadoop.registry.server.services;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
index 39c10fb..1ee8181 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
@@ -45,17 +45,15 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
- * <p>
  * Initializes {@link TimelineAuthenticationFilter} which provides support for
  * Kerberos HTTP SPNEGO authentication.
- * <p/>
  * <p>
  * It enables Kerberos HTTP SPNEGO plus delegation token authentication for the
  * timeline server.
- * <p/>
- * Refer to the <code>core-default.xml</code> file, after the comment 'HTTP
+ * <p>
+ * Refer to the {@code core-default.xml} file, after the comment 'HTTP
  * Authentication' for details on the configuration options. All related
- * configuration properties have 'hadoop.http.authentication.' as prefix.
+ * configuration properties have {@code hadoop.http.authentication.} as prefix.
  */
 public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
 
@@ -71,14 +69,11 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
   Map<String, String> filterConfig;
 
   /**
-   * <p>
    * Initializes {@link TimelineAuthenticationFilter}
-   * <p/>
    * <p>
    * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
    * configuration properties prefixed with
-   * "yarn.timeline-service.authentication."
-   * </p>
+   * {@code yarn.timeline-service.authentication.}
    * 
    * @param container
    *          The filter container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
index e675308..ba130c6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
@@ -40,7 +40,7 @@ public class ZKClient {
    * the zookeeper client library to 
    * talk to zookeeper 
    * @param string the host
-   * @throws throws IOException
+   * @throws IOException
    */
   public ZKClient(String string) throws IOException {
     zkClient = new ZooKeeper(string, 30000, new ZKWatcher());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
index 0e3d7e4..366c32c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
@@ -52,7 +52,8 @@ public abstract class RegisterNodeManagerRequest {
    * We introduce this here because currently YARN RM doesn't persist nodes info
    * for application running. When RM restart happened, we cannot determinate if
    * a node should do application cleanup (like log-aggregation, status update,
-   * etc.) or not. <p/>
+   * etc.) or not.
+   * <p>
    * When we have this running application list in node manager register
    * request, we can recover nodes info for running applications. And then we
    * can take actions accordingly

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java
index bc5825a..b21b880 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java
@@ -26,19 +26,17 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>NodeHealthStatus</code> is a summary of the health status of the
- * node.</p>
- *
- * <p>It includes information such as:
- *   <ul>
- *     <li>
- *       An indicator of whether the node is healthy, as determined by the 
- *       health-check script.
- *     </li>
- *     <li>The previous time at which the health status was reported.</li>
- *     <li>A diagnostic report on the health status.</li>
- *   </ul>
- * </p>
+ * {@code NodeHealthStatus} is a summary of the health status of the node.
+ * <p>
+ * It includes information such as:
+ * <ul>
+ *   <li>
+ *     An indicator of whether the node is healthy, as determined by the
+ *     health-check script.
+ *   </li>
+ *   <li>The previous time at which the health status was reported.</li>
+ *   <li>A diagnostic report on the health status.</li>
+ * </ul>
  * 
  * @see NodeReport
  * @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.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/ContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
index 248a393..377fd1d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
@@ -102,13 +102,15 @@ public abstract class ContainerExecutor implements Configurable {
   
   /**
    * Prepare the environment for containers in this application to execute.
+   * <pre>
    * For $x in local.dirs
    *   create $x/$user/$appId
-   * Copy $nmLocal/appTokens -> $N/$user/$appId
+   * Copy $nmLocal/appTokens {@literal ->} $N/$user/$appId
    * For $rsrc in private resources
-   *   Copy $rsrc -> $N/$user/filecache/[idef]
+   *   Copy $rsrc {@literal ->} $N/$user/filecache/[idef]
    * For $rsrc in job resources
-   *   Copy $rsrc -> $N/$user/$appId/filecache/idef
+   *   Copy $rsrc {@literal ->} $N/$user/$appId/filecache/idef
+   * </pre>
    * @param user user name of application owner
    * @param appId id of the application
    * @param nmPrivateContainerTokens path to localized credentials, rsrc by NM

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.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/util/NodeManagerHardwareUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
index 431cf5d..77db1e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
@@ -65,11 +65,11 @@ public class NodeManagerHardwareUtils {
   }
 
   /**
-   * Gets the percentage of physical CPU that is configured for YARN containers
-   * This is percent > 0 and <= 100  based on
-   * YarnConfiguration.NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT
+   * Gets the percentage of physical CPU that is configured for YARN containers.
+   * This is percent {@literal >} 0 and {@literal <=} 100 based on
+   * {@link YarnConfiguration#NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT}
    * @param conf Configuration object
-   * @return percent > 0 and <= 100
+   * @return percent {@literal >} 0 and {@literal <=} 100
    */
   public static int getNodeCpuPercentage(Configuration conf) {
     int nodeCpuPercentage =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/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 cf8c2bb..b85174e 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
@@ -216,11 +216,14 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
   /**
    * Return the flag which indicates whether the attempt failure should be
    * counted to attempt retry count.
-   * <ul>
+   * <p>
    * There failure types should not be counted to attempt retry count:
-   * <li>preempted by the scheduler.</li>
-   * <li>hardware failures, such as NM failing, lost NM and NM disk errors.</li>
-   * <li>killed by RM because of RM restart or failover.</li>
+   * <ul>
+   *   <li>preempted by the scheduler.</li>
+   *   <li>
+   *     hardware failures, such as NM failing, lost NM and NM disk errors.
+   *   </li>
+   *   <li>killed by RM because of RM restart or failover.</li>
    * </ul>
    */
   boolean shouldCountTowardsMaxAttemptRetry();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 957e8f6..2901134 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -114,7 +114,7 @@ public abstract class SchedulerNode {
 
   /**
    * Get the name of the node for scheduling matching decisions.
-   * <p/>
+   * <p>
    * Typically this is the 'hostname' reported by the node, but it could be
    * configured to be 'hostname:port' reported by the node via the
    * {@link YarnConfiguration#RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME} constant.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.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/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
index 65d6859..248cc08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
@@ -194,8 +194,7 @@ public class SchedulerUtils {
    * Utility method to validate a resource request, by insuring that the
    * requested memory/vcore is non-negative and not greater than max
    * 
-   * @throws <code>InvalidResourceRequestException</code> when there is invalid
-   *         request
+   * @throws InvalidResourceRequestException when there is invalid request
    */
   public static void validateResourceRequest(ResourceRequest resReq,
       Resource maximumResource, String queueName, YarnScheduler scheduler)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.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/policies/ComputeFairShares.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/policies/ComputeFairShares.java
index 3bea985..f4fad32 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/policies/ComputeFairShares.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/policies/ComputeFairShares.java
@@ -71,7 +71,7 @@ public class ComputeFairShares {
    * fair shares. The min and max shares and of the Schedulables are assumed to
    * be set beforehand. We compute the fairest possible allocation of shares to
    * the Schedulables that respects their min and max shares.
-   * 
+   * <p>
    * To understand what this method does, we must first define what weighted
    * fair sharing means in the presence of min and max shares. If there
    * were no minimum or maximum shares, then weighted fair sharing would be
@@ -79,30 +79,31 @@ public class ComputeFairShares {
    * Schedulable and all slots were assigned. Minimum and maximum shares add a
    * further twist - Some Schedulables may have a min share higher than their
    * assigned share or a max share lower than their assigned share.
-   * 
+   * <p>
    * To deal with these possibilities, we define an assignment of slots as being
    * fair if there exists a ratio R such that: Schedulables S where S.minShare
-   * > R * S.weight are given share S.minShare - Schedulables S where S.maxShare
-   * < R * S.weight are given S.maxShare - All other Schedulables S are
-   * assigned share R * S.weight - The sum of all the shares is totalSlots.
-   * 
+   * {@literal >} R * S.weight are given share S.minShare - Schedulables S
+   * where S.maxShare {@literal <} R * S.weight are given S.maxShare -
+   * All other Schedulables S are assigned share R * S.weight -
+   * The sum of all the shares is totalSlots.
+   * <p>
    * We call R the weight-to-slots ratio because it converts a Schedulable's
    * weight to the number of slots it is assigned.
-   * 
+   * <p>
    * We compute a fair allocation by finding a suitable weight-to-slot ratio R.
    * To do this, we use binary search. Given a ratio R, we compute the number of
    * slots that would be used in total with this ratio (the sum of the shares
    * computed using the conditions above). If this number of slots is less than
    * totalSlots, then R is too small and more slots could be assigned. If the
    * number of slots is more than totalSlots, then R is too large.
-   * 
+   * <p>
    * We begin the binary search with a lower bound on R of 0 (which means that
    * all Schedulables are only given their minShare) and an upper bound computed
    * to be large enough that too many slots are given (by doubling R until we
    * use more than totalResources resources). The helper method
    * resourceUsedWithWeightToResourceRatio computes the total resources used with a
    * given value of R.
-   * 
+   * <p>
    * The running time of this algorithm is linear in the number of Schedulables,
    * because resourceUsedWithWeightToResourceRatio is linear-time and the number of
    * iterations of binary search is a constant (dependent on desired precision).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index dfcceb8..cb456d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -364,7 +364,6 @@ public class DelegationTokenRenewer extends AbstractService {
    * @param shouldCancelAtEnd true if tokens should be canceled when the app is
    * done else false. 
    * @param user user
-   * @throws IOException
    */
   public void addApplicationAsync(ApplicationId applicationId, Credentials ts,
       boolean shouldCancelAtEnd, String user) {
@@ -634,7 +633,6 @@ public class DelegationTokenRenewer extends AbstractService {
   
   /**
    * removing failed DT
-   * @param applicationId
    */
   private void removeFailedDelegationToken(DelegationTokenToRenew t) {
     ApplicationId applicationId = t.applicationId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
index d678edf..e130225 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
@@ -181,7 +181,7 @@ public class ProxyUriUtils {
   
   /**
    * Returns the scheme if present in the url
-   * eg. "https://issues.apache.org/jira/browse/YARN" > "https"
+   * eg. "https://issues.apache.org/jira/browse/YARN" {@literal ->} "https"
    */
   public static String getSchemeFromUrl(String url) {
     int index = 0;


[11/50] hadoop git commit: MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configurable to better control to launch/kill containers. Contributed by Zhihai Xu

Posted by zj...@apache.org.
MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configurable to better control to launch/kill containers. Contributed by Zhihai Xu


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

Branch: refs/heads/YARN-2928
Commit: 9d38520c8e42530a817a7f69c9aa73a9ad40639c
Parents: 32741cf
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sat Mar 14 16:44:02 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sat Mar 14 16:44:02 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +++
 .../v2/app/launcher/ContainerLauncherImpl.java  | 14 +++++++++----
 .../v2/app/launcher/TestContainerLauncher.java  | 21 +++++++++++++++-----
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  8 ++++++++
 .../src/main/resources/mapred-default.xml       |  8 ++++++++
 5 files changed, 45 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 0bbe85c..ab6eef5 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -340,6 +340,9 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-6263. Configurable timeout between YARNRunner terminate the 
     application and forcefully kill. (Eric Payne via junping_du)
 
+    MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configurable 
+    to better control to launch/kill containers. (Zhihai Xu via ozawa)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6169. MergeQueue should release reference to the current item 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/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 666f757..9c1125d 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
@@ -70,7 +70,7 @@ public class ContainerLauncherImpl extends AbstractService implements
     new ConcurrentHashMap<ContainerId, Container>(); 
   private final AppContext context;
   protected ThreadPoolExecutor launcherPool;
-  protected static final int INITIAL_POOL_SIZE = 10;
+  protected int initialPoolSize;
   private int limitOnPoolSize;
   private Thread eventHandlingThread;
   protected BlockingQueue<ContainerLauncherEvent> eventQueue =
@@ -246,6 +246,12 @@ public class ContainerLauncherImpl extends AbstractService implements
         MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT,
         MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
     LOG.info("Upper limit on the thread pool size is " + this.limitOnPoolSize);
+
+    this.initialPoolSize = conf.getInt(
+        MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE,
+        MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE);
+    LOG.info("The thread pool initial size is " + this.initialPoolSize);
+
     super.serviceInit(conf);
     cmProxy = new ContainerManagementProtocolProxy(conf);
   }
@@ -256,7 +262,7 @@ public class ContainerLauncherImpl extends AbstractService implements
         "ContainerLauncher #%d").setDaemon(true).build();
 
     // Start with a default core-pool size of 10 and change it dynamically.
-    launcherPool = new ThreadPoolExecutor(INITIAL_POOL_SIZE,
+    launcherPool = new ThreadPoolExecutor(initialPoolSize,
         Integer.MAX_VALUE, 1, TimeUnit.HOURS,
         new LinkedBlockingQueue<Runnable>(),
         tf);
@@ -289,11 +295,11 @@ public class ContainerLauncherImpl extends AbstractService implements
             int idealPoolSize = Math.min(limitOnPoolSize, numNodes);
 
             if (poolSize < idealPoolSize) {
-              // Bump up the pool size to idealPoolSize+INITIAL_POOL_SIZE, the
+              // Bump up the pool size to idealPoolSize+initialPoolSize, the
               // later is just a buffer so we are not always increasing the
               // pool-size
               int newPoolSize = Math.min(limitOnPoolSize, idealPoolSize
-                  + INITIAL_POOL_SIZE);
+                  + initialPoolSize);
               LOG.info("Setting ContainerLauncher pool size to " + newPoolSize
                   + " as number-of-nodes to talk to is " + numNodes);
               launcherPool.setCorePoolSize(newPoolSize);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/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 dc1d72f..41ee65d 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
@@ -90,7 +90,7 @@ public class TestContainerLauncher {
 
   static final Log LOG = LogFactory.getLog(TestContainerLauncher.class);
 
-  @Test (timeout = 5000)
+  @Test (timeout = 10000)
   public void testPoolSize() throws InterruptedException {
 
     ApplicationId appId = ApplicationId.newInstance(12345, 67);
@@ -108,12 +108,14 @@ public class TestContainerLauncher {
     ThreadPoolExecutor threadPool = containerLauncher.getThreadPool();
 
     // No events yet
+    Assert.assertEquals(containerLauncher.initialPoolSize,
+        MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE);
     Assert.assertEquals(0, threadPool.getPoolSize());
-    Assert.assertEquals(ContainerLauncherImpl.INITIAL_POOL_SIZE,
+    Assert.assertEquals(containerLauncher.initialPoolSize,
       threadPool.getCorePoolSize());
     Assert.assertNull(containerLauncher.foundErrors);
 
-    containerLauncher.expectedCorePoolSize = ContainerLauncherImpl.INITIAL_POOL_SIZE;
+    containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize;
     for (int i = 0; i < 10; i++) {
       ContainerId containerId = ContainerId.newContainerId(appAttemptId, i);
       TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, i);
@@ -152,7 +154,7 @@ public class TestContainerLauncher {
     // Different hosts, there should be an increase in core-thread-pool size to
     // 21(11hosts+10buffer)
     // Core pool size should be 21 but the live pool size should be only 11.
-    containerLauncher.expectedCorePoolSize = 11 + ContainerLauncherImpl.INITIAL_POOL_SIZE;
+    containerLauncher.expectedCorePoolSize = 11 + containerLauncher.initialPoolSize;
     containerLauncher.finishEventHandling = false;
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, 21);
     TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, 21);
@@ -164,6 +166,15 @@ public class TestContainerLauncher {
     Assert.assertNull(containerLauncher.foundErrors);
 
     containerLauncher.stop();
+
+    // change configuration MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE
+    // and verify initialPoolSize value.
+    Configuration conf = new Configuration();
+    conf.setInt(MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE,
+        20);
+    containerLauncher = new CustomContainerLauncher(context);
+    containerLauncher.init(conf);
+    Assert.assertEquals(containerLauncher.initialPoolSize, 20);
   }
 
   @Test(timeout = 5000)
@@ -187,7 +198,7 @@ public class TestContainerLauncher {
     ThreadPoolExecutor threadPool = containerLauncher.getThreadPool();
 
     // 10 different hosts
-    containerLauncher.expectedCorePoolSize = ContainerLauncherImpl.INITIAL_POOL_SIZE;
+    containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize;
     for (int i = 0; i < 10; i++) {
       containerLauncher.handle(new ContainerLauncherEvent(taskAttemptId,
         containerId, "host" + i + ":1234", null,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 9f671cd..3aa304a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -504,6 +504,14 @@ public interface MRJobConfig {
   public static final int DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT = 
       500;
 
+  /**
+   * The initial size of thread pool to launch containers in the app master
+   */
+  public static final String MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE =
+      MR_AM_PREFIX+"containerlauncher.threadpool-initial-size";
+  public static final int DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE =
+      10;
+
   /** Number of threads to handle job client RPC requests.*/
   public static final String MR_AM_JOB_CLIENT_THREAD_COUNT =
     MR_AM_PREFIX + "job.client.thread-count";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index d7bec9c..820c1ac 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1694,4 +1694,12 @@
     calculated as (heapSize / mapreduce.heap.memory-mb.ratio).
   </description>
 </property>
+
+<property>
+  <name>yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size</name>
+  <value>10</value>
+  <description>The initial size of thread pool to launch containers in the
+    app master.
+  </description>
+</property>
 </configuration>


[32/50] hadoop git commit: HDFS-2360. Ugly stacktrce when quota exceeds. (harsh)

Posted by zj...@apache.org.
HDFS-2360. Ugly stacktrce when quota exceeds. (harsh)


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

Branch: refs/heads/YARN-2928
Commit: 046521cd6511b7fc6d9478cb2bed90d8e75fca20
Parents: 5608520
Author: Harsh J <ha...@cloudera.com>
Authored: Tue Mar 17 00:59:50 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Tue Mar 17 10:28:17 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                 | 2 ++
 .../main/java/org/apache/hadoop/hdfs/DFSOutputStream.java   | 9 ++++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/046521cd/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index d313b6c..9339b97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -321,6 +321,8 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HDFS-2360. Ugly stacktrace when quota exceeds. (harsh)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/046521cd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 130bb6e..286ae7d 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -551,7 +552,13 @@ public class DFSOutputStream extends FSOutputSummer
         } catch (Throwable e) {
           // Log warning if there was a real error.
           if (restartingNodeIndex.get() == -1) {
-            DFSClient.LOG.warn("DataStreamer Exception", e);
+            // Since their messages are descriptive enough, do not always
+            // log a verbose stack-trace WARN for quota exceptions.
+            if (e instanceof QuotaExceededException) {
+              DFSClient.LOG.debug("DataStreamer Quota Exception", e);
+            } else {
+              DFSClient.LOG.warn("DataStreamer Exception", e);
+            }
           }
           if (e instanceof IOException) {
             setLastException((IOException)e);


[07/50] hadoop git commit: HDFS-2605. Remove redundant "Release 0.21.1" section from CHANGES.txt. Contributed by Allen Wittenauer.

Posted by zj...@apache.org.
HDFS-2605. Remove redundant "Release 0.21.1" section from CHANGES.txt. Contributed by Allen Wittenauer.

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

Branch: refs/heads/YARN-2928
Commit: dfd32017001e6902829671dc8cc68afbca61e940
Parents: 6acb7f2
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Fri Mar 13 13:32:45 2015 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Mar 13 13:32:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd32017/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a149f18..c3f9367 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -746,6 +746,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7435. PB encoding of block reports is very inefficient.
     (Daryn Sharp via kihwal)
 
+    HDFS-2605. Remove redundant "Release 0.21.1" section from CHANGES.txt.
+    (Allen Wittenauer via shv)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
@@ -10299,8 +10302,6 @@ Release 0.22.0 - 2011-11-29
 
     HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
 
-Release 0.21.1 - Unreleased
-
     HDFS-1466. TestFcHdfsSymlink relies on /tmp/test not existing. (eli)
 
     HDFS-874. TestHDFSFileContextMainOperations fails on weirdly 


[02/50] hadoop git commit: HADOOP-11711. Provide a default value for AES/CTR/NoPadding CryptoCodec classes.

Posted by zj...@apache.org.
HADOOP-11711. Provide a default value for AES/CTR/NoPadding CryptoCodec classes.


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

Branch: refs/heads/YARN-2928
Commit: 387f271c81f7b3bf53bddc5368d5f4486530c2e1
Parents: a852910
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Mar 12 21:40:58 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Mar 12 21:40:58 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/crypto/CryptoCodec.java   | 10 +++++-
 .../fs/CommonConfigurationKeysPublic.java       | 11 ++++++
 .../crypto/TestCryptoStreamsForLocalFS.java     |  8 +----
 ...stCryptoStreamsWithJceAesCtrCryptoCodec.java | 38 ++++++++++++++++++++
 ...yptoStreamsWithOpensslAesCtrCryptoCodec.java |  7 ++++
 5 files changed, 66 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
index c5ac2ae..493e23d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.util.PerformanceAdvisory;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.slf4j.Logger;
@@ -105,7 +106,14 @@ public abstract class CryptoCodec implements Configurable {
     List<Class<? extends CryptoCodec>> result = Lists.newArrayList();
     String configName = HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX + 
         cipherSuite.getConfigSuffix();
-    String codecString = conf.get(configName);
+    String codecString;
+    if (configName.equals(CommonConfigurationKeysPublic
+        .HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY)) {
+      codecString = conf.get(configName, CommonConfigurationKeysPublic
+          .HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_DEFAULT);
+    } else {
+      codecString = conf.get(configName);
+    }
     if (codecString == null) {
       PerformanceAdvisory.LOG.debug(
           "No crypto codec classes with cipher suite configured.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 470b4d0..87c2aba 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.JceAesCtrCryptoCodec;
+import org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec;
 
 /** 
  * This class contains constants for configuration keys used
@@ -299,6 +302,14 @@ public class CommonConfigurationKeysPublic {
     "hadoop.security.saslproperties.resolver.class";
   public static final String HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX = 
     "hadoop.security.crypto.codec.classes";
+  public static final String
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY =
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+          + CipherSuite.AES_CTR_NOPADDING.getConfigSuffix();
+  public static final String
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_DEFAULT =
+      OpensslAesCtrCryptoCodec.class.getName() + "," +
+          JceAesCtrCryptoCodec.class.getName();
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY =
     "hadoop.security.crypto.cipher.suite";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java
index 765a364..0a65085 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java
@@ -47,15 +47,9 @@ public class TestCryptoStreamsForLocalFS extends CryptoStreamsTestBase {
   
   @BeforeClass
   public static void init() throws Exception {
-    Configuration conf = new Configuration();
-    conf = new Configuration(false);
+    Configuration conf = new Configuration(false);
     conf.set("fs.file.impl", LocalFileSystem.class.getName());
     fileSys = FileSystem.getLocal(conf);
-    conf.set(
-        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
-            + CipherSuite.AES_CTR_NOPADDING.getConfigSuffix(),
-        OpensslAesCtrCryptoCodec.class.getName() + ","
-            + JceAesCtrCryptoCodec.class.getName());
     codec = CryptoCodec.getInstance(conf);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceAesCtrCryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceAesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceAesCtrCryptoCodec.java
new file mode 100644
index 0000000..76c39d6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceAesCtrCryptoCodec.java
@@ -0,0 +1,38 @@
+/**
+ * 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.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+
+public class TestCryptoStreamsWithJceAesCtrCryptoCodec extends 
+    TestCryptoStreams {
+
+  @BeforeClass
+  public static void init() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY,
+        JceAesCtrCryptoCodec.class.getName());
+    codec = CryptoCodec.getInstance(conf);
+    Assert.assertEquals(JceAesCtrCryptoCodec.class.getCanonicalName(),
+        codec.getClass().getCanonicalName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
index f64e8dc..4f90a0c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.crypto;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 
 public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec 
@@ -26,6 +28,11 @@ public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec
   @BeforeClass
   public static void init() throws Exception {
     Configuration conf = new Configuration();
+    conf.set(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY,
+        OpensslAesCtrCryptoCodec.class.getName());
     codec = CryptoCodec.getInstance(conf);
+    Assert.assertEquals(OpensslAesCtrCryptoCodec.class.getCanonicalName(), 
+        codec.getClass().getCanonicalName());
   }
 }


[39/50] hadoop git commit: HADOOP-11721. switch jenkins patch tester to use git clean instead of mvn clean (temp commit)

Posted by zj...@apache.org.
HADOOP-11721. switch jenkins patch tester to use git clean instead of mvn clean (temp commit)


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

Branch: refs/heads/YARN-2928
Commit: a89b087c45e549e1f5b5fc953de4657fcbb97195
Parents: 7179f94
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Mar 17 21:39:14 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Mar 17 21:39:14 2015 +0530

----------------------------------------------------------------------
 dev-support/test-patch.sh | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a89b087c/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index b0fbb80..574a4fd 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -292,6 +292,10 @@ prebuildWithoutPatch () {
     cd -
   fi
   echo "Compiling $(pwd)"
+  if [[ -d "$(pwd)"/hadoop-hdfs-project/hadoop-hdfs/target/test/data/dfs ]]; then
+    echo "Changing permission $(pwd)/hadoop-hdfs-project/hadoop-hdfs/target/test/data/dfs to avoid broken builds "
+    chmod +x -R "$(pwd)/hadoop-hdfs-project/hadoop-hdfs/target/test/data/dfs"
+  fi
   echo "$MVN clean test -DskipTests -D${PROJECT_NAME}PatchProcess -Ptest-patch > $PATCH_DIR/trunkJavacWarnings.txt 2>&1"
   $MVN clean test -DskipTests -D${PROJECT_NAME}PatchProcess -Ptest-patch > $PATCH_DIR/trunkJavacWarnings.txt 2>&1
   if [[ $? != 0 ]] ; then


[09/50] hadoop git commit: HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)

Posted by zj...@apache.org.
HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)


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

Branch: refs/heads/YARN-2928
Commit: 5aa892ed486d42ae6b94c4866b92cd2b382ea640
Parents: 6fdef76
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Mar 13 18:29:49 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Mar 13 18:29:49 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 23 ++++-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  2 +
 .../datatransfer/DataTransferProtocol.java      |  5 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |  2 +-
 .../hdfs/protocol/datatransfer/Sender.java      |  4 +-
 .../hdfs/server/datanode/DataXceiver.java       | 95 ++++++++++++--------
 .../server/datanode/ShortCircuitRegistry.java   | 13 ++-
 .../src/main/proto/datatransfer.proto           | 11 +++
 .../shortcircuit/TestShortCircuitCache.java     | 63 +++++++++++++
 10 files changed, 178 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c3f9367..ff00b0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1177,6 +1177,9 @@ Release 2.7.0 - UNRELEASED
       HDFS-7722. DataNode#checkDiskError should also remove Storage when error
       is found. (Lei Xu via Colin P. McCabe)
 
+      HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and
+      fail to tell the DFSClient about it because of a network error (cmccabe)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
index ba48c79..1e915b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -69,6 +71,12 @@ import com.google.common.base.Preconditions;
 public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
 
+  public static class FailureInjector {
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      // do nothing
+    }
+  }
+
   @VisibleForTesting
   static ShortCircuitReplicaCreator
       createShortCircuitReplicaInfoCallback = null;
@@ -76,6 +84,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private final DFSClient.Conf conf;
 
   /**
+   * Injects failures into specific operations during unit tests.
+   */
+  private final FailureInjector failureInjector;
+
+  /**
    * The file name, for logging and debugging purposes.
    */
   private String fileName;
@@ -169,6 +182,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
 
   public BlockReaderFactory(DFSClient.Conf conf) {
     this.conf = conf;
+    this.failureInjector = conf.brfFailureInjector;
     this.remainingCacheTries = conf.nCachedConnRetry;
   }
 
@@ -518,11 +532,12 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     SlotId slotId = slot == null ? null : slot.getSlotId();
-    new Sender(out).requestShortCircuitFds(block, token, slotId, 1);
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1, true);
     DataInputStream in = new DataInputStream(peer.getInputStream());
     BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
     DomainSocket sock = peer.getDomainSocket();
+    failureInjector.injectRequestFileDescriptorsFailure();
     switch (resp.getStatus()) {
     case SUCCESS:
       byte buf[] = new byte[1];
@@ -532,8 +547,13 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       try {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+        if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
+          LOG.trace("Sending receipt verification byte for slot " + slot);
+          sock.getOutputStream().write(0);
+        }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
             Time.monotonicNow(), slot);
+        return new ShortCircuitReplicaInfo(replica);
       } catch (IOException e) {
         // This indicates an error reading from disk, or a format error.  Since
         // it's not a socket communication problem, we return null rather than
@@ -545,7 +565,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
         }
       }
-      return new ShortCircuitReplicaInfo(replica);
     case ERROR_UNSUPPORTED:
       if (!resp.hasShortCircuitAccessVersion()) {
         LOG.warn("short-circuit read access is disabled for " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index aac7b51..f970fef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -337,6 +337,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     final long shortCircuitCacheStaleThresholdMs;
 
     final long keyProviderCacheExpiryMs;
+    public BlockReaderFactory.FailureInjector brfFailureInjector =
+      new BlockReaderFactory.FailureInjector();
 
     public Conf(Configuration conf) {
       // The hdfsTimeout is currently the same as the ipc timeout 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
index 4be42a8..48e931d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
@@ -138,10 +138,13 @@ public interface DataTransferProtocol {
    *                          to use no slot id.
    * @param maxVersion      Maximum version of the block data the client 
    *                          can understand.
+   * @param supportsReceiptVerification  True if the client supports
+   *                          receipt verification.
    */
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion) throws IOException;
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException;
 
   /**
    * Release a pair of short-circuit FDs requested earlier.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
index 7994027..31bdc5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
@@ -186,7 +186,7 @@ public abstract class Receiver implements DataTransferProtocol {
     try {
       requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
           PBHelper.convert(proto.getHeader().getToken()),
-          slotId, proto.getMaxVersion());
+          slotId, proto.getMaxVersion(), true);
     } finally {
       if (traceScope != null) traceScope.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index 7fea33e..df69125 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -181,7 +181,8 @@ public class Sender implements DataTransferProtocol {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException {
     OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
           .setHeader(DataTransferProtoUtil.buildBaseHeader(
@@ -189,6 +190,7 @@ public class Sender implements DataTransferProtocol {
     if (slotId != null) {
       builder.setSlotId(PBHelper.convert(slotId));
     }
+    builder.setSupportsReceiptVerification(supportsReceiptVerification);
     OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index e9547a8..84504fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -22,6 +22,8 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ER
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.DO_NOT_USE_RECEIPT_VERIFICATION;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 import static org.apache.hadoop.util.Time.now;
 
@@ -291,64 +293,83 @@ class DataXceiver extends Receiver implements Runnable {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> token,
-      SlotId slotId, int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException {
     updateCurrentThreadName("Passing file descriptors for block " + blk);
     BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
     FileInputStream fis[] = null;
+    SlotId registeredSlotId = null;
+    boolean success = false;
     try {
-      if (peer.getDomainSocket() == null) {
-        throw new IOException("You cannot pass file descriptors over " +
-            "anything but a UNIX domain socket.");
-      }
-      if (slotId != null) {
-        boolean isCached = datanode.data.
-            isCached(blk.getBlockPoolId(), blk.getBlockId());
-        datanode.shortCircuitRegistry.registerSlot(
-            ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
-      }
       try {
-        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
-      } finally {
-        if ((fis == null) && (slotId != null)) {
-          datanode.shortCircuitRegistry.unregisterSlot(slotId);
+        if (peer.getDomainSocket() == null) {
+          throw new IOException("You cannot pass file descriptors over " +
+              "anything but a UNIX domain socket.");
         }
+        if (slotId != null) {
+          boolean isCached = datanode.data.
+              isCached(blk.getBlockPoolId(), blk.getBlockId());
+          datanode.shortCircuitRegistry.registerSlot(
+              ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
+          registeredSlotId = slotId;
+        }
+        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
+        Preconditions.checkState(fis != null);
+        bld.setStatus(SUCCESS);
+        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+      } catch (ShortCircuitFdsVersionException e) {
+        bld.setStatus(ERROR_UNSUPPORTED);
+        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+        bld.setMessage(e.getMessage());
+      } catch (ShortCircuitFdsUnsupportedException e) {
+        bld.setStatus(ERROR_UNSUPPORTED);
+        bld.setMessage(e.getMessage());
+      } catch (InvalidToken e) {
+        bld.setStatus(ERROR_ACCESS_TOKEN);
+        bld.setMessage(e.getMessage());
+      } catch (IOException e) {
+        bld.setStatus(ERROR);
+        bld.setMessage(e.getMessage());
       }
-      bld.setStatus(SUCCESS);
-      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-    } catch (ShortCircuitFdsVersionException e) {
-      bld.setStatus(ERROR_UNSUPPORTED);
-      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-      bld.setMessage(e.getMessage());
-    } catch (ShortCircuitFdsUnsupportedException e) {
-      bld.setStatus(ERROR_UNSUPPORTED);
-      bld.setMessage(e.getMessage());
-    } catch (InvalidToken e) {
-      bld.setStatus(ERROR_ACCESS_TOKEN);
-      bld.setMessage(e.getMessage());
-    } catch (IOException e) {
-      bld.setStatus(ERROR);
-      bld.setMessage(e.getMessage());
-    }
-    try {
       bld.build().writeDelimitedTo(socketOut);
       if (fis != null) {
         FileDescriptor fds[] = new FileDescriptor[fis.length];
         for (int i = 0; i < fds.length; i++) {
           fds[i] = fis[i].getFD();
         }
-        byte buf[] = new byte[] { (byte)0 };
-        peer.getDomainSocket().
-          sendFileDescriptors(fds, buf, 0, buf.length);
+        byte buf[] = new byte[1];
+        if (supportsReceiptVerification) {
+          buf[0] = (byte)USE_RECEIPT_VERIFICATION.getNumber();
+        } else {
+          buf[0] = (byte)DO_NOT_USE_RECEIPT_VERIFICATION.getNumber();
+        }
+        DomainSocket sock = peer.getDomainSocket();
+        sock.sendFileDescriptors(fds, buf, 0, buf.length);
+        if (supportsReceiptVerification) {
+          LOG.trace("Reading receipt verification byte for " + slotId);
+          int val = sock.getInputStream().read();
+          if (val < 0) {
+            throw new EOFException();
+          }
+        } else {
+          LOG.trace("Receipt verification is not enabled on the DataNode.  " +
+                    "Not verifying " + slotId);
+        }
+        success = true;
       }
     } finally {
+      if ((!success) && (registeredSlotId != null)) {
+        LOG.info("Unregistering " + registeredSlotId + " because the " +
+            "requestShortCircuitFdsForRead operation failed.");
+        datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
+      }
       if (ClientTraceLog.isInfoEnabled()) {
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
             .getBlockPoolId());
         BlockSender.ClientTraceLog.info(String.format(
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             " blockid: %s, srvID: %s, success: %b",
-            blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
-          ));
+            blk.getBlockId(), dnR.getDatanodeUuid(), success));
       }
       if (fis != null) {
         IOUtils.cleanup(LOG, fis);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
index 32906f4..b32c0d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
@@ -30,6 +30,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -83,7 +84,7 @@ public class ShortCircuitRegistry {
 
   private static final int SHM_LENGTH = 8192;
 
-  private static class RegisteredShm extends ShortCircuitShm
+  public static class RegisteredShm extends ShortCircuitShm
       implements DomainSocketWatcher.Handler {
     private final String clientName;
     private final ShortCircuitRegistry registry;
@@ -383,4 +384,14 @@ public class ShortCircuitRegistry {
     }
     IOUtils.closeQuietly(watcher);
   }
+
+  public static interface Visitor {
+    void accept(HashMap<ShmId, RegisteredShm> segments,
+                HashMultimap<ExtendedBlockId, Slot> slots);
+  }
+
+  @VisibleForTesting
+  public synchronized void visit(Visitor visitor) {
+    visitor.accept(segments, slots);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
index d72bb5e1..8426198 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
@@ -179,6 +179,12 @@ message OpRequestShortCircuitAccessProto {
    * The shared memory slot to use, if we are using one.
    */
   optional ShortCircuitShmSlotProto slotId = 3;
+
+  /**
+   * True if the client supports verifying that the file descriptor has been
+   * sent successfully.
+   */
+  optional bool supportsReceiptVerification = 4 [default = false];
 }
 
 message ReleaseShortCircuitAccessRequestProto {
@@ -230,6 +236,11 @@ enum Status {
   IN_PROGRESS = 12;
 }
 
+enum ShortCircuitFdResponse {
+  DO_NOT_USE_RECEIPT_VERIFICATION = 0;
+  USE_RECEIPT_VERIFICATION = 1;
+}
+
 message PipelineAckProto {
   required sint64 seqno = 1;
   repeated uint32 reply = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index bfa871c..7daabd0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -36,13 +36,16 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
+import com.google.common.collect.HashMultimap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -52,11 +55,14 @@ import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.RegisteredShm;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.PerDatanodeVisitorInfo;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -615,4 +621,61 @@ public class TestShortCircuitCache {
     cluster.shutdown();
     sockDir.close();
   }
+
+  public static class TestCleanupFailureInjector
+        extends BlockReaderFactory.FailureInjector {
+    @Override
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      throw new IOException("injected I/O error");
+    }
+  }
+
+  // Regression test for HDFS-7915
+  @Test(timeout=60000)
+  public void testDataXceiverCleansUpSlotsOnFailure() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration conf = createShortCircuitConf(
+        "testDataXceiverCleansUpSlotsOnFailure", sockDir);
+    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+        1000000000L);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    final Path TEST_PATH1 = new Path("/test_file1");
+    final Path TEST_PATH2 = new Path("/test_file2");
+    final int TEST_FILE_LEN = 4096;
+    final int SEED = 0xFADE1;
+    DFSTestUtil.createFile(fs, TEST_PATH1, TEST_FILE_LEN,
+        (short)1, SEED);
+    DFSTestUtil.createFile(fs, TEST_PATH2, TEST_FILE_LEN,
+        (short)1, SEED);
+
+    // The first read should allocate one shared memory segment and slot.
+    DFSTestUtil.readFileBuffer(fs, TEST_PATH1);
+
+    // The second read should fail, and we should only have 1 segment and 1 slot
+    // left.
+    fs.getClient().getConf().brfFailureInjector =
+        new TestCleanupFailureInjector();
+    try {
+      DFSTestUtil.readFileBuffer(fs, TEST_PATH2);
+    } catch (Throwable t) {
+      GenericTestUtils.assertExceptionContains("TCP reads were disabled for " +
+          "testing, but we failed to do a non-TCP read.", t);
+    }
+    ShortCircuitRegistry registry =
+      cluster.getDataNodes().get(0).getShortCircuitRegistry();
+    registry.visit(new ShortCircuitRegistry.Visitor() {
+      @Override
+      public void accept(HashMap<ShmId, RegisteredShm> segments,
+                         HashMultimap<ExtendedBlockId, Slot> slots) {
+        Assert.assertEquals(1, segments.size());
+        Assert.assertEquals(1, slots.size());
+      }
+    });
+    cluster.shutdown();
+    sockDir.close();
+  }
 }


[08/50] hadoop git commit: YARN-2854. Updated the documentation of the timeline service and the generic history service. Contributed by Naganarasimha G R.

Posted by zj...@apache.org.
YARN-2854. Updated the documentation of the timeline service and the generic history service. Contributed by Naganarasimha G R.


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

Branch: refs/heads/YARN-2928
Commit: 6fdef76cc3e818856ddcc4d385c2899a8e6ba916
Parents: dfd3201
Author: Zhijie Shen <zj...@apache.org>
Authored: Fri Mar 13 13:58:42 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Mar 13 14:00:09 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../src/site/markdown/TimelineServer.md         | 318 ++++++++++---------
 .../resources/images/timeline_structure.jpg     | Bin 0 -> 23070 bytes
 3 files changed, 165 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fdef76c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 94f992d..77f8819 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -387,6 +387,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3187. Documentation of Capacity Scheduler Queue mapping based on user
     or group. (Gururaj Shetty via jianhe)
 
+    YARN-2854. Updated the documentation of the timeline service and the generic
+    history service. (Naganarasimha G R via zjshen)
+
   OPTIMIZATIONS
 
     YARN-2990. FairScheduler's delay-scheduling always waits for node-local and 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fdef76c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
index 4889936..31fe4ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
@@ -16,144 +16,122 @@ YARN Timeline Server
 ====================
 
 * [Overview](#Overview)
-* [Current Status](#Current_Status)
-* [Basic Configuration](#Basic_Configuration)
-* [Advanced Configuration](#Advanced_Configuration)
-* [Generic-data related Configuration](#Generic-data_related_Configuration)
-* [Per-framework-date related Configuration](#Per-framework-date_related_Configuration)
-* [Running Timeline server](#Running_Timeline_server)
-* [Accessing generic-data via command-line](#Accessing_generic-data_via_command-line)
-* [Publishing of per-framework data by applications](#Publishing_of_per-framework_data_by_applications)
+    * [Introduction](#Introduction)
+    * [Current Status](#Current_Status)
+    * [Timeline Structure](#Timeline_Structure)
+* [Deployment](#Deployment)
+    * [Configurations](#Configurations)
+    * [Running Timeline server](#Running_Timeline_server)
+    * [Accessing generic-data via command-line](#Accessing_generic-data_via_command-line)
+* [Publishing of application specific data](#Publishing_of_application_specific_data)
 
 Overview
---------
+---------
 
-Storage and retrieval of applications' current as well as historic information in a generic fashion is solved in YARN through the Timeline Server (previously also called Generic Application History Server). This serves two responsibilities:
+### Introduction  
 
-* Generic information about completed applications
-    
-    Generic information includes application level data like queue-name, user information etc in the ApplicationSubmissionContext, list of application-attempts that ran for an application, information about each application-attempt, list of containers run under each application-attempt, and information about each container. Generic data is stored by ResourceManager to a history-store (default implementation on a file-system) and used by the web-UI to display information about completed applications.
+ Storage and retrieval of application's current as well as historic information in a generic fashion is solved in YARN through the Timeline Server. This serves two responsibilities:
 
-* Per-framework information of running and completed applications
-    
-    Per-framework information is completely specific to an application or framework. For example, Hadoop MapReduce framework can include pieces of information like number of map tasks, reduce tasks, counters etc. Application developers can publish the specific information to the Timeline server via TimelineClient from within a client, the ApplicationMaster and/or the application's containers. This information is then queryable via REST APIs for rendering by application/framework specific UIs.
+#### Application specific information
 
-Current Status
---------------
+  Supports collection of information completely specific to an application or framework. For example, Hadoop MapReduce framework can include pieces of information like number of map tasks, reduce tasks, counters etc. Application developers can publish the specific information to the Timeline server via TimelineClient, the ApplicationMaster and/or the application's containers. This information is then queryable via REST APIs for rendering by application/framework specific UIs.
 
-Timeline sever is a work in progress. The basic storage and retrieval of information, both generic and framework specific, are in place. Timeline server doesn't work in secure mode yet. The generic information and the per-framework information are today collected and presented separately and thus are not integrated well together. Finally, the per-framework information is only available via RESTful APIs, using JSON type content - ability to install framework specific UIs in YARN isn't supported yet.
+#### Generic information about completed applications
+  
+  Previously this was done by Application History Server but with  timeline server its just one use case of Timeline server functionality. Generic information includes application level data like queue-name, user information etc in the ApplicationSubmissionContext, list of application-attempts that ran for an application, information about each application-attempt, list of containers run under each application-attempt, and information about each container. Generic data is published by ResourceManager to the timeline store and used by the web-UI to display information about completed applications.
+ 
 
-Basic Configuration
--------------------
+### Current Status
 
-Users need to configure the Timeline server before starting it. The simplest configuration you should add in `yarn-site.xml` is to set the hostname of the Timeline server.
+  Current version of Timeline sever has been completed. Essential functionality of the timeline server can work in both secure and non secure modes. The generic history service is also ridden on the timeline store. In subsequent releases we will be rolling out next generation timeline service which is scalable and reliable. Finally, the Application specific information is only available via RESTful APIs, using JSON type content - ability to install framework specific UIs in YARN isn't supported yet.
 
-```xml
-<property>
-  <description>The hostname of the Timeline service web application.</description>
-  <name>yarn.timeline-service.hostname</name>
-  <value>0.0.0.0</value>
-</property>
-```
+### Timeline Structure
 
-Advanced Configuration
-----------------------
+![Timeline Structure] (./images/timeline_structure.jpg)
 
-In addition to the hostname, admins can also configure whether the service is enabled or not, the ports of the RPC and the web interfaces, and the number of RPC handler threads.
+#### TimelineDomain
 
-```xml
-<property>
-  <description>Address for the Timeline server to start the RPC server.</description>
-  <name>yarn.timeline-service.address</name>
-  <value>${yarn.timeline-service.hostname}:10200</value>
-</property>
+  Domain is like namespace for Timeline server and users can host multiple entities, isolating them from others. Timeline server Security is defined at this level. Domain majorly stores owner info, read & write ACL information, created and modified time stamp information. Domain is uniquely identified by ID.
 
-<property>
-  <description>The http address of the Timeline service web application.</description>
-  <name>yarn.timeline-service.webapp.address</name>
-  <value>${yarn.timeline-service.hostname}:8188</value>
-</property>
+#### TimelineEntity
 
-<property>
-  <description>The https address of the Timeline service web application.</description>
-  <name>yarn.timeline-service.webapp.https.address</name>
-  <value>${yarn.timeline-service.hostname}:8190</value>
-</property>
+  Entity contains the the meta information of some conceptual entity and its related events. The entity can be an application, an application attempt, a container or whatever the user-defined object. It contains Primary filters which will be used to index the entities in TimelineStore, such that users should carefully choose the information they want to store as the primary filters. The remaining data can be stored as other information. Entity is uniquely identified by EntityId and EntityType.
 
-<property>
-  <description>Handler thread count to serve the client RPC requests.</description>
-  <name>yarn.timeline-service.handler-thread-count</name>
-  <value>10</value>
-</property>
+#### TimelineEvent
 
-<property>
-  <description>Enables cross-origin support (CORS) for web services where
-  cross-origin web response headers are needed. For example, javascript making
-  a web services request to the timeline server.</description>
-  <name>yarn.timeline-service.http-cross-origin.enabled</name>
-  <value>false</value>
-</property>
+  TimelineEvent contains the information of an event that is related to some conceptual entity of an application. Users are free to define what the event means, such as starting an application, getting allocated a container and etc.
 
-<property>
-  <description>Comma separated list of origins that are allowed for web
-  services needing cross-origin (CORS) support. Wildcards (*) and patterns
-  allowed</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-origins</name>
-  <value>*</value>
-</property>
+Deployment
+----------
 
-<property>
-  <description>Comma separated list of methods that are allowed for web
-  services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-methods</name>
-  <value>GET,POST,HEAD</value>
-</property>
+###Configurations
 
-<property>
-  <description>Comma separated list of headers that are allowed for web
-  services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-headers</name>
-  <value>X-Requested-With,Content-Type,Accept,Origin</value>
-</property>
+#### Basic Configuration
 
-<property>
-  <description>The number of seconds a pre-flighted request can be cached
-  for web services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.max-age</name>
-  <value>1800</value>
-</property>
-```
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.enabled` | Indicate to clients whether Timeline service is enabled or not. If enabled, the TimelineClient library used by end-users will post entities and events to the Timeline server. Defaults to false. |
+| `yarn.resourcemanager.system-metrics-publisher.enabled` | The setting that controls whether yarn system metrics is published on the timeline server or not by RM. Defaults to false. |
+| `yarn.timeline-service.generic-application-history.enabled` | Indicate to clients whether to query generic application data from timeline history-service. If not enabled then application data is only queried from Resource Manager. Defaults to false. |
 
-Generic-data related Configuration
-----------------------------------
+#### Advanced configuration
 
-Users can specify whether the generic data collection is enabled or not, and also choose the storage-implementation class for the generic data. There are more configurations related to generic data collection, and users can refer to `yarn-default.xml` for all of them.
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.ttl-enable` | Enable age off of timeline store data. Defaults to true. |
+| `yarn.timeline-service.ttl-ms` | Time to live for timeline store data in milliseconds. Defaults to 604800000 (7 days). |
+| `yarn.timeline-service.handler-thread-count` | Handler thread count to serve the client RPC requests. Defaults to 10. |
+| `yarn.timeline-service.client.max-retries` | Default maximum number of retires for timeline servive client. Defaults to 30. |
+| `yarn.timeline-service.client.retry-interval-ms` | Default retry time interval for timeline servive client. Defaults to 1000. |
 
-```xml
-<property>
-  <description>Indicate to ResourceManager as well as clients whether
-  history-service is enabled or not. If enabled, ResourceManager starts
-  recording historical data that Timelien service can consume. Similarly,
-  clients can redirect to the history service when applications
-  finish if this is enabled.</description>
-  <name>yarn.timeline-service.generic-application-history.enabled</name>
-  <value>false</value>
-</property>
+#### Timeline store and state store configuration
 
-<property>
-  <description>Store class name for history store, defaulting to file system
-  store</description>
-  <name>yarn.timeline-service.generic-application-history.store-class</name>
-  <value>org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore</value>
-</property>
-```
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.store-class` | Store class name for timeline store. Defaults to org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore. |
+| `yarn.timeline-service.leveldb-timeline-store.path` | Store file name for leveldb timeline store. Defaults to ${hadoop.tmp.dir}/yarn/timeline. |
+| `yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms` | Length of time to wait between deletion cycles of leveldb timeline store in milliseconds. Defaults to 300000. |
+| `yarn.timeline-service.leveldb-timeline-store.read-cache-size` | Size of read cache for uncompressed blocks for leveldb timeline store in bytes. Defaults to 104857600. |
+| `yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size` | Size of cache for recently read entity start times for leveldb timeline store in number of entities. Defaults to 10000. |
+| `yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size` | Size of cache for recently written entity start times for leveldb timeline store in number of entities. Defaults to 10000. |
+| `yarn.timeline-service.recovery.enabled` | Defaults to false. |
+| `yarn.timeline-service.state-store-class` | Store class name for timeline state store. Defaults to org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore. |
+| `yarn.timeline-service.leveldb-state-store.path` | Store file name for leveldb timeline state store. |
+
+#### Web and RPC Configuration
+
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.hostname` | The hostname of the Timeline service web application. Defaults to 0.0.0.0. |
+| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to ${yarn.timeline-service.hostname}:10200. |
+| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to ${yarn.timeline-service.hostname}:8188. |
+| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to ${yarn.timeline-service.hostname}:8190. |
+| `yarn.timeline-service.bind-host` | The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in yarn.timeline-service.address and yarn.timeline-service.webapp.address, respectively. This is most useful for making the service listen to all interfaces by setting to 0.0.0.0. |
+| `yarn.timeline-service.http-cross-origin.enabled` | Enables cross-origin support (CORS) for web services where cross-origin web response headers are needed. For example, javascript making a web services request to the timeline server. Defaults to false. |
+| `yarn.timeline-service.http-cross-origin.allowed-origins` | Comma separated list of origins that are allowed for web services needing cross-origin (CORS) support. Wildcards `(*)` and patterns allowed. Defaults to `*`. |
+| yarn.timeline-service.http-cross-origin.allowed-methods | Comma separated list of methods that are allowed for web services needing cross-origin (CORS) support. Defaults to GET,POST,HEAD. |
+| `yarn.timeline-service.http-cross-origin.allowed-headers` | Comma separated list of headers that are allowed for web services needing cross-origin (CORS) support. Defaults to X-Requested-With,Content-Type,Accept,Origin. |
+| `yarn.timeline-service.http-cross-origin.max-age` | The number of seconds a pre-flighted request can be cached for web services needing cross-origin (CORS) support. Defaults to 1800. |
+
+#### Security Configuration
+
+ Security can be enabled by setting yarn.timeline-service.http-authentication.type to kerberos and further following configurations can be done.
+
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.http-authentication.type` | Defines authentication used for the timeline server HTTP endpoint. Supported values are: simple / kerberos / #AUTHENTICATION_HANDLER_CLASSNAME#. Defaults to simple. |
+| `yarn.timeline-service.http-authentication.simple.anonymous.allowed` | Indicates if anonymous requests are allowed by the timeline server when using 'simple' authentication. Defaults to true. |
+| `yarn.timeline-service.principal` | The Kerberos principal for the timeline server. |
+| yarn.timeline-service.keytab | The Kerberos keytab for the timeline server. Defaults to /etc/krb5.keytab. |
+| `yarn.timeline-service.delegation.key.update-interval` | Defaults to 86400000 (1 day). |
+| `yarn.timeline-service.delegation.token.renew-interval` | Defaults to 86400000 (1 day). |
+| `yarn.timeline-service.delegation.token.max-lifetime` | Defaults to 604800000 (7 day). |
 
-Per-framework-date related Configuration
-----------------------------------------
+#### Enabling the timeline service and the generic history service
 
-Users can specify whether per-framework data service is enabled or not, choose the store implementation for the per-framework data, and tune the retention of the per-framework data. There are more configurations related to per-framework data service, and users can refer to `yarn-default.xml` for all of them.
+  Following are the basic configuration to start Timeline server.
 
-```xml
+```
 <property>
   <description>Indicate to clients whether Timeline service is enabled or not.
   If enabled, the TimelineClient library used by end-users will post entities
@@ -163,69 +141,97 @@ Users can specify whether per-framework data service is enabled or not, choose t
 </property>
 
 <property>
-  <description>Store class name for timeline store.</description>
-  <name>yarn.timeline-service.store-class</name>
-  <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
+    <description>The setting that controls whether yarn system metrics is
+    published on the timeline server or not by RM.</description>
+    <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
+    <value>true</value>
 </property>
 
 <property>
-  <description>Enable age off of timeline store data.</description>
-  <name>yarn.timeline-service.ttl-enable</name>
+  <description>Indicate to clients whether to query generic application data from
+  timeline history-service. If not enabled then application data is only queried 
+  from Resource Manager</description>
+  <name>yarn.timeline-service.generic-application-history.enabled</name>
   <value>true</value>
 </property>
-
-<property>
-  <description>Time to live for timeline store data in milliseconds.</description>
-  <name>yarn.timeline-service.ttl-ms</name>
-  <value>604800000</value>
-</property>
 ```
 
-Running Timeline server
------------------------
+### Running Timeline server
 
-Assuming all the aforementioned configurations are set properly, admins can start the Timeline server/history service with the following command:
+  Assuming all the aforementioned configurations are set properly, admins can start the Timeline server/history service with the following command:
 
-      $ yarn timelineserver
+```
+  $ yarn timelineserver
+```
 
-Or users can start the Timeline server / history service as a daemon:
+  Or users can start the Timeline server / history service as a daemon:
 
-      $ yarn --daemon start timelineserver
+```
+  $ yarn --daemon start timelineserver
+```
 
-Accessing generic-data via command-line
----------------------------------------
+### Accessing generic-data via command-line
 
-Users can access applications' generic historic data via the command line as below. Note that the same commands are usable to obtain the corresponding information about running applications.
+  Users can access applications' generic historic data via the command line as below. Note that the same commands are usable to obtain the corresponding information about running applications.
 
 ```
-      $ yarn application -status <Application ID>
-      $ yarn applicationattempt -list <Application ID>
-      $ yarn applicationattempt -status <Application Attempt ID>
-      $ yarn container -list <Application Attempt ID>
-      $ yarn container -status <Container ID>
+  $ yarn application -status <Application ID>
+  $ yarn applicationattempt -list <Application ID>
+  $ yarn applicationattempt -status <Application Attempt ID>
+  $ yarn container -list <Application Attempt ID>
+  $ yarn container -status <Container ID>
 ```
 
-Publishing of per-framework data by applications
+Publishing of application specific data
 ------------------------------------------------
 
-Developers can define what information they want to record for their applications by composing `TimelineEntity` and `TimelineEvent` objects, and put the entities and events to the Timeline server via `TimelineClient`. Following is an example:
-
-```java
-// Create and start the Timeline client
-TimelineClient client = TimelineClient.createTimelineClient();
-client.init(conf);
-client.start();
-
-TimelineEntity entity = null;
-// Compose the entity
-try {
-  TimelinePutResponse response = client.putEntities(entity);
-} catch (IOException e) {
-  // Handle the exception
-} catch (YarnException e) {
-  // Handle the exception
-}
-
-// Stop the Timeline client
-client.stop();
+  Developers can define what information they want to record for their applications by composing `TimelineEntity` and  `TimelineEvent` objects, and put the entities and events to the Timeline server via `TimelineClient`. Below is an example:
+
+```
+  // Create and start the Timeline client
+  TimelineClient client = TimelineClient.createTimelineClient();
+  client.init(conf);
+  client.start();
+
+  try {
+    TimelineDomain myDomain = new TimelineDomain();
+    myDomain.setID("MyDomain");
+    // Compose other Domain info ....
+
+    client.putDomain(myDomain);
+
+    TimelineEntity myEntity = new TimelineEntity();
+    myEntity.setDomainId(myDomain.getId());
+    myEntity.setEntityType("APPLICATION");
+    myEntity.setEntityID("MyApp1")
+    // Compose other entity info
+
+    TimelinePutResponse response = client.putEntities(entity);
+
+    
+    TimelineEvent event = new TimelineEvent();
+    event.setEventType("APP_FINISHED");
+    event.setTimestamp(System.currentTimeMillis());
+    event.addEventInfo("Exit Status", "SUCCESS");
+    // Compose other Event info ....
+
+    myEntity.addEvent(event);
+    timelineClient.putEntities(entity);
+
+  } catch (IOException e) {
+    // Handle the exception
+  } catch (YarnException e) {
+    // Handle the exception
+  }
+
+  // Stop the Timeline client
+  client.stop();
 ```
+
+  **Note** : Following are the points which needs to be observed during updating a entity.
+
+  * Domain ID should not be modified for already existing entity.
+
+  * Its advisable to have same primary filters for all updates on entity. As on modification of primary filter by subsequent updates will result in not fetching the information before the update when queried with updated primary filter.
+
+  * On modification of Primary filter value, new value will be appended with the old value.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fdef76c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_structure.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_structure.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_structure.jpg
new file mode 100644
index 0000000..dbfce25
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_structure.jpg differ


[46/50] hadoop git commit: YARN-3205. FileSystemRMStateStore should disable FileSystem Cache to avoid get a Filesystem with an old configuration. Contributed by Zhihai Xu.

Posted by zj...@apache.org.
YARN-3205. FileSystemRMStateStore should disable FileSystem Cache to avoid get a Filesystem with an old configuration. Contributed by Zhihai Xu.


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

Branch: refs/heads/YARN-2928
Commit: 3bc72cc16d8c7b8addd8f565523001dfcc32b891
Parents: fc90bf7
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Mar 18 11:53:14 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed Mar 18 11:53:19 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../recovery/FileSystemRMStateStore.java        | 22 +++++++++++++++-----
 .../recovery/TestFSRMStateStore.java            |  5 +++++
 3 files changed, 25 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3bc72cc1/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index bb752ab..c869113 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -72,6 +72,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3305. Normalize AM resource request on app submission. (Rohith Sharmaks
     via jianhe)
 
+    YARN-3205 FileSystemRMStateStore should disable FileSystem Cache to avoid
+    get a Filesystem with an old configuration. (Zhihai Xu via ozawa)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3bc72cc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.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/recovery/FileSystemRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
index 8147597..7652a07 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
@@ -84,7 +84,10 @@ public class FileSystemRMStateStore extends RMStateStore {
   protected static final String AMRMTOKEN_SECRET_MANAGER_NODE =
       "AMRMTokenSecretManagerNode";
 
+  @VisibleForTesting
   protected FileSystem fs;
+  @VisibleForTesting
+  protected Configuration fsConf;
 
   private Path rootDirPath;
   @Private
@@ -121,14 +124,23 @@ public class FileSystemRMStateStore extends RMStateStore {
     // create filesystem only now, as part of service-start. By this time, RM is
     // authenticated with kerberos so we are good to create a file-system
     // handle.
-    Configuration conf = new Configuration(getConfig());
-    conf.setBoolean("dfs.client.retry.policy.enabled", true);
+    fsConf = new Configuration(getConfig());
+    fsConf.setBoolean("dfs.client.retry.policy.enabled", true);
     String retryPolicy =
-        conf.get(YarnConfiguration.FS_RM_STATE_STORE_RETRY_POLICY_SPEC,
+        fsConf.get(YarnConfiguration.FS_RM_STATE_STORE_RETRY_POLICY_SPEC,
           YarnConfiguration.DEFAULT_FS_RM_STATE_STORE_RETRY_POLICY_SPEC);
-    conf.set("dfs.client.retry.policy.spec", retryPolicy);
+    fsConf.set("dfs.client.retry.policy.spec", retryPolicy);
+
+    String scheme = fsWorkingPath.toUri().getScheme();
+    if (scheme == null) {
+      scheme = FileSystem.getDefaultUri(fsConf).getScheme();
+    }
+    if (scheme != null) {
+      String disableCacheName = String.format("fs.%s.impl.disable.cache", scheme);
+      fsConf.setBoolean(disableCacheName, true);
+    }
 
-    fs = fsWorkingPath.getFileSystem(conf);
+    fs = fsWorkingPath.getFileSystem(fsConf);
     mkdirsWithRetries(rmDTSecretManagerRoot);
     mkdirsWithRetries(rmAppRoot);
     mkdirsWithRetries(amrmTokenSecretManagerRoot);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3bc72cc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.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/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
index d2eddd6..c5f5578 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
@@ -106,6 +106,11 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       this.store = new TestFileSystemRMStore(conf);
       Assert.assertEquals(store.getNumRetries(), 8);
       Assert.assertEquals(store.getRetryInterval(), 900L);
+      Assert.assertTrue(store.fs.getConf() == store.fsConf);
+      FileSystem previousFs = store.fs;
+      store.startInternal();
+      Assert.assertTrue(store.fs != previousFs);
+      Assert.assertTrue(store.fs.getConf() == store.fsConf);
       return store;
     }
 


[10/50] hadoop git commit: Revert "HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)" (jenkins didn't run yet)

Posted by zj...@apache.org.
Revert "HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)" (jenkins didn't run yet)

This reverts commit 5aa892ed486d42ae6b94c4866b92cd2b382ea640.


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

Branch: refs/heads/YARN-2928
Commit: 32741cf3d25d85a92e3deb11c302cc2a718d71dd
Parents: 5aa892e
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Mar 13 18:40:20 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Mar 13 18:40:20 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 -
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 23 +----
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  2 -
 .../datatransfer/DataTransferProtocol.java      |  5 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |  2 +-
 .../hdfs/protocol/datatransfer/Sender.java      |  4 +-
 .../hdfs/server/datanode/DataXceiver.java       | 95 ++++++++------------
 .../server/datanode/ShortCircuitRegistry.java   | 13 +--
 .../src/main/proto/datatransfer.proto           | 11 ---
 .../shortcircuit/TestShortCircuitCache.java     | 63 -------------
 10 files changed, 43 insertions(+), 178 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ff00b0c..c3f9367 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1177,9 +1177,6 @@ Release 2.7.0 - UNRELEASED
       HDFS-7722. DataNode#checkDiskError should also remove Storage when error
       is found. (Lei Xu via Colin P. McCabe)
 
-      HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and
-      fail to tell the DFSClient about it because of a network error (cmccabe)
-
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
index 1e915b2..ba48c79 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
-
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -71,12 +69,6 @@ import com.google.common.base.Preconditions;
 public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
 
-  public static class FailureInjector {
-    public void injectRequestFileDescriptorsFailure() throws IOException {
-      // do nothing
-    }
-  }
-
   @VisibleForTesting
   static ShortCircuitReplicaCreator
       createShortCircuitReplicaInfoCallback = null;
@@ -84,11 +76,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private final DFSClient.Conf conf;
 
   /**
-   * Injects failures into specific operations during unit tests.
-   */
-  private final FailureInjector failureInjector;
-
-  /**
    * The file name, for logging and debugging purposes.
    */
   private String fileName;
@@ -182,7 +169,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
 
   public BlockReaderFactory(DFSClient.Conf conf) {
     this.conf = conf;
-    this.failureInjector = conf.brfFailureInjector;
     this.remainingCacheTries = conf.nCachedConnRetry;
   }
 
@@ -532,12 +518,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     SlotId slotId = slot == null ? null : slot.getSlotId();
-    new Sender(out).requestShortCircuitFds(block, token, slotId, 1, true);
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1);
     DataInputStream in = new DataInputStream(peer.getInputStream());
     BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
     DomainSocket sock = peer.getDomainSocket();
-    failureInjector.injectRequestFileDescriptorsFailure();
     switch (resp.getStatus()) {
     case SUCCESS:
       byte buf[] = new byte[1];
@@ -547,13 +532,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       try {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
-        if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
-          LOG.trace("Sending receipt verification byte for slot " + slot);
-          sock.getOutputStream().write(0);
-        }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
             Time.monotonicNow(), slot);
-        return new ShortCircuitReplicaInfo(replica);
       } catch (IOException e) {
         // This indicates an error reading from disk, or a format error.  Since
         // it's not a socket communication problem, we return null rather than
@@ -565,6 +545,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
         }
       }
+      return new ShortCircuitReplicaInfo(replica);
     case ERROR_UNSUPPORTED:
       if (!resp.hasShortCircuitAccessVersion()) {
         LOG.warn("short-circuit read access is disabled for " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index f970fef..aac7b51 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -337,8 +337,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     final long shortCircuitCacheStaleThresholdMs;
 
     final long keyProviderCacheExpiryMs;
-    public BlockReaderFactory.FailureInjector brfFailureInjector =
-      new BlockReaderFactory.FailureInjector();
 
     public Conf(Configuration conf) {
       // The hdfsTimeout is currently the same as the ipc timeout 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
index 48e931d..4be42a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
@@ -138,13 +138,10 @@ public interface DataTransferProtocol {
    *                          to use no slot id.
    * @param maxVersion      Maximum version of the block data the client 
    *                          can understand.
-   * @param supportsReceiptVerification  True if the client supports
-   *                          receipt verification.
    */
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
-        throws IOException;
+      SlotId slotId, int maxVersion) throws IOException;
 
   /**
    * Release a pair of short-circuit FDs requested earlier.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
index 31bdc5e..7994027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
@@ -186,7 +186,7 @@ public abstract class Receiver implements DataTransferProtocol {
     try {
       requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
           PBHelper.convert(proto.getHeader().getToken()),
-          slotId, proto.getMaxVersion(), true);
+          slotId, proto.getMaxVersion());
     } finally {
       if (traceScope != null) traceScope.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index df69125..7fea33e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -181,8 +181,7 @@ public class Sender implements DataTransferProtocol {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
-        throws IOException {
+      SlotId slotId, int maxVersion) throws IOException {
     OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
           .setHeader(DataTransferProtoUtil.buildBaseHeader(
@@ -190,7 +189,6 @@ public class Sender implements DataTransferProtocol {
     if (slotId != null) {
       builder.setSlotId(PBHelper.convert(slotId));
     }
-    builder.setSupportsReceiptVerification(supportsReceiptVerification);
     OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 84504fb..e9547a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -22,8 +22,6 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ER
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.DO_NOT_USE_RECEIPT_VERIFICATION;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 import static org.apache.hadoop.util.Time.now;
 
@@ -293,83 +291,64 @@ class DataXceiver extends Receiver implements Runnable {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> token,
-      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
-        throws IOException {
+      SlotId slotId, int maxVersion) throws IOException {
     updateCurrentThreadName("Passing file descriptors for block " + blk);
     BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
     FileInputStream fis[] = null;
-    SlotId registeredSlotId = null;
-    boolean success = false;
     try {
+      if (peer.getDomainSocket() == null) {
+        throw new IOException("You cannot pass file descriptors over " +
+            "anything but a UNIX domain socket.");
+      }
+      if (slotId != null) {
+        boolean isCached = datanode.data.
+            isCached(blk.getBlockPoolId(), blk.getBlockId());
+        datanode.shortCircuitRegistry.registerSlot(
+            ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
+      }
       try {
-        if (peer.getDomainSocket() == null) {
-          throw new IOException("You cannot pass file descriptors over " +
-              "anything but a UNIX domain socket.");
-        }
-        if (slotId != null) {
-          boolean isCached = datanode.data.
-              isCached(blk.getBlockPoolId(), blk.getBlockId());
-          datanode.shortCircuitRegistry.registerSlot(
-              ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
-          registeredSlotId = slotId;
-        }
         fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
-        Preconditions.checkState(fis != null);
-        bld.setStatus(SUCCESS);
-        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-      } catch (ShortCircuitFdsVersionException e) {
-        bld.setStatus(ERROR_UNSUPPORTED);
-        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-        bld.setMessage(e.getMessage());
-      } catch (ShortCircuitFdsUnsupportedException e) {
-        bld.setStatus(ERROR_UNSUPPORTED);
-        bld.setMessage(e.getMessage());
-      } catch (InvalidToken e) {
-        bld.setStatus(ERROR_ACCESS_TOKEN);
-        bld.setMessage(e.getMessage());
-      } catch (IOException e) {
-        bld.setStatus(ERROR);
-        bld.setMessage(e.getMessage());
+      } finally {
+        if ((fis == null) && (slotId != null)) {
+          datanode.shortCircuitRegistry.unregisterSlot(slotId);
+        }
       }
+      bld.setStatus(SUCCESS);
+      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+    } catch (ShortCircuitFdsVersionException e) {
+      bld.setStatus(ERROR_UNSUPPORTED);
+      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+      bld.setMessage(e.getMessage());
+    } catch (ShortCircuitFdsUnsupportedException e) {
+      bld.setStatus(ERROR_UNSUPPORTED);
+      bld.setMessage(e.getMessage());
+    } catch (InvalidToken e) {
+      bld.setStatus(ERROR_ACCESS_TOKEN);
+      bld.setMessage(e.getMessage());
+    } catch (IOException e) {
+      bld.setStatus(ERROR);
+      bld.setMessage(e.getMessage());
+    }
+    try {
       bld.build().writeDelimitedTo(socketOut);
       if (fis != null) {
         FileDescriptor fds[] = new FileDescriptor[fis.length];
         for (int i = 0; i < fds.length; i++) {
           fds[i] = fis[i].getFD();
         }
-        byte buf[] = new byte[1];
-        if (supportsReceiptVerification) {
-          buf[0] = (byte)USE_RECEIPT_VERIFICATION.getNumber();
-        } else {
-          buf[0] = (byte)DO_NOT_USE_RECEIPT_VERIFICATION.getNumber();
-        }
-        DomainSocket sock = peer.getDomainSocket();
-        sock.sendFileDescriptors(fds, buf, 0, buf.length);
-        if (supportsReceiptVerification) {
-          LOG.trace("Reading receipt verification byte for " + slotId);
-          int val = sock.getInputStream().read();
-          if (val < 0) {
-            throw new EOFException();
-          }
-        } else {
-          LOG.trace("Receipt verification is not enabled on the DataNode.  " +
-                    "Not verifying " + slotId);
-        }
-        success = true;
+        byte buf[] = new byte[] { (byte)0 };
+        peer.getDomainSocket().
+          sendFileDescriptors(fds, buf, 0, buf.length);
       }
     } finally {
-      if ((!success) && (registeredSlotId != null)) {
-        LOG.info("Unregistering " + registeredSlotId + " because the " +
-            "requestShortCircuitFdsForRead operation failed.");
-        datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
-      }
       if (ClientTraceLog.isInfoEnabled()) {
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
             .getBlockPoolId());
         BlockSender.ClientTraceLog.info(String.format(
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             " blockid: %s, srvID: %s, success: %b",
-            blk.getBlockId(), dnR.getDatanodeUuid(), success));
+            blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
+          ));
       }
       if (fis != null) {
         IOUtils.cleanup(LOG, fis);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
index b32c0d1..32906f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
@@ -30,7 +30,6 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -84,7 +83,7 @@ public class ShortCircuitRegistry {
 
   private static final int SHM_LENGTH = 8192;
 
-  public static class RegisteredShm extends ShortCircuitShm
+  private static class RegisteredShm extends ShortCircuitShm
       implements DomainSocketWatcher.Handler {
     private final String clientName;
     private final ShortCircuitRegistry registry;
@@ -384,14 +383,4 @@ public class ShortCircuitRegistry {
     }
     IOUtils.closeQuietly(watcher);
   }
-
-  public static interface Visitor {
-    void accept(HashMap<ShmId, RegisteredShm> segments,
-                HashMultimap<ExtendedBlockId, Slot> slots);
-  }
-
-  @VisibleForTesting
-  public synchronized void visit(Visitor visitor) {
-    visitor.accept(segments, slots);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
index 8426198..d72bb5e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
@@ -179,12 +179,6 @@ message OpRequestShortCircuitAccessProto {
    * The shared memory slot to use, if we are using one.
    */
   optional ShortCircuitShmSlotProto slotId = 3;
-
-  /**
-   * True if the client supports verifying that the file descriptor has been
-   * sent successfully.
-   */
-  optional bool supportsReceiptVerification = 4 [default = false];
 }
 
 message ReleaseShortCircuitAccessRequestProto {
@@ -236,11 +230,6 @@ enum Status {
   IN_PROGRESS = 12;
 }
 
-enum ShortCircuitFdResponse {
-  DO_NOT_USE_RECEIPT_VERIFICATION = 0;
-  USE_RECEIPT_VERIFICATION = 1;
-}
-
 message PipelineAckProto {
   required sint64 seqno = 1;
   repeated uint32 reply = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index 7daabd0..bfa871c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -36,16 +36,13 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
-import com.google.common.collect.HashMultimap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.BlockReaderTestUtil;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -55,14 +52,11 @@ import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
-import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
-import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.RegisteredShm;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.PerDatanodeVisitorInfo;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -621,61 +615,4 @@ public class TestShortCircuitCache {
     cluster.shutdown();
     sockDir.close();
   }
-
-  public static class TestCleanupFailureInjector
-        extends BlockReaderFactory.FailureInjector {
-    @Override
-    public void injectRequestFileDescriptorsFailure() throws IOException {
-      throw new IOException("injected I/O error");
-    }
-  }
-
-  // Regression test for HDFS-7915
-  @Test(timeout=60000)
-  public void testDataXceiverCleansUpSlotsOnFailure() throws Exception {
-    BlockReaderTestUtil.enableShortCircuitShmTracing();
-    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
-    Configuration conf = createShortCircuitConf(
-        "testDataXceiverCleansUpSlotsOnFailure", sockDir);
-    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
-        1000000000L);
-    MiniDFSCluster cluster =
-        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitActive();
-    DistributedFileSystem fs = cluster.getFileSystem();
-    final Path TEST_PATH1 = new Path("/test_file1");
-    final Path TEST_PATH2 = new Path("/test_file2");
-    final int TEST_FILE_LEN = 4096;
-    final int SEED = 0xFADE1;
-    DFSTestUtil.createFile(fs, TEST_PATH1, TEST_FILE_LEN,
-        (short)1, SEED);
-    DFSTestUtil.createFile(fs, TEST_PATH2, TEST_FILE_LEN,
-        (short)1, SEED);
-
-    // The first read should allocate one shared memory segment and slot.
-    DFSTestUtil.readFileBuffer(fs, TEST_PATH1);
-
-    // The second read should fail, and we should only have 1 segment and 1 slot
-    // left.
-    fs.getClient().getConf().brfFailureInjector =
-        new TestCleanupFailureInjector();
-    try {
-      DFSTestUtil.readFileBuffer(fs, TEST_PATH2);
-    } catch (Throwable t) {
-      GenericTestUtils.assertExceptionContains("TCP reads were disabled for " +
-          "testing, but we failed to do a non-TCP read.", t);
-    }
-    ShortCircuitRegistry registry =
-      cluster.getDataNodes().get(0).getShortCircuitRegistry();
-    registry.visit(new ShortCircuitRegistry.Visitor() {
-      @Override
-      public void accept(HashMap<ShmId, RegisteredShm> segments,
-                         HashMultimap<ExtendedBlockId, Slot> slots) {
-        Assert.assertEquals(1, segments.size());
-        Assert.assertEquals(1, slots.size());
-      }
-    });
-    cluster.shutdown();
-    sockDir.close();
-  }
 }


[45/50] hadoop git commit: HADOOP-11722. Some Instances of Services using ZKDelegationTokenSecretManager go down when old token cannot be deleted. Contributed by Arun Suresh.

Posted by zj...@apache.org.
HADOOP-11722. Some Instances of Services using ZKDelegationTokenSecretManager go down when old token cannot be deleted. Contributed by Arun Suresh.


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

Branch: refs/heads/YARN-2928
Commit: fc90bf7b27cc20486f2806670a14fd7d654b0a31
Parents: 968425e
Author: Aaron T. Myers <at...@apache.org>
Authored: Tue Mar 17 19:41:36 2015 -0700
Committer: Aaron T. Myers <at...@apache.org>
Committed: Tue Mar 17 19:41:36 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  4 ++++
 .../ZKDelegationTokenSecretManager.java         | 21 ++++++++++++++++++--
 2 files changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc90bf7b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 3817054..a6bd68d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1111,6 +1111,10 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11720. [JDK8] Fix javadoc errors caused by incorrect or illegal
     tags in hadoop-tools. (Akira AJISAKA via ozawa)
 
+    HADOOP-11722. Some Instances of Services using
+    ZKDelegationTokenSecretManager go down when old token cannot be deleted.
+    (Arun Suresh via atm)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc90bf7b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
index ec522dcf..73c3ab8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.ZooDefs.Perms;
 import org.apache.zookeeper.client.ZooKeeperSaslClient;
 import org.apache.zookeeper.data.ACL;
@@ -709,7 +710,15 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
     try {
       if (zkClient.checkExists().forPath(nodeRemovePath) != null) {
         while(zkClient.checkExists().forPath(nodeRemovePath) != null){
-          zkClient.delete().guaranteed().forPath(nodeRemovePath);
+          try {
+            zkClient.delete().guaranteed().forPath(nodeRemovePath);
+          } catch (NoNodeException nne) {
+            // It is possible that the node might be deleted between the
+            // check and the actual delete.. which might lead to an
+            // exception that can bring down the daemon running this
+            // SecretManager
+            LOG.debug("Node already deleted by peer " + nodeRemovePath);
+          }
         }
       } else {
         LOG.debug("Attempted to delete a non-existing znode " + nodeRemovePath);
@@ -761,7 +770,15 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
     try {
       if (zkClient.checkExists().forPath(nodeRemovePath) != null) {
         while(zkClient.checkExists().forPath(nodeRemovePath) != null){
-          zkClient.delete().guaranteed().forPath(nodeRemovePath);
+          try {
+            zkClient.delete().guaranteed().forPath(nodeRemovePath);
+          } catch (NoNodeException nne) {
+            // It is possible that the node might be deleted between the
+            // check and the actual delete.. which might lead to an
+            // exception that can bring down the daemon running this
+            // SecretManager
+            LOG.debug("Node already deleted by peer " + nodeRemovePath);
+          }
         }
       } else {
         LOG.debug("Attempted to remove a non-existing znode " + nodeRemovePath);